From bb995c74ccfaf9ca28edebbd55fba5d32a4739bb Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 13 Jun 2024 16:30:38 +0200 Subject: [PATCH 001/178] Revert "Revert "S3: reduce retires time for queries, increase retries count for backups"" This reverts commit 7683f06188d8dc901bd912c4ace935a4b3f498e2. --- src/Backups/BackupIO_S3.cpp | 6 +++--- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/IO/S3/Client.h | 2 +- .../integration/test_mask_sensitive_info/configs/users.xml | 1 + 5 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 92f086295a0..1ea59c1d38b 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -54,9 +54,9 @@ namespace S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, context->getRemoteHostFilter(), - static_cast(global_settings.s3_max_redirects), - static_cast(global_settings.s3_retry_attempts), - global_settings.enable_s3_requests_logging, + static_cast(local_settings.s3_max_redirects), + static_cast(local_settings.backup_restore_s3_retry_attempts), + local_settings.enable_s3_requests_logging, /* for_disk_s3 = */ false, request_settings.get_request_throttler, request_settings.put_request_throttler, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b3e83092a77..7f99243e285 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -517,6 +517,7 @@ class IColumn; M(UInt64, backup_restore_keeper_value_max_size, 1048576, "Maximum size of data of a [Zoo]Keeper's node during backup", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup or restore", 0) \ M(UInt64, backup_restore_batch_size_for_keeper_multi, 1000, "Maximum size of batch for multi request to [Zoo]Keeper during backup or restore", 0) \ + M(UInt64, backup_restore_s3_retry_attempts, 1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore.", 0) \ M(UInt64, max_backup_bandwidth, 0, "The maximum read speed in bytes per second for particular backup on server. Zero means unlimited.", 0) \ \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 69bc8c5d207..b447421671e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -115,6 +115,7 @@ static const std::map& error, long attemptedRetries) const override; diff --git a/tests/integration/test_mask_sensitive_info/configs/users.xml b/tests/integration/test_mask_sensitive_info/configs/users.xml index f129a5bb3e3..f767216e907 100644 --- a/tests/integration/test_mask_sensitive_info/configs/users.xml +++ b/tests/integration/test_mask_sensitive_info/configs/users.xml @@ -2,6 +2,7 @@ 5 + 5 From 13e00d4f8b621c800b286e383b20765e669d5163 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 13 Jun 2024 17:40:42 +0200 Subject: [PATCH 002/178] put backup_restore_s3_retry_attempts to 24.6 --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b447421671e..496c7a46798 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,6 +105,7 @@ static const std::map Date: Fri, 14 Jun 2024 00:40:44 +0800 Subject: [PATCH 003/178] feat: support accept_invalid_certificate in client's config --- programs/client/Client.cpp | 7 +- ...9_clickhouse_accept_invalid_certificate.sh | 107 ++++++++++++++++++ .../01889_clickhouse_client_config_format.sh | 31 +++-- .../02550_client_connections_credentials.sh | 13 ++- 4 files changed, 138 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index efe23d57478..f5e86b756fb 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -178,6 +178,9 @@ void Client::parseConnectionsCredentials(Poco::Util::AbstractConfiguration & con history_file = home_path + "/" + history_file.substr(1); config.setString("history_file", history_file); } + if (config.has(prefix + "accept-invalid-certificate")) { + config.setString("accept-invalid-certificate", prefix + "accept-invalid-certificate"); + } } if (!connection_name.empty() && !connection_found) @@ -721,7 +724,7 @@ bool Client::processWithFuzzing(const String & full_query) } if (auto *q = orig_ast->as()) { - if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet() == "kusto") + if (auto *set_dialect = q->changes.tryGet("dialect"); set_dialect && set_dialect->safeGet() == "kusto") return true; } @@ -1102,7 +1105,7 @@ void Client::processOptions(const OptionsDescription & options_description, config().setBool("no-warnings", true); if (options.count("fake-drop")) config().setString("ignore_drop_queries_probability", "1"); - if (options.count("accept-invalid-certificate")) + if (options.count("accept-invalid-certificate") || config().has("accept-invalid-certificate")) { config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler"); config().setString("openSSL.client.verificationMode", "none"); diff --git a/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh b/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh new file mode 100644 index 00000000000..00378abdb8e --- /dev/null +++ b/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh @@ -0,0 +1,107 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# use $CLICKHOUSE_DATABASE so that clickhouse-test will replace it with default to match .reference +config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE +xml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.xml +XML_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.XML +conf_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.conf +yml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yml +yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml +ini_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.ini + +function cleanup() { + rm "${config:?}" + rm "${xml_config:?}" + rm "${XML_config:?}" + rm "${conf_config:?}" + rm "${yml_config:?}" + rm "${yaml_config:?}" + rm "${ini_config:?}" +} +trap cleanup EXIT + +cat >"$config" < + + + + RejectCertificateHandler + + + + +EOL +cat >"$conf_config" < + + + + RejectCertificateHandler + + + + +EOL +cat >"$xml_config" < + + + + RejectCertificateHandler + + + + +EOL +cat >"$XML_config" < + + + + RejectCertificateHandler + + + + +EOL +cat >"$yml_config" <"$yaml_config" <"$ini_config" <RejectCertificateHandler') -q "select getSetting('invalidCertificateHandler')" + +echo 'yml' +$CLICKHOUSE_CLIENT --config "$yml_config" -q "select getSetting('invalidCertificateHandler')" +echo 'yaml' +$CLICKHOUSE_CLIENT --config "$yaml_config" -q "select getSetting('invalidCertificateHandler')" +echo 'ini' +$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('invalidCertificateHandler')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 4e970f17d3a..4af4e27653c 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -14,45 +14,44 @@ yml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yml yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml ini_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.ini -function cleanup() -{ - rm "${config:?}" - rm "${xml_config:?}" - rm "${XML_config:?}" - rm "${conf_config:?}" - rm "${yml_config:?}" - rm "${yaml_config:?}" - rm "${ini_config:?}" +function cleanup() { + rm "${config:?}" + rm "${xml_config:?}" + rm "${XML_config:?}" + rm "${conf_config:?}" + rm "${yml_config:?}" + rm "${yaml_config:?}" + rm "${ini_config:?}" } trap cleanup EXIT -cat > "$config" <"$config" < 2 EOL -cat > "$conf_config" <"$conf_config" < 2 EOL -cat > "$xml_config" <"$xml_config" < 2 EOL -cat > "$XML_config" <"$XML_config" < 2 EOL -cat > "$yml_config" <"$yml_config" < "$yaml_config" <"$yaml_config" < "$ini_config" <"$ini_config" < $CONFIG <$CONFIG < $TEST_HOST $TEST_PORT @@ -66,12 +66,21 @@ cat > $CONFIG <$TEST_HOST /no/such/dir/.history + + + test_accept_invalid_certificate + $TEST_HOST + $CLICKHOUSE_DATABASE + + RejectCertificateHandler + + EOL CONFIG_ROOT_OVERRIDES=$CLICKHOUSE_TMP/client_user_pass.xml -cat > $CONFIG_ROOT_OVERRIDES <$CONFIG_ROOT_OVERRIDES < $TEST_HOST $TEST_PORT From 6e747ae5ca0c575e6c8e6f46392fe0f06675fcc0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 20 Jun 2024 19:09:15 +0200 Subject: [PATCH 004/178] backup_restore_s3_retry_attempts moved to 24.7 --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 0370368a8f8..5f332f6301e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static const std::map settings_changes_history = { + {"24.7", {{"backup_restore_s3_retry_attempts", 0, 1000, "A new setting."}}}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, @@ -114,7 +115,6 @@ static const std::map Date: Thu, 20 Jun 2024 22:01:16 +0000 Subject: [PATCH 005/178] Update libunwind to 18.1.7 --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index d6a01c46327..02f17ec85cd 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit d6a01c46327e56fd86beb8aaa31591fcd9a6b7df +Subproject commit 02f17ec85cd6b28540d0a4b42f32c2f8a7c8d79d From 424224b2023fbdc72993f83e19f0dd7c6ff0b84b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 22:05:16 +0000 Subject: [PATCH 006/178] Set _LIBUNWIND_REMEMBER_STACK_ALLOC explicitly just in case --- contrib/libunwind-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 37a2f29afcf..8d99d729010 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -32,6 +32,7 @@ set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake") target_include_directories(unwind SYSTEM BEFORE PUBLIC $) target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1) +target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_REMEMBER_STACK_ALLOC=1) # NOTE: from this macros sizeof(unw_context_t)/sizeof(unw_cursor_t) is depends, so it should be set always target_compile_definitions(unwind PUBLIC -D_LIBUNWIND_IS_NATIVE_ONLY) From 237de4e775ba4080087496303c350cddbebc9043 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 14 Jun 2024 12:31:52 +0000 Subject: [PATCH 007/178] add feature --- .../system-tables/detached_tables.md | 36 + src/Databases/DatabaseAtomic.cpp | 20 +- src/Databases/DatabaseAtomic.h | 7 +- src/Databases/DatabaseOnDisk.cpp | 3 + src/Databases/DatabaseOrdinary.cpp | 37 +- src/Databases/DatabaseOrdinary.h | 3 + src/Databases/DatabasesCommon.cpp | 54 +- src/Databases/DatabasesCommon.h | 6 +- src/Databases/IDatabase.h | 63 +- src/Storages/IStorage_fwd.h | 4 +- src/Storages/System/ReadFromSystemTables.cpp | 138 ++++ src/Storages/System/ReadFromSystemTables.h | 47 ++ .../System/StorageSystemDetachedTables.cpp | 56 ++ .../System/StorageSystemDetachedTables.h | 32 + src/Storages/System/StorageSystemTables.cpp | 761 ++---------------- src/Storages/System/TablesBlockSource.cpp | 569 +++++++++++++ src/Storages/System/TablesBlockSource.h | 96 +++ src/Storages/System/attachSystemTables.cpp | 2 + .../test_system_detached_tables/__init__.py | 0 .../test_system_detached_tables/test.py | 38 + .../03172_system_detached_tables.reference | 5 + .../03172_system_detached_tables.sql | 28 + 22 files changed, 1289 insertions(+), 716 deletions(-) create mode 100644 docs/en/operations/system-tables/detached_tables.md create mode 100644 src/Storages/System/ReadFromSystemTables.cpp create mode 100644 src/Storages/System/ReadFromSystemTables.h create mode 100644 src/Storages/System/StorageSystemDetachedTables.cpp create mode 100644 src/Storages/System/StorageSystemDetachedTables.h create mode 100644 src/Storages/System/TablesBlockSource.cpp create mode 100644 src/Storages/System/TablesBlockSource.h create mode 100644 tests/integration/test_system_detached_tables/__init__.py create mode 100644 tests/integration/test_system_detached_tables/test.py create mode 100644 tests/queries/0_stateless/03172_system_detached_tables.reference create mode 100644 tests/queries/0_stateless/03172_system_detached_tables.sql diff --git a/docs/en/operations/system-tables/detached_tables.md b/docs/en/operations/system-tables/detached_tables.md new file mode 100644 index 00000000000..d86e58d2899 --- /dev/null +++ b/docs/en/operations/system-tables/detached_tables.md @@ -0,0 +1,36 @@ +--- +slug: /en/operations/system-tables/detached_tables +--- +# detached_tables + +Contains information of each detached table. + +Columns: + +- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in. + +- `name` ([String](../../sql-reference/data-types/string.md)) — Table name. + +- `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database). + +- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY. + +- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system. + + +**Example** + +```sql +SELECT * FROM system.detached_tables FORMAT Vertical; +``` + +```text +Row 1: +────── +database: base +name: t1 +uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736 +data_paths: ['/var/lib/clickhouse/store/81b/81b1c20a-b7c6-4116-a2ce-7583fb6b6736/'] +metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql +is_permanently: 1 +``` diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ccab72cfbae..ddc30627cde 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -37,8 +37,10 @@ namespace ErrorCodes class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator { public: - explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) - : DatabaseTablesSnapshotIterator(std::move(base)) {} + explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base) noexcept + : DatabaseTablesSnapshotIterator(std::move(base)) + { + } UUID uuid() const override { return table()->getStorageID().uuid; } }; @@ -101,6 +103,8 @@ void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, auto table_id = table->getStorageID(); assertDetachedTableNotInUse(table_id.uuid); DatabaseOrdinary::attachTableUnlocked(name, table); + detached_tables.erase(table_id.uuid); + table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } @@ -108,11 +112,11 @@ StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & { DetachedTables not_in_use; std::lock_guard lock(mutex); - auto table = DatabaseOrdinary::detachTableUnlocked(name); + auto detached_table = DatabaseOrdinary::detachTableUnlocked(name); table_name_to_path.erase(name); - detached_tables.emplace(table->getStorageID().uuid, table); + detached_tables.emplace(detached_table->getStorageID().uuid, detached_table); not_in_use = cleanupDetachedTables(); - return table; + return detached_table; } void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool sync) @@ -433,6 +437,12 @@ DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::Fil return std::make_unique(std::move(typeid_cast(*base_iter))); } +DatabaseDetachedTablesSnapshotIteratorPtr DatabaseAtomic::getDetachedTablesIterator( + ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name, const bool skip_not_loaded) const +{ + return DatabaseOrdinary::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded); +} + UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const { if (auto table = tryGetTable(table_name, getContext())) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index b59edd479ba..47afc4dc5ef 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -1,7 +1,8 @@ #pragma once -#include #include +#include +#include namespace DB @@ -48,6 +49,9 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + DatabaseDetachedTablesSnapshotIteratorPtr + getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; @@ -81,6 +85,7 @@ protected: //TODO store path in DatabaseWithOwnTables::tables using NameToPathMap = std::unordered_map; + NameToPathMap table_name_to_path TSA_GUARDED_BY(mutex); DetachedTables detached_tables TSA_GUARDED_BY(mutex); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index b8154372116..25e5347e017 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -307,6 +307,9 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri try { FS::createFile(detached_permanently_flag); + + std::lock_guard lock(mutex); + snapshot_detached_tables.at(table_name).is_permanently = true; } catch (Exception & e) { diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 10a8e06e8f0..046cbfa40be 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -187,7 +187,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables size_t prev_tables_count = metadata.parsed_tables.size(); size_t prev_total_dictionaries = metadata.total_dictionaries; - auto process_metadata = [&metadata, is_startup, this](const String & file_name) + auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name) mutable { fs::path path(getMetadataPath()); fs::path file_path(file_name); @@ -195,7 +195,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables try { - auto ast = parseQueryFromMetadata(log, getContext(), full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); + auto ast = parseQueryFromMetadata(log, local_context, full_path.string(), /*throw_on_error*/ true, /*remove_empty*/ false); if (ast) { FunctionNameNormalizer::visit(ast.get()); @@ -226,6 +226,33 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); permanently_detached_tables.push_back(table_name); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); + + // @TODO refactoring + auto parsed_table_metadata = ParsedTableMetadata{full_path.string(), ast}; + const auto & query = parsed_table_metadata.ast->as(); + + std::lock_guard lock(mutex); + + auto [detached_table_name, table] = createTableFromAST( + query, + database_name, + getTableDataPath(query), + std::const_pointer_cast(local_context), + LoadingStrictnessLevel::CREATE); + + const auto storage_id = table->getStorageID(); + + SnapshotDetachedTable snapshot_detached_table; + snapshot_detached_table.detabase = storage_id.getDatabaseName(); + snapshot_detached_table.table = detached_table_name; + snapshot_detached_table.uuid = storage_id.uuid; + snapshot_detached_table.is_permanently = true; + snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); + + + snapshot_detached_tables.emplace(detached_table_name, std::move(snapshot_detached_table)); + + LOG_TRACE(log, "Add detached table {} to system.detached_tables", detached_table_name); return; } @@ -487,6 +514,12 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded); } +DatabaseDetachedTablesSnapshotIteratorPtr DatabaseOrdinary::getDetachedTablesIterator( + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const +{ + return DatabaseWithOwnTablesBase::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded); +} + Strings DatabaseOrdinary::getAllTableNames(ContextPtr) const { std::set unique_names; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index ef00ac8fdfa..d2891147922 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -57,6 +57,9 @@ public: LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + Strings getAllTableNames(ContextPtr context) const override; void alterTable( diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fd38a31da5c..312c0a4477f 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -2,12 +2,9 @@ #include #include -#include -#include -#include -#include #include #include +#include #include #include #include @@ -16,6 +13,10 @@ #include #include #include +#include +#include +#include +#include namespace DB @@ -237,6 +238,35 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPt return std::make_unique(std::move(filtered_tables), database_name); } +DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetachedTablesIterator( + ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const +{ + std::lock_guard lock(mutex); + if (!filter_by_table_name) + return std::make_unique(snapshot_detached_tables); + + SnapshotDetachedTables filtered_tables; + for (const auto & [table_name, storage] : tables) + if (filter_by_table_name(table_name)) + { + SnapshotDetachedTable snapshot_detached_table; + snapshot_detached_table.detabase = storage->getStorageID().getDatabaseName(); + snapshot_detached_table.table = table_name; + if (storage->getStorageID().hasUUID()) + { + snapshot_detached_table.uuid = storage->getStorageID().uuid; + } + + snapshot_detached_table.is_permanently = false; + snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); + + filtered_tables.emplace(table_name, std::move(snapshot_detached_table)); + } + + + return std::make_unique(std::move(filtered_tables)); +} + bool DatabaseWithOwnTablesBase::empty() const { std::lock_guard lock(mutex); @@ -258,6 +288,19 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(database_name), backQuote(table_name)); res = it->second; + + SnapshotDetachedTable snapshot_detached_table; + snapshot_detached_table.detabase = it->second->getStorageID().getDatabaseName(); + snapshot_detached_table.table = it->first; + if (it->second->getStorageID().hasUUID()) + { + snapshot_detached_table.uuid = it->second->getStorageID().uuid; + } + snapshot_detached_table.is_permanently = false; + snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); + + snapshot_detached_tables.emplace(it->first, std::move(snapshot_detached_table)); + tables.erase(it); res->is_detached = true; CurrentMetrics::sub(getAttachedCounterForStorage(res), 1); @@ -298,6 +341,8 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName()); } + snapshot_detached_tables.erase(table_name); + /// It is important to reset is_detached here since in case of RENAME in /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; @@ -333,6 +378,7 @@ void DatabaseWithOwnTablesBase::shutdown() std::lock_guard lock(mutex); tables.clear(); + snapshot_detached_tables.clear(); } DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 2eecf8a564f..1ca49e90c23 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -37,6 +37,9 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + DatabaseDetachedTablesSnapshotIteratorPtr + getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + std::vector> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override; void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr restore_coordination, UInt64 timeout_ms) override; @@ -46,12 +49,13 @@ public: protected: Tables tables TSA_GUARDED_BY(mutex); + SnapshotDetachedTables snapshot_detached_tables TSA_GUARDED_BY(mutex); LoggerPtr log; DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context); void attachTableUnlocked(const String & table_name, const StoragePtr & table) TSA_REQUIRES(mutex); - StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); + StoragePtr detachTableUnlocked(const String & table_name) TSA_REQUIRES(mutex); StoragePtr getTableUnlocked(const String & table_name) const TSA_REQUIRES(mutex); StoragePtr tryGetTableNoWait(const String & table_name) const; }; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index b00f2fe4baf..7c6b3b3004c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -5,20 +5,22 @@ #include #include #include +#include +#include #include #include -#include #include +#include #include #include -#include #include #include +#include #include #include +#include #include -#include namespace DB @@ -110,6 +112,55 @@ public: using DatabaseTablesIteratorPtr = std::unique_ptr; +struct SnapshotDetachedTable final +{ + String detabase; + String table; + UUID uuid = UUIDHelpers::Nil; + String metadata_path; + bool is_permanently{false}; +}; + +class DatabaseDetachedTablesSnapshotIterator +{ +private: + SnapshotDetachedTables snapshot; + SnapshotDetachedTables::iterator it; + +protected: + DatabaseDetachedTablesSnapshotIterator(DatabaseDetachedTablesSnapshotIterator && other) noexcept + { + size_t idx = std::distance(other.snapshot.begin(), other.it); + std::swap(snapshot, other.snapshot); + other.it = other.snapshot.end(); + it = snapshot.begin(); + std::advance(it, idx); + } + +public: + explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin()) { } + + explicit DatabaseDetachedTablesSnapshotIterator(SnapshotDetachedTables && tables_) : snapshot(std::move(tables_)), it(snapshot.begin()) + { + } + + void next() { ++it; } + + bool isValid() const { return it != snapshot.end(); } + + String database() const { return it->second.detabase; } + + String table() const { return it->second.table; } + + UUID uuid() const { return it->second.uuid; } + + String metadataPath() const { return it->second.metadata_path; } + + bool isPermanently() const { return it->second.is_permanently; } +}; + +using DatabaseDetachedTablesSnapshotIteratorPtr = std::unique_ptr; + /** Database engine. * It is responsible for: @@ -232,6 +283,12 @@ public: /// Wait for all tables to be loaded and started up. If `skip_not_loaded` is true, then not yet loaded or not yet started up (at the moment of iterator creation) tables are excluded. virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}, bool skip_not_loaded = false) const = 0; /// NOLINT + virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( + ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no get detached tables for Database{}", getEngineName()); + } + /// Returns list of table names. virtual Strings getAllTableNames(ContextPtr context) const { diff --git a/src/Storages/IStorage_fwd.h b/src/Storages/IStorage_fwd.h index b9243b029b0..4cbc586a745 100644 --- a/src/Storages/IStorage_fwd.h +++ b/src/Storages/IStorage_fwd.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -9,9 +10,10 @@ namespace DB { class IStorage; +struct SnapshotDetachedTable; using ConstStoragePtr = std::shared_ptr; using StoragePtr = std::shared_ptr; using Tables = std::map; - +using SnapshotDetachedTables = std::map; } diff --git a/src/Storages/System/ReadFromSystemTables.cpp b/src/Storages/System/ReadFromSystemTables.cpp new file mode 100644 index 00000000000..6e9c6a869d1 --- /dev/null +++ b/src/Storages/System/ReadFromSystemTables.cpp @@ -0,0 +1,138 @@ +#include "ReadFromSystemTables.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables( + const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool need_detached_tables) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + if (need_detached_tables) + { + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + else + { + auto table_it = database->getTablesIterator(context); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +} + +ReadFromSystemTables::ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_, + const bool need_detached_tables_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , need_detached_tables(need_detached_tables_) +{ +} + +void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context, need_detached_tables); +} + +void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + if (need_detached_tables) + { + pipeline.init(createPipe()); + } + else + { + pipeline.init(createPipe()); + } +} +} diff --git a/src/Storages/System/ReadFromSystemTables.h b/src/Storages/System/ReadFromSystemTables.h new file mode 100644 index 00000000000..f8280ffcc8d --- /dev/null +++ b/src/Storages/System/ReadFromSystemTables.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ReadFromSystemTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_, + bool need_detached_tables); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + const bool need_detached_tables; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; + + template + Pipe createPipe() + { + return Pipe(std::make_shared( + std::move(columns_mask), + getOutputStream().header, + max_block_size, + std::move(filtered_databases_column), + std::move(filtered_tables_column), + context)); + } +}; +} diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp new file mode 100644 index 00000000000..1596ac6a046 --- /dev/null +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -0,0 +1,56 @@ +#include "StorageSystemDetachedTables.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_) +{ + StorageInMemoryMetadata storage_metadata; + + auto description = ColumnsDescription{ + ColumnDescription{"database", std::make_shared(), "The name of the database the table is in."}, + ColumnDescription{"name", std::make_shared(), "Table name."}, + ColumnDescription{"uuid", std::make_shared(), "Table uuid (Atomic database)."}, + ColumnDescription{"metadata_path", std::make_shared(), "Path to the table metadata in the file system."}, + ColumnDescription{"is_permanently", std::make_shared(), "Table was detached permanently."}, + }; + + storage_metadata.setColumns(std::move(description)); + + setInMemoryMetadata(storage_metadata); +} + +void StorageSystemDetachedTables::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t max_block_size, + size_t /*num_streams*/) +{ + storage_snapshot->check(column_names); + auto sample_block = storage_snapshot->metadata->getSampleBlock(); + + auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size, true); + + query_plan.addStep(std::move(reading)); +} +} diff --git a/src/Storages/System/StorageSystemDetachedTables.h b/src/Storages/System/StorageSystemDetachedTables.h new file mode 100644 index 00000000000..cd042f51eaa --- /dev/null +++ b/src/Storages/System/StorageSystemDetachedTables.h @@ -0,0 +1,32 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + +/** Implements the system table `detached_tables`, which allows you to get information about detached tables. + */ +class StorageSystemDetachedTables final : public IStorage +{ +public: + explicit StorageSystemDetachedTables(const StorageID & table_id_); + + std::string getName() const override { return "SystemDetachedTables"; } + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + bool isSystemStorage() const override { return true; } +}; +} diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..5a8d6315d57 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,29 +1,31 @@ +#include "StorageSystemTables.h" + +#include #include -#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include #include #include #include #include #include -#include +#include +#include +#include +#include +#include +#include +#include #include @@ -32,8 +34,7 @@ namespace DB { -StorageSystemTables::StorageSystemTables(const StorageID & table_id_) - : IStorage(table_id_) +StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -46,9 +47,13 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) {"data_paths", std::make_shared(std::make_shared()), "Paths to the table data in the file systems."}, {"metadata_path", std::make_shared(), "Path to the table metadata in the file system."}, {"metadata_modification_time", std::make_shared(), "Time of latest modification of the table metadata."}, - {"metadata_version", std::make_shared(), "Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table."}, + {"metadata_version", + std::make_shared(), + "Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table."}, {"dependencies_database", std::make_shared(std::make_shared()), "Database dependencies."}, - {"dependencies_table", std::make_shared(std::make_shared()), "Table dependencies (materialized views the current table)."}, + {"dependencies_table", + std::make_shared(std::make_shared()), + "Table dependencies (materialized views the current table)."}, {"create_table_query", std::make_shared(), "The query that was used to create the table."}, {"engine_full", std::make_shared(), "Parameters of the table engine."}, {"as_select", std::make_shared(), "SELECT query for view."}, @@ -57,676 +62,54 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) {"primary_key", std::make_shared(), "The primary key expression specified in the table."}, {"sampling_key", std::make_shared(), "The sampling key expression specified in the table."}, {"storage_policy", std::make_shared(), "The storage policy."}, - {"total_rows", std::make_shared(std::make_shared()), - "Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise NULL (including underlying Buffer table)." - }, - {"total_bytes", std::make_shared(std::make_shared()), - "Total number of bytes, if it is possible to quickly determine exact number " - "of bytes for the table on storage, otherwise NULL (does not includes any underlying storage). " - "If the table stores data on disk, returns used space on disk (i.e. compressed). " - "If the table stores data in memory, returns approximated number of used bytes in memory." - }, - {"total_bytes_uncompressed", std::make_shared(std::make_shared()), - "Total number of uncompressed bytes, if it's possible to quickly determine the exact number " - "of bytes from the part checksums for the table on storage, otherwise NULL (does not take underlying storage (if any) into account)." - }, + {"total_rows", + std::make_shared(std::make_shared()), + "Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise NULL (including " + "underlying Buffer table)."}, + {"total_bytes", + std::make_shared(std::make_shared()), + "Total number of bytes, if it is possible to quickly determine exact number " + "of bytes for the table on storage, otherwise NULL (does not includes any underlying storage). " + "If the table stores data on disk, returns used space on disk (i.e. compressed). " + "If the table stores data in memory, returns approximated number of used bytes in memory."}, + {"total_bytes_uncompressed", + std::make_shared(std::make_shared()), + "Total number of uncompressed bytes, if it's possible to quickly determine the exact number " + "of bytes from the part checksums for the table on storage, otherwise NULL (does not take underlying storage (if any) into " + "account)."}, {"parts", std::make_shared(std::make_shared()), "The total number of parts in this table."}, - {"active_parts", std::make_shared(std::make_shared()), "The number of active parts in this table."}, - {"total_marks", std::make_shared(std::make_shared()), "The total number of marks in all parts in this table."}, - {"lifetime_rows", std::make_shared(std::make_shared()), - "Total number of rows INSERTed since server start (only for Buffer tables)." - }, - {"lifetime_bytes", std::make_shared(std::make_shared()), - "Total number of bytes INSERTed since server start (only for Buffer tables)." - }, + {"active_parts", + std::make_shared(std::make_shared()), + "The number of active parts in this table."}, + {"total_marks", + std::make_shared(std::make_shared()), + "The total number of marks in all parts in this table."}, + {"lifetime_rows", + std::make_shared(std::make_shared()), + "Total number of rows INSERTed since server start (only for Buffer tables)."}, + {"lifetime_bytes", + std::make_shared(std::make_shared()), + "Total number of bytes INSERTed since server start (only for Buffer tables)."}, {"comment", std::make_shared(), "The comment for the table."}, - {"has_own_data", std::make_shared(), - "Flag that indicates whether the table itself stores some data on disk or only accesses some other source." - }, - {"loading_dependencies_database", std::make_shared(std::make_shared()), - "Database loading dependencies (list of objects which should be loaded before the current object)." - }, - {"loading_dependencies_table", std::make_shared(std::make_shared()), - "Table loading dependencies (list of objects which should be loaded before the current object)." - }, - {"loading_dependent_database", std::make_shared(std::make_shared()), - "Dependent loading database." - }, - {"loading_dependent_table", std::make_shared(std::make_shared()), - "Dependent loading table." - }, + {"has_own_data", + std::make_shared(), + "Flag that indicates whether the table itself stores some data on disk or only accesses some other source."}, + {"loading_dependencies_database", + std::make_shared(std::make_shared()), + "Database loading dependencies (list of objects which should be loaded before the current object)."}, + {"loading_dependencies_table", + std::make_shared(std::make_shared()), + "Table loading dependencies (list of objects which should be loaded before the current object)."}, + {"loading_dependent_database", std::make_shared(std::make_shared()), "Dependent loading database."}, + {"loading_dependent_table", std::make_shared(std::make_shared()), "Dependent loading table."}, }; - description.setAliases({ - {"table", std::make_shared(), "name"} - }); + description.setAliases({{"table", std::make_shared(), "name"}}); storage_metadata.setColumns(std::move(description)); setInMemoryMetadata(storage_metadata); } - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample { - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") - }; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - - Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - - -class TablesBlockSource : public ISource -{ -public: - TablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = tables_->size(); - tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - tables.insert(tables_->getDataAt(idx).toString()); - } - - String getName() const override { return "Tables"; } - -protected: - Chunk generate() override - { - if (done) - return {}; - - MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - size_t rows_count = 0; - while (rows_count < max_block_size) - { - if (tables_it && !tables_it->isValid()) - ++database_idx; - - while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - { - /// Database was deleted just now or the user has no access. - ++database_idx; - continue; - } - - break; - } - - /// This is for temporary tables. They are output in single block regardless to max_block_size. - if (database_idx >= databases->size()) - { - if (context->hasSessionContext()) - { - Tables external_tables = context->getSessionContext()->getExternalTables(); - - for (auto & table : external_tables) - { - size_t src_index = 0; - size_t res_index = 0; - - // database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // name - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.first); - - // uuid - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getStorageID().uuid); - - // engine - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - // is_temporary - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(1u); - - // data_paths - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_path - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_modification_time - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_version - // Temporary tables does not support replication - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_table - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // create_table_query - if (columns_mask[src_index++]) - { - auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); - ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - } - - // engine_full - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - const auto & settings = context->getSettingsRef(); - while (src_index < columns_mask.size()) - { - // total_rows - if (src_index == 19 && columns_mask[src_index]) - { - if (auto total_rows = table.second->totalRows(settings)) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - // total_bytes - else if (src_index == 20 && columns_mask[src_index]) - { - if (auto total_bytes = table.second->totalBytes(settings)) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - /// Fill the rest columns with defaults - else if (columns_mask[src_index]) - res_columns[res_index++]->insertDefault(); - src_index++; - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - done = true; - return Chunk(std::move(res_columns), num_rows); - } - - const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(context); - - const bool need_table = needTable(database, getPort().getHeader()); - - for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) - { - auto table_name = tables_it->name(); - if (!tables.contains(table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) - continue; - - StoragePtr table = nullptr; - TableLockHolder lock; - if (need_table) - { - table = tables_it->table(); - if (!table) - // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) - continue; - - /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) - /// and it's not protected internally by other mutexes - static const size_t DATA_PATHS_INDEX = 5; - if (columns_mask[DATA_PATHS_INDEX]) - { - lock = table->tryLockForShare(context->getCurrentQueryId(), - context->getSettingsRef().lock_acquire_timeout); - if (!lock) - // Table was dropped while acquiring the lock, skipping table - continue; - } - } - ++rows_count; - - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(tables_it->uuid()); - - if (columns_mask[src_index++]) - { - chassert(table != nullptr); - res_columns[res_index++]->insert(table->getName()); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(0u); // is_temporary - - if (columns_mask[src_index++]) - { - chassert(lock != nullptr); - Array table_paths_array; - auto paths = table->getDataPaths(); - table_paths_array.reserve(paths.size()); - for (const String & path : paths) - table_paths_array.push_back(path); - res_columns[res_index++]->insert(table_paths_array); - /// We don't need the lock anymore - lock = nullptr; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); - - StorageMetadataPtr metadata_snapshot; - if (table) - metadata_snapshot = table->getInMemoryMetadataPtr(); - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && table->supportsReplication()) - res_columns[res_index++]->insert(metadata_snapshot->metadata_version); - else - res_columns[res_index++]->insertDefault(); - } - - { - Array views_table_name_array; - Array views_database_name_array; - if (columns_mask[src_index] || columns_mask[src_index + 1]) - { - const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); - - views_table_name_array.reserve(view_ids.size()); - views_database_name_array.reserve(view_ids.size()); - for (const auto & view_id : view_ids) - { - views_table_name_array.push_back(view_id.table_name); - views_database_name_array.push_back(view_id.database_name); - } - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_database_name_array); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_table_name_array); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) - { - ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); - auto * ast_create = ast ? ast->as() : nullptr; - - if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) - { - ast_create->uuid = UUIDHelpers::Nil; - ast_create->to_inner_uuid = UUIDHelpers::Nil; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - - if (columns_mask[src_index++]) - { - String engine_full; - - if (ast_create && ast_create->storage) - { - engine_full = format({context, *ast_create->storage}); - - static const char * const extra_head = " ENGINE = "; - if (startsWith(engine_full, extra_head)) - engine_full = engine_full.substr(strlen(extra_head)); - } - - res_columns[res_index++]->insert(engine_full); - } - - if (columns_mask[src_index++]) - { - String as_select; - if (ast_create && ast_create->select) - as_select = format({context, *ast_create->select}); - res_columns[res_index++]->insert(as_select); - } - } - else - src_index += 3; - - ASTPtr expression_ptr; - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto policy = table ? table->getStoragePolicy() : nullptr; - if (policy) - res_columns[res_index++]->insert(policy->getName()); - else - res_columns[res_index++]->insertDefault(); - } - - auto settings = context->getSettingsRef(); - settings.select_sequential_consistency = 0; - if (columns_mask[src_index++]) - { - auto total_rows = table ? table->totalRows(settings) : std::nullopt; - if (total_rows) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes = table->totalBytes(settings); - if (total_bytes) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); - if (total_bytes_uncompressed) - res_columns[res_index++]->insert(*total_bytes_uncompressed); - else - res_columns[res_index++]->insertDefault(); - } - - auto table_merge_tree = std::dynamic_pointer_cast(table); - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - { - res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); - } - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; - if (lifetime_rows) - res_columns[res_index++]->insert(*lifetime_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; - if (lifetime_bytes) - res_columns[res_index++]->insert(*lifetime_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot) - res_columns[res_index++]->insert(metadata_snapshot->comment); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table) - res_columns[res_index++]->insert(table->storesDataOnDisk()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) - { - auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); - auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); - - Array dependencies_databases; - Array dependencies_tables; - dependencies_databases.reserve(dependencies.size()); - dependencies_tables.reserve(dependencies.size()); - for (const auto & dependency : dependencies) - { - dependencies_databases.push_back(dependency.database_name); - dependencies_tables.push_back(dependency.table_name); - } - - Array dependents_databases; - Array dependents_tables; - dependents_databases.reserve(dependents.size()); - dependents_tables.reserve(dependents.size()); - for (const auto & dependent : dependents) - { - dependents_databases.push_back(dependent.database_name); - dependents_tables.push_back(dependent.table_name); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_tables); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_tables); - - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - return Chunk(std::move(res_columns), num_rows); - } -private: - std::vector columns_mask; - UInt64 max_block_size; - ColumnPtr databases; - NameSet tables; - size_t database_idx = 0; - DatabaseTablesIteratorPtr tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; -}; - -} - -class ReadFromSystemTables : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_) - : SourceStepWithFilter( - DataStream{.header = std::move(sample_block)}, - column_names_, - query_info_, - storage_snapshot_, - context_) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - { - } - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -private: - std::vector columns_mask; - size_t max_block_size; - - ColumnPtr filtered_databases_column; - ColumnPtr filtered_tables_column; -}; - void StorageSystemTables::read( QueryPlan & query_plan, const Names & column_names, @@ -743,28 +126,8 @@ void StorageSystemTables::read( auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); auto reading = std::make_unique( - column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size); + column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size, false); query_plan.addStep(std::move(reading)); } - -void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); -} - -void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - Pipe pipe(std::make_shared( - std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context)); - pipeline.init(std::move(pipe)); -} - } diff --git a/src/Storages/System/TablesBlockSource.cpp b/src/Storages/System/TablesBlockSource.cpp new file mode 100644 index 00000000000..4ea5bb53deb --- /dev/null +++ b/src/Storages/System/TablesBlockSource.cpp @@ -0,0 +1,569 @@ +#include "TablesBlockSource.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} + +} + +Chunk TablesBlockSource::generate() +{ + if (done) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + { + /// Database was deleted just now or the user has no access. + ++database_idx; + continue; + } + + break; + } + + /// This is for temporary tables. They are output in single block regardless to max_block_size. + if (database_idx >= databases->size()) + { + if (context->hasSessionContext()) + { + Tables external_tables = context->getSessionContext()->getExternalTables(); + + for (auto & table : external_tables) + { + size_t src_index = 0; + size_t res_index = 0; + + // database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // name + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.first); + + // uuid + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getStorageID().uuid); + + // engine + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + // is_temporary + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(1u); + + // data_paths + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_path + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_modification_time + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_version + // Temporary tables does not support replication + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_table + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // create_table_query + if (columns_mask[src_index++]) + { + auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); + ASTPtr ast + = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + } + + // engine_full + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + const auto & settings = context->getSettingsRef(); + while (src_index < columns_mask.size()) + { + // total_rows + if (src_index == 19 && columns_mask[src_index]) + { + if (auto total_rows = table.second->totalRows(settings)) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + // total_bytes + else if (src_index == 20 && columns_mask[src_index]) + { + if (auto total_bytes = table.second->totalBytes(settings)) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + /// Fill the rest columns with defaults + else if (columns_mask[src_index]) + res_columns[res_index++]->insertDefault(); + src_index++; + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + done = true; + return Chunk(std::move(res_columns), num_rows); + } + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool need_table = needTable(database, getPort().getHeader()); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (!tables.contains(table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + StoragePtr table = nullptr; + TableLockHolder lock; + if (need_table) + { + table = tables_it->table(); + if (!table) + // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) + continue; + + /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) + /// and it's not protected internally by other mutexes + static const size_t DATA_PATHS_INDEX = 5; + if (columns_mask[DATA_PATHS_INDEX]) + { + lock = table->tryLockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); + if (!lock) + // Table was dropped while acquiring the lock, skipping table + continue; + } + } + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(tables_it->uuid()); + + if (columns_mask[src_index++]) + { + chassert(table != nullptr); + res_columns[res_index++]->insert(table->getName()); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(0u); // is_temporary + + if (columns_mask[src_index++]) + { + chassert(lock != nullptr); + Array table_paths_array; + auto paths = table->getDataPaths(); + table_paths_array.reserve(paths.size()); + for (const String & path : paths) + table_paths_array.push_back(path); + res_columns[res_index++]->insert(table_paths_array); + /// We don't need the lock anymore + lock = nullptr; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); + + StorageMetadataPtr metadata_snapshot; + if (table) + metadata_snapshot = table->getInMemoryMetadataPtr(); + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && table->supportsReplication()) + res_columns[res_index++]->insert(metadata_snapshot->metadata_version); + else + res_columns[res_index++]->insertDefault(); + } + + { + Array views_table_name_array; + Array views_database_name_array; + if (columns_mask[src_index] || columns_mask[src_index + 1]) + { + const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); + + views_table_name_array.reserve(view_ids.size()); + views_database_name_array.reserve(view_ids.size()); + for (const auto & view_id : view_ids) + { + views_table_name_array.push_back(view_id.table_name); + views_database_name_array.push_back(view_id.database_name); + } + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_database_name_array); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_table_name_array); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) + { + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + auto * ast_create = ast ? ast->as() : nullptr; + + if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + ast_create->uuid = UUIDHelpers::Nil; + ast_create->to_inner_uuid = UUIDHelpers::Nil; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + + if (columns_mask[src_index++]) + { + String engine_full; + + if (ast_create && ast_create->storage) + { + engine_full = format({context, *ast_create->storage}); + + static const char * const extra_head = " ENGINE = "; + if (startsWith(engine_full, extra_head)) + engine_full = engine_full.substr(strlen(extra_head)); + } + + res_columns[res_index++]->insert(engine_full); + } + + if (columns_mask[src_index++]) + { + String as_select; + if (ast_create && ast_create->select) + as_select = format({context, *ast_create->select}); + res_columns[res_index++]->insert(as_select); + } + } + else + src_index += 3; + + ASTPtr expression_ptr; + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto policy = table ? table->getStoragePolicy() : nullptr; + if (policy) + res_columns[res_index++]->insert(policy->getName()); + else + res_columns[res_index++]->insertDefault(); + } + + auto settings = context->getSettingsRef(); + settings.select_sequential_consistency = 0; + if (columns_mask[src_index++]) + { + auto total_rows = table ? table->totalRows(settings) : std::nullopt; + if (total_rows) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes = table->totalBytes(settings); + if (total_bytes) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); + if (total_bytes_uncompressed) + res_columns[res_index++]->insert(*total_bytes_uncompressed); + else + res_columns[res_index++]->insertDefault(); + } + + auto table_merge_tree = std::dynamic_pointer_cast(table); + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + { + res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); + } + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; + if (lifetime_rows) + res_columns[res_index++]->insert(*lifetime_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; + if (lifetime_bytes) + res_columns[res_index++]->insert(*lifetime_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot) + res_columns[res_index++]->insert(metadata_snapshot->comment); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table) + res_columns[res_index++]->insert(table->storesDataOnDisk()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) + { + auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); + + Array dependencies_databases; + Array dependencies_tables; + dependencies_databases.reserve(dependencies.size()); + dependencies_tables.reserve(dependencies.size()); + for (const auto & dependency : dependencies) + { + dependencies_databases.push_back(dependency.database_name); + dependencies_tables.push_back(dependency.table_name); + } + + Array dependents_databases; + Array dependents_tables; + dependents_databases.reserve(dependents.size()); + dependents_tables.reserve(dependents.size()); + for (const auto & dependent : dependents) + { + dependents_databases.push_back(dependent.database_name); + dependents_tables.push_back(dependent.table_name); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_tables); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_tables); + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + return Chunk(std::move(res_columns), num_rows); +} + +Chunk DetachedTablesBlockSource::generate() +{ + if (done) + return {}; + + MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + { + LOG_DEBUG(lg, "Database was deleted just now or the user has no access"); + continue; + } + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!detached_tables_it || !detached_tables_it->isValid()) + detached_tables_it = database->getDetachedTablesIterator(context, {}, false); + + for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) + { + const auto detached_table_name = detached_tables_it->table(); + LOG_DEBUG(lg, "detached_table_name={}", detached_table_name); + + if (!detached_tables.contains(detached_table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) + continue; + + fillResultColumnsByDetachedTableIterator(result_columns); + ++rows_count; + } + } + + const UInt64 num_rows = result_columns.at(0)->size(); + done = true; + return Chunk(std::move(result_columns), num_rows); +} + +void DetachedTablesBlockSource::fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const +{ + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->database()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->table()); + + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->uuid()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->metadataPath()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->isPermanently()); +} +} diff --git a/src/Storages/System/TablesBlockSource.h b/src/Storages/System/TablesBlockSource.h new file mode 100644 index 00000000000..9698fdd494c --- /dev/null +++ b/src/Storages/System/TablesBlockSource.h @@ -0,0 +1,96 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class TablesBlockSource : public ISource +{ +public: + TablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = tables_->size(); + tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + tables.insert(tables_->getDataAt(idx).toString()); + } + + String getName() const override { return "Tables"; } + +protected: + Chunk generate() override; + +private: + std::vector columns_mask; + UInt64 max_block_size; + ColumnPtr databases; + NameSet tables; + size_t database_idx = 0; + DatabaseTablesIteratorPtr tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; +}; + +class DetachedTablesBlockSource : public ISource +{ +public: + DetachedTablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr detached_tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = detached_tables_->size(); + detached_tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + { + detached_tables.insert(detached_tables_->getDataAt(idx).toString()); + } + } + + String getName() const override { return "DetachedTables"; } + +protected: + Chunk generate() override; + +private: + const std::vector columns_mask; + const UInt64 max_block_size; + const ColumnPtr databases; + NameSet detached_tables; + DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; + + // temp log for debug + LoggerPtr lg = getLogger("DetachedTablesBlockSource"); + + void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const; +}; +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 6ff86b26ca9..97eda1db3fa 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -129,6 +130,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attachNoDescription(context, system_database, "zeros_mt", "Multithreaded version of system.zeros.", true); attach(context, system_database, "databases", "Lists all databases of the current server."); attachNoDescription(context, system_database, "tables", "Lists all tables of the current server."); + attachNoDescription(context, system_database, "detached_tables", "Lists all detached tables of the current server."); attachNoDescription(context, system_database, "columns", "Lists all columns from all tables of the current server."); attach(context, system_database, "functions", "Contains a list of all available ordinary and aggregate functions with their descriptions."); attach(context, system_database, "events", "Contains profiling events and their current value."); diff --git a/tests/integration/test_system_detached_tables/__init__.py b/tests/integration/test_system_detached_tables/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py new file mode 100644 index 00000000000..2e0165deba2 --- /dev/null +++ b/tests/integration/test_system_detached_tables/test.py @@ -0,0 +1,38 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node_default", stay_alive=True) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_system_detached_tables(): + node.query("CREATE TABLE test_table (n Int64) ENGINE=MergeTree ORDER BY n;") + node.query("CREATE TABLE test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n;") + + result = node.query("SELECT * FROM system.detached_tables") + assert result == "" + + node.query("DETACH TABLE test_table") + node.query("DETACH TABLE test_table_perm PERMANENTLY") + + result = node.query("SELECT name FROM system.detached_tables") + assert result == "test_table\ntest_table_perm\n" + + node.restart_clickhouse() + + result = node.query("SELECT name FROM system.detached_tables") + assert result == "test_table_perm\n" + + node.restart_clickhouse() + + result = node.query("SELECT name FROM system.detached_tables") + assert result == "test_table_perm\n" diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference new file mode 100644 index 00000000000..1cf9e0275c9 --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -0,0 +1,5 @@ +test_system_detached_tables test_table 0 +test_system_detached_tables test_table_perm 1 +test_system_detached_tables test_table 0 +test_system_detached_tables test_table_perm 1 +test_system_detached_tables test_table 0 diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql new file mode 100644 index 00000000000..13465d02b80 --- /dev/null +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -0,0 +1,28 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS test_system_detached_tables; +CREATE DATABASE IF NOT EXISTS test_system_detached_tables; + +CREATE TABLE test_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables; + +DETACH TABLE test_system_detached_tables.test_table; +SELECT database, name, is_permanently FROM system.detached_tables; + +ATTACH TABLE test_system_detached_tables.test_table; + +CREATE TABLE test_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables; + +DETACH TABLE test_system_detached_tables.test_table_perm PERMANENTLY; +SELECT database, name, is_permanently FROM system.detached_tables; + +DETACH TABLE test_system_detached_tables.test_table SYNC; +SELECT database, name, is_permanently FROM system.detached_tables; + +SELECT database, name, is_permanently FROM system.detached_tables WHERE name='test_table'; + +DROP DATABASE test_system_detached_tables; + + + From 21936f32a655b1008b6233ee274549b0ce2b33c7 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 21 Jun 2024 12:32:54 +0000 Subject: [PATCH 008/178] fix style --- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/DatabasesCommon.cpp | 4 ++-- src/Databases/IDatabase.h | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 046cbfa40be..ac40bf4e90e 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -243,7 +243,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables const auto storage_id = table->getStorageID(); SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.detabase = storage_id.getDatabaseName(); + snapshot_detached_table.database = storage_id.getDatabaseName(); snapshot_detached_table.table = detached_table_name; snapshot_detached_table.uuid = storage_id.uuid; snapshot_detached_table.is_permanently = true; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 312c0a4477f..c4cad663878 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -250,7 +250,7 @@ DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetached if (filter_by_table_name(table_name)) { SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.detabase = storage->getStorageID().getDatabaseName(); + snapshot_detached_table.database = storage->getStorageID().getDatabaseName(); snapshot_detached_table.table = table_name; if (storage->getStorageID().hasUUID()) { @@ -290,7 +290,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n res = it->second; SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.detabase = it->second->getStorageID().getDatabaseName(); + snapshot_detached_table.database = it->second->getStorageID().getDatabaseName(); snapshot_detached_table.table = it->first; if (it->second->getStorageID().hasUUID()) { diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 7c6b3b3004c..2ce4650798c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -114,7 +114,7 @@ using DatabaseTablesIteratorPtr = std::unique_ptr; struct SnapshotDetachedTable final { - String detabase; + String database; String table; UUID uuid = UUIDHelpers::Nil; String metadata_path; @@ -148,7 +148,7 @@ public: bool isValid() const { return it != snapshot.end(); } - String database() const { return it->second.detabase; } + String database() const { return it->second.database; } String table() const { return it->second.table; } From 7aa7b7977221d5bc9b65d0e09d272ffca96acddc Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 21 Jun 2024 16:33:09 +0000 Subject: [PATCH 009/178] refactor --- .../system-tables/detached_tables.md | 9 +- src/Databases/DatabaseAtomic.h | 1 - src/Databases/DatabaseOrdinary.cpp | 6 +- src/Storages/System/ReadFromSystemTables.cpp | 138 ---- src/Storages/System/ReadFromSystemTables.h | 47 -- .../System/StorageSystemDetachedTables.cpp | 252 +++++- src/Storages/System/StorageSystemTables.cpp | 761 ++++++++++++++++-- src/Storages/System/TablesBlockSource.cpp | 569 ------------- src/Storages/System/TablesBlockSource.h | 96 --- .../test_system_detached_tables/test.py | 6 +- .../03172_system_detached_tables.sql | 8 +- 11 files changed, 959 insertions(+), 934 deletions(-) delete mode 100644 src/Storages/System/ReadFromSystemTables.cpp delete mode 100644 src/Storages/System/ReadFromSystemTables.h delete mode 100644 src/Storages/System/TablesBlockSource.cpp delete mode 100644 src/Storages/System/TablesBlockSource.h diff --git a/docs/en/operations/system-tables/detached_tables.md b/docs/en/operations/system-tables/detached_tables.md index d86e58d2899..f669c8fd251 100644 --- a/docs/en/operations/system-tables/detached_tables.md +++ b/docs/en/operations/system-tables/detached_tables.md @@ -9,14 +9,14 @@ Columns: - `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database the table is in. -- `name` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `table` ([String](../../sql-reference/data-types/string.md)) — Table name. - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid (Atomic database). -- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY. - - `metadata_path` ([String](../../sql-reference/data-types/string.md)) - Path to the table metadata in the file system. +- `is_permanently` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag indicates that the table was detached PERMANENTLY. + **Example** @@ -28,9 +28,8 @@ SELECT * FROM system.detached_tables FORMAT Vertical; Row 1: ────── database: base -name: t1 +table: t1 uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736 -data_paths: ['/var/lib/clickhouse/store/81b/81b1c20a-b7c6-4116-a2ce-7583fb6b6736/'] metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql is_permanently: 1 ``` diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 47afc4dc5ef..25c5cdfbde1 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -85,7 +85,6 @@ protected: //TODO store path in DatabaseWithOwnTables::tables using NameToPathMap = std::unordered_map; - NameToPathMap table_name_to_path TSA_GUARDED_BY(mutex); DetachedTables detached_tables TSA_GUARDED_BY(mutex); diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index ac40bf4e90e..7086f6e628e 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -187,7 +187,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables size_t prev_tables_count = metadata.parsed_tables.size(); size_t prev_total_dictionaries = metadata.total_dictionaries; - auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name) mutable + auto process_metadata = [&metadata, is_startup, local_context, this](const String & file_name) { fs::path path(getMetadataPath()); fs::path file_path(file_name); @@ -227,7 +227,6 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables permanently_detached_tables.push_back(table_name); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); - // @TODO refactoring auto parsed_table_metadata = ParsedTableMetadata{full_path.string(), ast}; const auto & query = parsed_table_metadata.ast->as(); @@ -249,10 +248,9 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables snapshot_detached_table.is_permanently = true; snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); - snapshot_detached_tables.emplace(detached_table_name, std::move(snapshot_detached_table)); - LOG_TRACE(log, "Add detached table {} to system.detached_tables", detached_table_name); + LOG_TRACE(log, "Add permanently detached table {} to system.detached_tables", detached_table_name); return; } diff --git a/src/Storages/System/ReadFromSystemTables.cpp b/src/Storages/System/ReadFromSystemTables.cpp deleted file mode 100644 index 6e9c6a869d1..00000000000 --- a/src/Storages/System/ReadFromSystemTables.cpp +++ /dev/null @@ -1,138 +0,0 @@ -#include "ReadFromSystemTables.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables( - const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool need_detached_tables) -{ - Block sample{ - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - if (need_detached_tables) - { - auto table_it = database->getDetachedTablesIterator(context, {}, false); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->table()); - } - } - else - { - auto table_it = database->getTablesIterator(context); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - } - - Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -} - -ReadFromSystemTables::ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_, - const bool need_detached_tables_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , need_detached_tables(need_detached_tables_) -{ -} - -void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context, need_detached_tables); -} - -void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - if (need_detached_tables) - { - pipeline.init(createPipe()); - } - else - { - pipeline.init(createPipe()); - } -} -} diff --git a/src/Storages/System/ReadFromSystemTables.h b/src/Storages/System/ReadFromSystemTables.h deleted file mode 100644 index f8280ffcc8d..00000000000 --- a/src/Storages/System/ReadFromSystemTables.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ReadFromSystemTables : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_, - bool need_detached_tables); - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -private: - std::vector columns_mask; - size_t max_block_size; - const bool need_detached_tables; - - ColumnPtr filtered_databases_column; - ColumnPtr filtered_tables_column; - - template - Pipe createPipe() - { - return Pipe(std::make_shared( - std::move(columns_mask), - getOutputStream().header, - max_block_size, - std::move(filtered_databases_column), - std::move(filtered_tables_column), - context)); - } -}; -} diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 1596ac6a046..9805fa2a8cf 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -1,28 +1,232 @@ #include "StorageSystemDetachedTables.h" +#include #include #include #include #include +#include +#include #include +#include +#include #include #include #include -#include -#include #include +#include + +#include namespace DB { +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +class DetachedTablesBlockSource : public ISource +{ +public: + DetachedTablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr detached_tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = detached_tables_->size(); + detached_tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + { + detached_tables.insert(detached_tables_->getDataAt(idx).toString()); + } + } + + String getName() const override { return "DetachedTables"; } + +protected: + Chunk generate() override + { + if (done) + return {}; + + MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + continue; + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!detached_tables_it || !detached_tables_it->isValid()) + detached_tables_it = database->getDetachedTablesIterator(context, {}, false); + + for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) + { + const auto detached_table_name = detached_tables_it->table(); + + if (!detached_tables.contains(detached_table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) + continue; + + fillResultColumnsByDetachedTableIterator(result_columns); + ++rows_count; + } + } + + const UInt64 num_rows = result_columns.at(0)->size(); + done = true; + return Chunk(std::move(result_columns), num_rows); + } + +private: + const std::vector columns_mask; + const UInt64 max_block_size; + const ColumnPtr databases; + NameSet detached_tables; + DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; + + void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const + { + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->database()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->table()); + + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->uuid()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->metadataPath()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->isPermanently()); + } +}; + +} + +class ReadFromSystemDetachedTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemDetachedTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; auto description = ColumnsDescription{ ColumnDescription{"database", std::make_shared(), "The name of the database the table is in."}, - ColumnDescription{"name", std::make_shared(), "Table name."}, + ColumnDescription{"table", std::make_shared(), "Table name."}, ColumnDescription{"uuid", std::make_shared(), "Table uuid (Atomic database)."}, ColumnDescription{"metadata_path", std::make_shared(), "Path to the table metadata in the file system."}, ColumnDescription{"is_permanently", std::make_shared(), "Table was detached permanently."}, @@ -48,9 +252,47 @@ void StorageSystemDetachedTables::read( auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); - auto reading = std::make_unique( - column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size, true); + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size); query_plan.addStep(std::move(reading)); } + +ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) +{ +} + +void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); +} + +void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto pipe = Pipe(std::make_shared( + std::move(columns_mask), + getOutputStream().header, + max_block_size, + std::move(filtered_databases_column), + std::move(filtered_tables_column), + context)); + pipeline.init(std::move(pipe)); +} } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 5a8d6315d57..783b899c978 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,31 +1,29 @@ -#include "StorageSystemTables.h" - -#include #include -#include +#include #include #include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include +#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include +#include #include @@ -34,7 +32,8 @@ namespace DB { -StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage(table_id_) +StorageSystemTables::StorageSystemTables(const StorageID & table_id_) + : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -47,13 +46,9 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage {"data_paths", std::make_shared(std::make_shared()), "Paths to the table data in the file systems."}, {"metadata_path", std::make_shared(), "Path to the table metadata in the file system."}, {"metadata_modification_time", std::make_shared(), "Time of latest modification of the table metadata."}, - {"metadata_version", - std::make_shared(), - "Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table."}, + {"metadata_version", std::make_shared(), "Metadata version for ReplicatedMergeTree table, 0 for non ReplicatedMergeTree table."}, {"dependencies_database", std::make_shared(std::make_shared()), "Database dependencies."}, - {"dependencies_table", - std::make_shared(std::make_shared()), - "Table dependencies (materialized views the current table)."}, + {"dependencies_table", std::make_shared(std::make_shared()), "Table dependencies (materialized views the current table)."}, {"create_table_query", std::make_shared(), "The query that was used to create the table."}, {"engine_full", std::make_shared(), "Parameters of the table engine."}, {"as_select", std::make_shared(), "SELECT query for view."}, @@ -62,54 +57,676 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage {"primary_key", std::make_shared(), "The primary key expression specified in the table."}, {"sampling_key", std::make_shared(), "The sampling key expression specified in the table."}, {"storage_policy", std::make_shared(), "The storage policy."}, - {"total_rows", - std::make_shared(std::make_shared()), - "Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise NULL (including " - "underlying Buffer table)."}, - {"total_bytes", - std::make_shared(std::make_shared()), - "Total number of bytes, if it is possible to quickly determine exact number " - "of bytes for the table on storage, otherwise NULL (does not includes any underlying storage). " - "If the table stores data on disk, returns used space on disk (i.e. compressed). " - "If the table stores data in memory, returns approximated number of used bytes in memory."}, - {"total_bytes_uncompressed", - std::make_shared(std::make_shared()), - "Total number of uncompressed bytes, if it's possible to quickly determine the exact number " - "of bytes from the part checksums for the table on storage, otherwise NULL (does not take underlying storage (if any) into " - "account)."}, + {"total_rows", std::make_shared(std::make_shared()), + "Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise NULL (including underlying Buffer table)." + }, + {"total_bytes", std::make_shared(std::make_shared()), + "Total number of bytes, if it is possible to quickly determine exact number " + "of bytes for the table on storage, otherwise NULL (does not includes any underlying storage). " + "If the table stores data on disk, returns used space on disk (i.e. compressed). " + "If the table stores data in memory, returns approximated number of used bytes in memory." + }, + {"total_bytes_uncompressed", std::make_shared(std::make_shared()), + "Total number of uncompressed bytes, if it's possible to quickly determine the exact number " + "of bytes from the part checksums for the table on storage, otherwise NULL (does not take underlying storage (if any) into account)." + }, {"parts", std::make_shared(std::make_shared()), "The total number of parts in this table."}, - {"active_parts", - std::make_shared(std::make_shared()), - "The number of active parts in this table."}, - {"total_marks", - std::make_shared(std::make_shared()), - "The total number of marks in all parts in this table."}, - {"lifetime_rows", - std::make_shared(std::make_shared()), - "Total number of rows INSERTed since server start (only for Buffer tables)."}, - {"lifetime_bytes", - std::make_shared(std::make_shared()), - "Total number of bytes INSERTed since server start (only for Buffer tables)."}, + {"active_parts", std::make_shared(std::make_shared()), "The number of active parts in this table."}, + {"total_marks", std::make_shared(std::make_shared()), "The total number of marks in all parts in this table."}, + {"lifetime_rows", std::make_shared(std::make_shared()), + "Total number of rows INSERTed since server start (only for Buffer tables)." + }, + {"lifetime_bytes", std::make_shared(std::make_shared()), + "Total number of bytes INSERTed since server start (only for Buffer tables)." + }, {"comment", std::make_shared(), "The comment for the table."}, - {"has_own_data", - std::make_shared(), - "Flag that indicates whether the table itself stores some data on disk or only accesses some other source."}, - {"loading_dependencies_database", - std::make_shared(std::make_shared()), - "Database loading dependencies (list of objects which should be loaded before the current object)."}, - {"loading_dependencies_table", - std::make_shared(std::make_shared()), - "Table loading dependencies (list of objects which should be loaded before the current object)."}, - {"loading_dependent_database", std::make_shared(std::make_shared()), "Dependent loading database."}, - {"loading_dependent_table", std::make_shared(std::make_shared()), "Dependent loading table."}, + {"has_own_data", std::make_shared(), + "Flag that indicates whether the table itself stores some data on disk or only accesses some other source." + }, + {"loading_dependencies_database", std::make_shared(std::make_shared()), + "Database loading dependencies (list of objects which should be loaded before the current object)." + }, + {"loading_dependencies_table", std::make_shared(std::make_shared()), + "Table loading dependencies (list of objects which should be loaded before the current object)." + }, + {"loading_dependent_database", std::make_shared(std::make_shared()), + "Dependent loading database." + }, + {"loading_dependent_table", std::make_shared(std::make_shared()), + "Dependent loading table." + }, }; - description.setAliases({{"table", std::make_shared(), "name"}}); + description.setAliases({ + {"table", std::make_shared(), "name"} + }); storage_metadata.setColumns(std::move(description)); setInMemoryMetadata(storage_metadata); } + +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) +{ + Block sample { + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") + }; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + + Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} + + +class TablesBlockSource : public ISource +{ +public: + TablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = tables_->size(); + tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + tables.insert(tables_->getDataAt(idx).toString()); + } + + String getName() const override { return "Tables"; } + +protected: + Chunk generate() override + { + if (done) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + { + /// Database was deleted just now or the user has no access. + ++database_idx; + continue; + } + + break; + } + + /// This is for temporary tables. They are output in single block regardless to max_block_size. + if (database_idx >= databases->size()) + { + if (context->hasSessionContext()) + { + Tables external_tables = context->getSessionContext()->getExternalTables(); + + for (auto & table : external_tables) + { + size_t src_index = 0; + size_t res_index = 0; + + // database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // name + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.first); + + // uuid + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getStorageID().uuid); + + // engine + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + // is_temporary + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(1u); + + // data_paths + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_path + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_modification_time + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_version + // Temporary tables does not support replication + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_table + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // create_table_query + if (columns_mask[src_index++]) + { + auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); + ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + } + + // engine_full + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + const auto & settings = context->getSettingsRef(); + while (src_index < columns_mask.size()) + { + // total_rows + if (src_index == 19 && columns_mask[src_index]) + { + if (auto total_rows = table.second->totalRows(settings)) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + // total_bytes + else if (src_index == 20 && columns_mask[src_index]) + { + if (auto total_bytes = table.second->totalBytes(settings)) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + /// Fill the rest columns with defaults + else if (columns_mask[src_index]) + res_columns[res_index++]->insertDefault(); + src_index++; + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + done = true; + return Chunk(std::move(res_columns), num_rows); + } + + const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool need_table = needTable(database, getPort().getHeader()); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (!tables.contains(table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + StoragePtr table = nullptr; + TableLockHolder lock; + if (need_table) + { + table = tables_it->table(); + if (!table) + // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) + continue; + + /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) + /// and it's not protected internally by other mutexes + static const size_t DATA_PATHS_INDEX = 5; + if (columns_mask[DATA_PATHS_INDEX]) + { + lock = table->tryLockForShare(context->getCurrentQueryId(), + context->getSettingsRef().lock_acquire_timeout); + if (!lock) + // Table was dropped while acquiring the lock, skipping table + continue; + } + } + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(tables_it->uuid()); + + if (columns_mask[src_index++]) + { + chassert(table != nullptr); + res_columns[res_index++]->insert(table->getName()); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(0u); // is_temporary + + if (columns_mask[src_index++]) + { + chassert(lock != nullptr); + Array table_paths_array; + auto paths = table->getDataPaths(); + table_paths_array.reserve(paths.size()); + for (const String & path : paths) + table_paths_array.push_back(path); + res_columns[res_index++]->insert(table_paths_array); + /// We don't need the lock anymore + lock = nullptr; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); + + StorageMetadataPtr metadata_snapshot; + if (table) + metadata_snapshot = table->getInMemoryMetadataPtr(); + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && table->supportsReplication()) + res_columns[res_index++]->insert(metadata_snapshot->metadata_version); + else + res_columns[res_index++]->insertDefault(); + } + + { + Array views_table_name_array; + Array views_database_name_array; + if (columns_mask[src_index] || columns_mask[src_index + 1]) + { + const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); + + views_table_name_array.reserve(view_ids.size()); + views_database_name_array.reserve(view_ids.size()); + for (const auto & view_id : view_ids) + { + views_table_name_array.push_back(view_id.table_name); + views_database_name_array.push_back(view_id.database_name); + } + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_database_name_array); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_table_name_array); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) + { + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + auto * ast_create = ast ? ast->as() : nullptr; + + if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + ast_create->uuid = UUIDHelpers::Nil; + ast_create->to_inner_uuid = UUIDHelpers::Nil; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + + if (columns_mask[src_index++]) + { + String engine_full; + + if (ast_create && ast_create->storage) + { + engine_full = format({context, *ast_create->storage}); + + static const char * const extra_head = " ENGINE = "; + if (startsWith(engine_full, extra_head)) + engine_full = engine_full.substr(strlen(extra_head)); + } + + res_columns[res_index++]->insert(engine_full); + } + + if (columns_mask[src_index++]) + { + String as_select; + if (ast_create && ast_create->select) + as_select = format({context, *ast_create->select}); + res_columns[res_index++]->insert(as_select); + } + } + else + src_index += 3; + + ASTPtr expression_ptr; + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto policy = table ? table->getStoragePolicy() : nullptr; + if (policy) + res_columns[res_index++]->insert(policy->getName()); + else + res_columns[res_index++]->insertDefault(); + } + + auto settings = context->getSettingsRef(); + settings.select_sequential_consistency = 0; + if (columns_mask[src_index++]) + { + auto total_rows = table ? table->totalRows(settings) : std::nullopt; + if (total_rows) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes = table->totalBytes(settings); + if (total_bytes) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); + if (total_bytes_uncompressed) + res_columns[res_index++]->insert(*total_bytes_uncompressed); + else + res_columns[res_index++]->insertDefault(); + } + + auto table_merge_tree = std::dynamic_pointer_cast(table); + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + { + res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); + } + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; + if (lifetime_rows) + res_columns[res_index++]->insert(*lifetime_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; + if (lifetime_bytes) + res_columns[res_index++]->insert(*lifetime_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot) + res_columns[res_index++]->insert(metadata_snapshot->comment); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table) + res_columns[res_index++]->insert(table->storesDataOnDisk()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) + { + auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); + + Array dependencies_databases; + Array dependencies_tables; + dependencies_databases.reserve(dependencies.size()); + dependencies_tables.reserve(dependencies.size()); + for (const auto & dependency : dependencies) + { + dependencies_databases.push_back(dependency.database_name); + dependencies_tables.push_back(dependency.table_name); + } + + Array dependents_databases; + Array dependents_tables; + dependents_databases.reserve(dependents.size()); + dependents_tables.reserve(dependents.size()); + for (const auto & dependent : dependents) + { + dependents_databases.push_back(dependent.database_name); + dependents_tables.push_back(dependent.table_name); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_tables); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_tables); + + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + return Chunk(std::move(res_columns), num_rows); + } +private: + std::vector columns_mask; + UInt64 max_block_size; + ColumnPtr databases; + NameSet tables; + size_t database_idx = 0; + DatabaseTablesIteratorPtr tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; +}; + +} + +class ReadFromSystemTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + void StorageSystemTables::read( QueryPlan & query_plan, const Names & column_names, @@ -126,8 +743,28 @@ void StorageSystemTables::read( auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); auto reading = std::make_unique( - column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size, false); + column_names, query_info, storage_snapshot, context, std::move(res_block), std::move(columns_mask), max_block_size); query_plan.addStep(std::move(reading)); } + +void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); +} + +void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipe pipe(std::make_shared( + std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context)); + pipeline.init(std::move(pipe)); +} + } diff --git a/src/Storages/System/TablesBlockSource.cpp b/src/Storages/System/TablesBlockSource.cpp deleted file mode 100644 index 4ea5bb53deb..00000000000 --- a/src/Storages/System/TablesBlockSource.cpp +++ /dev/null @@ -1,569 +0,0 @@ -#include "TablesBlockSource.h" - -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace -{ - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - -} - -Chunk TablesBlockSource::generate() -{ - if (done) - return {}; - - MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - size_t rows_count = 0; - while (rows_count < max_block_size) - { - if (tables_it && !tables_it->isValid()) - ++database_idx; - - while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - { - /// Database was deleted just now or the user has no access. - ++database_idx; - continue; - } - - break; - } - - /// This is for temporary tables. They are output in single block regardless to max_block_size. - if (database_idx >= databases->size()) - { - if (context->hasSessionContext()) - { - Tables external_tables = context->getSessionContext()->getExternalTables(); - - for (auto & table : external_tables) - { - size_t src_index = 0; - size_t res_index = 0; - - // database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // name - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.first); - - // uuid - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getStorageID().uuid); - - // engine - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - // is_temporary - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(1u); - - // data_paths - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_path - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_modification_time - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_version - // Temporary tables does not support replication - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_table - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // create_table_query - if (columns_mask[src_index++]) - { - auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); - ASTPtr ast - = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - } - - // engine_full - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - const auto & settings = context->getSettingsRef(); - while (src_index < columns_mask.size()) - { - // total_rows - if (src_index == 19 && columns_mask[src_index]) - { - if (auto total_rows = table.second->totalRows(settings)) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - // total_bytes - else if (src_index == 20 && columns_mask[src_index]) - { - if (auto total_bytes = table.second->totalBytes(settings)) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - /// Fill the rest columns with defaults - else if (columns_mask[src_index]) - res_columns[res_index++]->insertDefault(); - src_index++; - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - done = true; - return Chunk(std::move(res_columns), num_rows); - } - - const bool need_to_check_access_for_tables - = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(context); - - const bool need_table = needTable(database, getPort().getHeader()); - - for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) - { - auto table_name = tables_it->name(); - if (!tables.contains(table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) - continue; - - StoragePtr table = nullptr; - TableLockHolder lock; - if (need_table) - { - table = tables_it->table(); - if (!table) - // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) - continue; - - /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) - /// and it's not protected internally by other mutexes - static const size_t DATA_PATHS_INDEX = 5; - if (columns_mask[DATA_PATHS_INDEX]) - { - lock = table->tryLockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); - if (!lock) - // Table was dropped while acquiring the lock, skipping table - continue; - } - } - ++rows_count; - - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(tables_it->uuid()); - - if (columns_mask[src_index++]) - { - chassert(table != nullptr); - res_columns[res_index++]->insert(table->getName()); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(0u); // is_temporary - - if (columns_mask[src_index++]) - { - chassert(lock != nullptr); - Array table_paths_array; - auto paths = table->getDataPaths(); - table_paths_array.reserve(paths.size()); - for (const String & path : paths) - table_paths_array.push_back(path); - res_columns[res_index++]->insert(table_paths_array); - /// We don't need the lock anymore - lock = nullptr; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); - - StorageMetadataPtr metadata_snapshot; - if (table) - metadata_snapshot = table->getInMemoryMetadataPtr(); - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && table->supportsReplication()) - res_columns[res_index++]->insert(metadata_snapshot->metadata_version); - else - res_columns[res_index++]->insertDefault(); - } - - { - Array views_table_name_array; - Array views_database_name_array; - if (columns_mask[src_index] || columns_mask[src_index + 1]) - { - const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); - - views_table_name_array.reserve(view_ids.size()); - views_database_name_array.reserve(view_ids.size()); - for (const auto & view_id : view_ids) - { - views_table_name_array.push_back(view_id.table_name); - views_database_name_array.push_back(view_id.database_name); - } - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_database_name_array); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_table_name_array); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) - { - ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); - auto * ast_create = ast ? ast->as() : nullptr; - - if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) - { - ast_create->uuid = UUIDHelpers::Nil; - ast_create->to_inner_uuid = UUIDHelpers::Nil; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - - if (columns_mask[src_index++]) - { - String engine_full; - - if (ast_create && ast_create->storage) - { - engine_full = format({context, *ast_create->storage}); - - static const char * const extra_head = " ENGINE = "; - if (startsWith(engine_full, extra_head)) - engine_full = engine_full.substr(strlen(extra_head)); - } - - res_columns[res_index++]->insert(engine_full); - } - - if (columns_mask[src_index++]) - { - String as_select; - if (ast_create && ast_create->select) - as_select = format({context, *ast_create->select}); - res_columns[res_index++]->insert(as_select); - } - } - else - src_index += 3; - - ASTPtr expression_ptr; - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto policy = table ? table->getStoragePolicy() : nullptr; - if (policy) - res_columns[res_index++]->insert(policy->getName()); - else - res_columns[res_index++]->insertDefault(); - } - - auto settings = context->getSettingsRef(); - settings.select_sequential_consistency = 0; - if (columns_mask[src_index++]) - { - auto total_rows = table ? table->totalRows(settings) : std::nullopt; - if (total_rows) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes = table->totalBytes(settings); - if (total_bytes) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); - if (total_bytes_uncompressed) - res_columns[res_index++]->insert(*total_bytes_uncompressed); - else - res_columns[res_index++]->insertDefault(); - } - - auto table_merge_tree = std::dynamic_pointer_cast(table); - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - { - res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); - } - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; - if (lifetime_rows) - res_columns[res_index++]->insert(*lifetime_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; - if (lifetime_bytes) - res_columns[res_index++]->insert(*lifetime_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot) - res_columns[res_index++]->insert(metadata_snapshot->comment); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table) - res_columns[res_index++]->insert(table->storesDataOnDisk()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) - { - auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); - auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); - - Array dependencies_databases; - Array dependencies_tables; - dependencies_databases.reserve(dependencies.size()); - dependencies_tables.reserve(dependencies.size()); - for (const auto & dependency : dependencies) - { - dependencies_databases.push_back(dependency.database_name); - dependencies_tables.push_back(dependency.table_name); - } - - Array dependents_databases; - Array dependents_tables; - dependents_databases.reserve(dependents.size()); - dependents_tables.reserve(dependents.size()); - for (const auto & dependent : dependents) - { - dependents_databases.push_back(dependent.database_name); - dependents_tables.push_back(dependent.table_name); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_tables); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_tables); - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - return Chunk(std::move(res_columns), num_rows); -} - -Chunk DetachedTablesBlockSource::generate() -{ - if (done) - return {}; - - MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - { - LOG_DEBUG(lg, "Database was deleted just now or the user has no access"); - continue; - } - - const bool need_to_check_access_for_tables - = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!detached_tables_it || !detached_tables_it->isValid()) - detached_tables_it = database->getDetachedTablesIterator(context, {}, false); - - for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) - { - const auto detached_table_name = detached_tables_it->table(); - LOG_DEBUG(lg, "detached_table_name={}", detached_table_name); - - if (!detached_tables.contains(detached_table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) - continue; - - fillResultColumnsByDetachedTableIterator(result_columns); - ++rows_count; - } - } - - const UInt64 num_rows = result_columns.at(0)->size(); - done = true; - return Chunk(std::move(result_columns), num_rows); -} - -void DetachedTablesBlockSource::fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const -{ - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->database()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->table()); - - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->uuid()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->metadataPath()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->isPermanently()); -} -} diff --git a/src/Storages/System/TablesBlockSource.h b/src/Storages/System/TablesBlockSource.h deleted file mode 100644 index 9698fdd494c..00000000000 --- a/src/Storages/System/TablesBlockSource.h +++ /dev/null @@ -1,96 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class TablesBlockSource : public ISource -{ -public: - TablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = tables_->size(); - tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - tables.insert(tables_->getDataAt(idx).toString()); - } - - String getName() const override { return "Tables"; } - -protected: - Chunk generate() override; - -private: - std::vector columns_mask; - UInt64 max_block_size; - ColumnPtr databases; - NameSet tables; - size_t database_idx = 0; - DatabaseTablesIteratorPtr tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; -}; - -class DetachedTablesBlockSource : public ISource -{ -public: - DetachedTablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr detached_tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = detached_tables_->size(); - detached_tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - { - detached_tables.insert(detached_tables_->getDataAt(idx).toString()); - } - } - - String getName() const override { return "DetachedTables"; } - -protected: - Chunk generate() override; - -private: - const std::vector columns_mask; - const UInt64 max_block_size; - const ColumnPtr databases; - NameSet detached_tables; - DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; - - // temp log for debug - LoggerPtr lg = getLogger("DetachedTablesBlockSource"); - - void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const; -}; -} diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index 2e0165deba2..02734d8bf14 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -24,15 +24,15 @@ def test_system_detached_tables(): node.query("DETACH TABLE test_table") node.query("DETACH TABLE test_table_perm PERMANENTLY") - result = node.query("SELECT name FROM system.detached_tables") + result = node.query("SELECT table FROM system.detached_tables") assert result == "test_table\ntest_table_perm\n" node.restart_clickhouse() - result = node.query("SELECT name FROM system.detached_tables") + result = node.query("SELECT table FROM system.detached_tables") assert result == "test_table_perm\n" node.restart_clickhouse() - result = node.query("SELECT name FROM system.detached_tables") + result = node.query("SELECT table FROM system.detached_tables") assert result == "test_table_perm\n" diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql index 13465d02b80..558dfe4cf18 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -7,7 +7,7 @@ CREATE TABLE test_system_detached_tables.test_table (n Int64) ENGINE=MergeTree O SELECT * FROM system.detached_tables; DETACH TABLE test_system_detached_tables.test_table; -SELECT database, name, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables; ATTACH TABLE test_system_detached_tables.test_table; @@ -15,12 +15,12 @@ CREATE TABLE test_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeT SELECT * FROM system.detached_tables; DETACH TABLE test_system_detached_tables.test_table_perm PERMANENTLY; -SELECT database, name, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables; DETACH TABLE test_system_detached_tables.test_table SYNC; -SELECT database, name, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables; -SELECT database, name, is_permanently FROM system.detached_tables WHERE name='test_table'; +SELECT database, table, is_permanently FROM system.detached_tables WHERE table='test_table'; DROP DATABASE test_system_detached_tables; From 92ff09f303dad8e43f4cf8ad6bb295882733aaf9 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 21 Jun 2024 22:53:38 +0000 Subject: [PATCH 010/178] There's no Unwind_AppleExtras.cpp anymore --- contrib/libunwind-cmake/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 8d99d729010..b566e8cb9b3 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -4,9 +4,6 @@ set(LIBUNWIND_CXX_SOURCES "${LIBUNWIND_SOURCE_DIR}/src/libunwind.cpp" "${LIBUNWIND_SOURCE_DIR}/src/Unwind-EHABI.cpp" "${LIBUNWIND_SOURCE_DIR}/src/Unwind-seh.cpp") -if (APPLE) - set(LIBUNWIND_CXX_SOURCES ${LIBUNWIND_CXX_SOURCES} "${LIBUNWIND_SOURCE_DIR}/src/Unwind_AppleExtras.cpp") -endif () set(LIBUNWIND_C_SOURCES "${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c" From ad246312d7ff6009547be89c65754a27e1dc9a22 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 21 Jun 2024 23:05:41 +0000 Subject: [PATCH 011/178] Switch to final commit hash --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 02f17ec85cd..010ea7c5f9d 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 02f17ec85cd6b28540d0a4b42f32c2f8a7c8d79d +Subproject commit 010ea7c5f9de966b37a6fcda855d52bdfe0a6ac6 From 00f74905a769d06a451c6472a0f461b25266b3be Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Mon, 24 Jun 2024 09:07:23 +0000 Subject: [PATCH 012/178] small up --- src/Databases/DatabaseOrdinary.cpp | 5 ++++- src/Databases/IDatabase.h | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 7086f6e628e..3975eb46a4b 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -244,7 +244,10 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables SnapshotDetachedTable snapshot_detached_table; snapshot_detached_table.database = storage_id.getDatabaseName(); snapshot_detached_table.table = detached_table_name; - snapshot_detached_table.uuid = storage_id.uuid; + if (storage_id.hasUUID()) + { + snapshot_detached_table.uuid = storage_id.uuid; + } snapshot_detached_table.is_permanently = true; snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 2ce4650798c..76d25908fda 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -286,7 +286,7 @@ public: virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no get detached tables for Database{}", getEngineName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no get detached tables for Database {}", getEngineName()); } /// Returns list of table names. From c7dc99757afec4dbe4ac5b1bcfab037e141839d8 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 26 Jun 2024 15:39:16 +0000 Subject: [PATCH 013/178] apply some comments --- src/Databases/DatabaseOrdinary.cpp | 31 ++++-------- src/Databases/DatabasesCommon.cpp | 48 +++++++------------ src/Databases/IDatabase.h | 4 +- .../test_system_detached_tables/test.py | 22 +++++---- 4 files changed, 42 insertions(+), 63 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 3975eb46a4b..ef0c981ddad 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -227,31 +227,18 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables permanently_detached_tables.push_back(table_name); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); - auto parsed_table_metadata = ParsedTableMetadata{full_path.string(), ast}; - const auto & query = parsed_table_metadata.ast->as(); - std::lock_guard lock(mutex); - auto [detached_table_name, table] = createTableFromAST( - query, - database_name, - getTableDataPath(query), - std::const_pointer_cast(local_context), - LoadingStrictnessLevel::CREATE); + const auto detached_table_name = create_query->getTable(); - const auto storage_id = table->getStorageID(); - - SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.database = storage_id.getDatabaseName(); - snapshot_detached_table.table = detached_table_name; - if (storage_id.hasUUID()) - { - snapshot_detached_table.uuid = storage_id.uuid; - } - snapshot_detached_table.is_permanently = true; - snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); - - snapshot_detached_tables.emplace(detached_table_name, std::move(snapshot_detached_table)); + snapshot_detached_tables.emplace( + detached_table_name, + SnapshotDetachedTable{ + .database = create_query->getDatabase(), + .table = detached_table_name, + .uuid = create_query->uuid, + .metadata_path = getObjectMetadataPath(detached_table_name), + .is_permanently = true}); LOG_TRACE(log, "Add permanently detached table {} to system.detached_tables", detached_table_name); return; diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 3b54980597c..470625ddd8b 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -2,6 +2,10 @@ #include #include +#include +#include +#include +#include #include #include #include @@ -13,10 +17,6 @@ #include #include #include -#include -#include -#include -#include namespace DB @@ -245,26 +245,15 @@ DatabaseDetachedTablesSnapshotIteratorPtr DatabaseWithOwnTablesBase::getDetached if (!filter_by_table_name) return std::make_unique(snapshot_detached_tables); - SnapshotDetachedTables filtered_tables; - for (const auto & [table_name, storage] : tables) - if (filter_by_table_name(table_name)) + SnapshotDetachedTables filtered_detached_tables; + for (const auto & [detached_table_name, snapshot] : snapshot_detached_tables) + if (filter_by_table_name(detached_table_name)) { - SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.database = storage->getStorageID().getDatabaseName(); - snapshot_detached_table.table = table_name; - if (storage->getStorageID().hasUUID()) - { - snapshot_detached_table.uuid = storage->getStorageID().uuid; - } - - snapshot_detached_table.is_permanently = false; - snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); - - filtered_tables.emplace(table_name, std::move(snapshot_detached_table)); + filtered_detached_tables.emplace(detached_table_name, snapshot); } - return std::make_unique(std::move(filtered_tables)); + return std::make_unique(std::move(filtered_detached_tables)); } bool DatabaseWithOwnTablesBase::empty() const @@ -289,17 +278,14 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n backQuote(database_name), backQuote(table_name)); res = it->second; - SnapshotDetachedTable snapshot_detached_table; - snapshot_detached_table.database = it->second->getStorageID().getDatabaseName(); - snapshot_detached_table.table = it->first; - if (it->second->getStorageID().hasUUID()) - { - snapshot_detached_table.uuid = it->second->getStorageID().uuid; - } - snapshot_detached_table.is_permanently = false; - snapshot_detached_table.metadata_path = getObjectMetadataPath(snapshot_detached_table.table); - - snapshot_detached_tables.emplace(it->first, std::move(snapshot_detached_table)); + snapshot_detached_tables.emplace( + it->first, + SnapshotDetachedTable{ + .database = it->second->getStorageID().getDatabaseName(), + .table = it->first, + .uuid = it->second->getStorageID().uuid, + .metadata_path = getObjectMetadataPath(it->first), + .is_permanently = false}); tables.erase(it); res->is_detached = true; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 76d25908fda..e856dd3d2f0 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -118,7 +118,7 @@ struct SnapshotDetachedTable final String table; UUID uuid = UUIDHelpers::Nil; String metadata_path; - bool is_permanently{false}; + bool is_permanently{}; }; class DatabaseDetachedTablesSnapshotIterator @@ -286,7 +286,7 @@ public: virtual DatabaseDetachedTablesSnapshotIteratorPtr getDetachedTablesIterator( ContextPtr /*context*/, const FilterByNameFunction & /*filter_by_table_name = {}*/, bool /*skip_not_loaded = false*/) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "There is no get detached tables for Database {}", getEngineName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get detached tables for Database{}", getEngineName()); } /// Returns list of table names. diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index 02734d8bf14..bbac5e391f8 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -18,21 +18,27 @@ def test_system_detached_tables(): node.query("CREATE TABLE test_table (n Int64) ENGINE=MergeTree ORDER BY n;") node.query("CREATE TABLE test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n;") - result = node.query("SELECT * FROM system.detached_tables") - assert result == "" + test_table_uuid = node.query("SELECT uuid FROM system.tables WHERE table='test_table'").rstrip('\n') + test_table_metadata_path = node.query("SELECT metadata_path FROM system.tables WHERE table='test_table'").rstrip('\n') + + test_table_perm_uuid = node.query("SELECT uuid FROM system.tables WHERE table='test_table_perm'").rstrip('\n') + test_table_perm_metadata_path = node.query("SELECT metadata_path FROM system.tables WHERE table='test_table_perm'").rstrip('\n') + + assert "" == node.query("SELECT * FROM system.detached_tables") node.query("DETACH TABLE test_table") node.query("DETACH TABLE test_table_perm PERMANENTLY") - result = node.query("SELECT table FROM system.detached_tables") - assert result == "test_table\ntest_table_perm\n" + querry = "SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables FORMAT Values" + result = node.query(querry) + assert result == f"('default','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" node.restart_clickhouse() - result = node.query("SELECT table FROM system.detached_tables") - assert result == "test_table_perm\n" + result = node.query(querry) + assert result == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" node.restart_clickhouse() - result = node.query("SELECT table FROM system.detached_tables") - assert result == "test_table_perm\n" + result = node.query(querry) + assert result == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" From de8f1fc6e3adb6c8f346b2fda4e34c2240e1bf8e Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 07:32:56 +0000 Subject: [PATCH 014/178] general logic for tables --- src/Storages/System/ReadFromSystemTables.cpp | 157 ++++ src/Storages/System/ReadFromSystemTables.h | 66 ++ .../System/StorageSystemDetachedTables.cpp | 237 +------ src/Storages/System/StorageSystemTables.cpp | 669 +----------------- src/Storages/System/TablesBlockSource.cpp | 564 +++++++++++++++ src/Storages/System/TablesBlockSource.h | 92 +++ 6 files changed, 894 insertions(+), 891 deletions(-) create mode 100644 src/Storages/System/ReadFromSystemTables.cpp create mode 100644 src/Storages/System/ReadFromSystemTables.h create mode 100644 src/Storages/System/TablesBlockSource.cpp create mode 100644 src/Storages/System/TablesBlockSource.h diff --git a/src/Storages/System/ReadFromSystemTables.cpp b/src/Storages/System/ReadFromSystemTables.cpp new file mode 100644 index 00000000000..638cab1b63b --- /dev/null +++ b/src/Storages/System/ReadFromSystemTables.cpp @@ -0,0 +1,157 @@ +#include "ReadFromSystemTables.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables( + const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool need_detached_tables) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + if (need_detached_tables) + { + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + else + { + auto table_it = database->getTablesIterator(context); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +} + +ReadFromSystemTables::ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) +{ +} + +void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + applyFiltersImpl(std::move(added_filter_nodes), false); +} + +void ReadFromSystemTables::applyFiltersImpl(ActionDAGNodes added_filter_nodes, const bool need_detached_tables_) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context, need_detached_tables_); +} + +void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + pipeline.init(createPipe()); +} + +ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + const size_t max_block_size_) + : ReadFromSystemTables( + column_names_, query_info_, storage_snapshot_, context_, std::move(sample_block), std::move(columns_mask_), max_block_size_) +{ +} + +void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + applyFiltersImpl(std::move(added_filter_nodes), true); +} + +void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + pipeline.init(createPipe()); +} +} diff --git a/src/Storages/System/ReadFromSystemTables.h b/src/Storages/System/ReadFromSystemTables.h new file mode 100644 index 00000000000..f8f34363159 --- /dev/null +++ b/src/Storages/System/ReadFromSystemTables.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ReadFromSystemTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +protected: + void applyFiltersImpl(ActionDAGNodes added_filter_nodes, bool need_detached_tables_); + + template + Pipe createPipe() + { + return Pipe(std::make_shared( + std::move(columns_mask), + getOutputStream().header, + max_block_size, + std::move(filtered_databases_column), + std::move(filtered_tables_column), + context)); + } + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + +class ReadFromSystemDetachedTables : public ReadFromSystemTables +{ +public: + std::string getName() const override { return "ReadFromSystemDetachedTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; +}; +} diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 9805fa2a8cf..99c15fdae0e 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -22,204 +23,6 @@ namespace DB { -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample{ - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - auto table_it = database->getDetachedTablesIterator(context, {}, false); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->table()); - } - } - - Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -class DetachedTablesBlockSource : public ISource -{ -public: - DetachedTablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr detached_tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = detached_tables_->size(); - detached_tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - { - detached_tables.insert(detached_tables_->getDataAt(idx).toString()); - } - } - - String getName() const override { return "DetachedTables"; } - -protected: - Chunk generate() override - { - if (done) - return {}; - - MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - continue; - - const bool need_to_check_access_for_tables - = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!detached_tables_it || !detached_tables_it->isValid()) - detached_tables_it = database->getDetachedTablesIterator(context, {}, false); - - for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) - { - const auto detached_table_name = detached_tables_it->table(); - - if (!detached_tables.contains(detached_table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) - continue; - - fillResultColumnsByDetachedTableIterator(result_columns); - ++rows_count; - } - } - - const UInt64 num_rows = result_columns.at(0)->size(); - done = true; - return Chunk(std::move(result_columns), num_rows); - } - -private: - const std::vector columns_mask; - const UInt64 max_block_size; - const ColumnPtr databases; - NameSet detached_tables; - DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; - - void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const - { - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->database()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->table()); - - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->uuid()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->metadataPath()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->isPermanently()); - } -}; - -} - -class ReadFromSystemDetachedTables : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemDetachedTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemDetachedTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_); - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -private: - std::vector columns_mask; - size_t max_block_size; - - ColumnPtr filtered_databases_column; - ColumnPtr filtered_tables_column; -}; - StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -257,42 +60,4 @@ void StorageSystemDetachedTables::read( query_plan.addStep(std::move(reading)); } - -ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) -{ -} - -void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); -} - -void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - auto pipe = Pipe(std::make_shared( - std::move(columns_mask), - getOutputStream().header, - max_block_size, - std::move(filtered_databases_column), - std::move(filtered_tables_column), - context)); - pipeline.init(std::move(pipe)); -} } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..40190f53fd3 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,29 +1,30 @@ +#include #include -#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include #include #include #include #include #include -#include +#include +#include +#include +#include +#include +#include #include @@ -105,628 +106,6 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample { - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") - }; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - - Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - - -class TablesBlockSource : public ISource -{ -public: - TablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = tables_->size(); - tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - tables.insert(tables_->getDataAt(idx).toString()); - } - - String getName() const override { return "Tables"; } - -protected: - Chunk generate() override - { - if (done) - return {}; - - MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - size_t rows_count = 0; - while (rows_count < max_block_size) - { - if (tables_it && !tables_it->isValid()) - ++database_idx; - - while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - { - /// Database was deleted just now or the user has no access. - ++database_idx; - continue; - } - - break; - } - - /// This is for temporary tables. They are output in single block regardless to max_block_size. - if (database_idx >= databases->size()) - { - if (context->hasSessionContext()) - { - Tables external_tables = context->getSessionContext()->getExternalTables(); - - for (auto & table : external_tables) - { - size_t src_index = 0; - size_t res_index = 0; - - // database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // name - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.first); - - // uuid - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getStorageID().uuid); - - // engine - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - // is_temporary - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(1u); - - // data_paths - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_path - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_modification_time - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_version - // Temporary tables does not support replication - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_table - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // create_table_query - if (columns_mask[src_index++]) - { - auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); - ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - } - - // engine_full - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - const auto & settings = context->getSettingsRef(); - while (src_index < columns_mask.size()) - { - // total_rows - if (src_index == 19 && columns_mask[src_index]) - { - if (auto total_rows = table.second->totalRows(settings)) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - // total_bytes - else if (src_index == 20 && columns_mask[src_index]) - { - if (auto total_bytes = table.second->totalBytes(settings)) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - /// Fill the rest columns with defaults - else if (columns_mask[src_index]) - res_columns[res_index++]->insertDefault(); - src_index++; - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - done = true; - return Chunk(std::move(res_columns), num_rows); - } - - const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(context); - - const bool need_table = needTable(database, getPort().getHeader()); - - for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) - { - auto table_name = tables_it->name(); - if (!tables.contains(table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) - continue; - - StoragePtr table = nullptr; - TableLockHolder lock; - if (need_table) - { - table = tables_it->table(); - if (!table) - // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) - continue; - - /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) - /// and it's not protected internally by other mutexes - static const size_t DATA_PATHS_INDEX = 5; - if (columns_mask[DATA_PATHS_INDEX]) - { - lock = table->tryLockForShare(context->getCurrentQueryId(), - context->getSettingsRef().lock_acquire_timeout); - if (!lock) - // Table was dropped while acquiring the lock, skipping table - continue; - } - } - ++rows_count; - - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(tables_it->uuid()); - - if (columns_mask[src_index++]) - { - chassert(table != nullptr); - res_columns[res_index++]->insert(table->getName()); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(0u); // is_temporary - - if (columns_mask[src_index++]) - { - chassert(lock != nullptr); - Array table_paths_array; - auto paths = table->getDataPaths(); - table_paths_array.reserve(paths.size()); - for (const String & path : paths) - table_paths_array.push_back(path); - res_columns[res_index++]->insert(table_paths_array); - /// We don't need the lock anymore - lock = nullptr; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); - - StorageMetadataPtr metadata_snapshot; - if (table) - metadata_snapshot = table->getInMemoryMetadataPtr(); - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && table->supportsReplication()) - res_columns[res_index++]->insert(metadata_snapshot->metadata_version); - else - res_columns[res_index++]->insertDefault(); - } - - { - Array views_table_name_array; - Array views_database_name_array; - if (columns_mask[src_index] || columns_mask[src_index + 1]) - { - const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); - - views_table_name_array.reserve(view_ids.size()); - views_database_name_array.reserve(view_ids.size()); - for (const auto & view_id : view_ids) - { - views_table_name_array.push_back(view_id.table_name); - views_database_name_array.push_back(view_id.database_name); - } - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_database_name_array); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_table_name_array); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) - { - ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); - auto * ast_create = ast ? ast->as() : nullptr; - - if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) - { - ast_create->uuid = UUIDHelpers::Nil; - ast_create->to_inner_uuid = UUIDHelpers::Nil; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - - if (columns_mask[src_index++]) - { - String engine_full; - - if (ast_create && ast_create->storage) - { - engine_full = format({context, *ast_create->storage}); - - static const char * const extra_head = " ENGINE = "; - if (startsWith(engine_full, extra_head)) - engine_full = engine_full.substr(strlen(extra_head)); - } - - res_columns[res_index++]->insert(engine_full); - } - - if (columns_mask[src_index++]) - { - String as_select; - if (ast_create && ast_create->select) - as_select = format({context, *ast_create->select}); - res_columns[res_index++]->insert(as_select); - } - } - else - src_index += 3; - - ASTPtr expression_ptr; - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto policy = table ? table->getStoragePolicy() : nullptr; - if (policy) - res_columns[res_index++]->insert(policy->getName()); - else - res_columns[res_index++]->insertDefault(); - } - - auto settings = context->getSettingsRef(); - settings.select_sequential_consistency = 0; - if (columns_mask[src_index++]) - { - auto total_rows = table ? table->totalRows(settings) : std::nullopt; - if (total_rows) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes = table->totalBytes(settings); - if (total_bytes) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); - if (total_bytes_uncompressed) - res_columns[res_index++]->insert(*total_bytes_uncompressed); - else - res_columns[res_index++]->insertDefault(); - } - - auto table_merge_tree = std::dynamic_pointer_cast(table); - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - { - res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); - } - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; - if (lifetime_rows) - res_columns[res_index++]->insert(*lifetime_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; - if (lifetime_bytes) - res_columns[res_index++]->insert(*lifetime_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot) - res_columns[res_index++]->insert(metadata_snapshot->comment); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table) - res_columns[res_index++]->insert(table->storesDataOnDisk()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) - { - auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); - auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); - - Array dependencies_databases; - Array dependencies_tables; - dependencies_databases.reserve(dependencies.size()); - dependencies_tables.reserve(dependencies.size()); - for (const auto & dependency : dependencies) - { - dependencies_databases.push_back(dependency.database_name); - dependencies_tables.push_back(dependency.table_name); - } - - Array dependents_databases; - Array dependents_tables; - dependents_databases.reserve(dependents.size()); - dependents_tables.reserve(dependents.size()); - for (const auto & dependent : dependents) - { - dependents_databases.push_back(dependent.database_name); - dependents_tables.push_back(dependent.table_name); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_tables); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_tables); - - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - return Chunk(std::move(res_columns), num_rows); - } -private: - std::vector columns_mask; - UInt64 max_block_size; - ColumnPtr databases; - NameSet tables; - size_t database_idx = 0; - DatabaseTablesIteratorPtr tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; -}; - -} - -class ReadFromSystemTables : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_) - : SourceStepWithFilter( - DataStream{.header = std::move(sample_block)}, - column_names_, - query_info_, - storage_snapshot_, - context_) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - { - } - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -private: - std::vector columns_mask; - size_t max_block_size; - - ColumnPtr filtered_databases_column; - ColumnPtr filtered_tables_column; -}; - void StorageSystemTables::read( QueryPlan & query_plan, const Names & column_names, @@ -747,24 +126,4 @@ void StorageSystemTables::read( query_plan.addStep(std::move(reading)); } - -void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); -} - -void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - Pipe pipe(std::make_shared( - std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context)); - pipeline.init(std::move(pipe)); -} - } diff --git a/src/Storages/System/TablesBlockSource.cpp b/src/Storages/System/TablesBlockSource.cpp new file mode 100644 index 00000000000..d5481856f1b --- /dev/null +++ b/src/Storages/System/TablesBlockSource.cpp @@ -0,0 +1,564 @@ +#include "TablesBlockSource.h" + +#include + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = {"database", "name", "uuid", "metadata_modification_time"}; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} + +} + +Chunk TablesBlockSource::generate() +{ + if (done) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + { + /// Database was deleted just now or the user has no access. + ++database_idx; + continue; + } + + break; + } + + /// This is for temporary tables. They are output in single block regardless to max_block_size. + if (database_idx >= databases->size()) + { + if (context->hasSessionContext()) + { + Tables external_tables = context->getSessionContext()->getExternalTables(); + + for (auto & table : external_tables) + { + size_t src_index = 0; + size_t res_index = 0; + + // database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // name + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.first); + + // uuid + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getStorageID().uuid); + + // engine + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + // is_temporary + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(1u); + + // data_paths + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_path + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_modification_time + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_version + // Temporary tables does not support replication + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_table + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // create_table_query + if (columns_mask[src_index++]) + { + auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); + ASTPtr ast + = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + } + + // engine_full + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + const auto & settings = context->getSettingsRef(); + while (src_index < columns_mask.size()) + { + // total_rows + if (src_index == 19 && columns_mask[src_index]) + { + if (auto total_rows = table.second->totalRows(settings)) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + // total_bytes + else if (src_index == 20 && columns_mask[src_index]) + { + if (auto total_bytes = table.second->totalBytes(settings)) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + /// Fill the rest columns with defaults + else if (columns_mask[src_index]) + res_columns[res_index++]->insertDefault(); + src_index++; + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + done = true; + return Chunk(std::move(res_columns), num_rows); + } + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool need_table = needTable(database, getPort().getHeader()); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (!tables.contains(table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + StoragePtr table = nullptr; + TableLockHolder lock; + if (need_table) + { + table = tables_it->table(); + if (!table) + // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) + continue; + + /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) + /// and it's not protected internally by other mutexes + static const size_t DATA_PATHS_INDEX = 5; + if (columns_mask[DATA_PATHS_INDEX]) + { + lock = table->tryLockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); + if (!lock) + // Table was dropped while acquiring the lock, skipping table + continue; + } + } + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(tables_it->uuid()); + + if (columns_mask[src_index++]) + { + chassert(table != nullptr); + res_columns[res_index++]->insert(table->getName()); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(0u); // is_temporary + + if (columns_mask[src_index++]) + { + chassert(lock != nullptr); + Array table_paths_array; + auto paths = table->getDataPaths(); + table_paths_array.reserve(paths.size()); + for (const String & path : paths) + table_paths_array.push_back(path); + res_columns[res_index++]->insert(table_paths_array); + /// We don't need the lock anymore + lock = nullptr; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); + + StorageMetadataPtr metadata_snapshot; + if (table) + metadata_snapshot = table->getInMemoryMetadataPtr(); + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && table->supportsReplication()) + res_columns[res_index++]->insert(metadata_snapshot->metadata_version); + else + res_columns[res_index++]->insertDefault(); + } + + { + Array views_table_name_array; + Array views_database_name_array; + if (columns_mask[src_index] || columns_mask[src_index + 1]) + { + const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); + + views_table_name_array.reserve(view_ids.size()); + views_database_name_array.reserve(view_ids.size()); + for (const auto & view_id : view_ids) + { + views_table_name_array.push_back(view_id.table_name); + views_database_name_array.push_back(view_id.database_name); + } + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_database_name_array); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_table_name_array); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) + { + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + auto * ast_create = ast ? ast->as() : nullptr; + + if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + ast_create->uuid = UUIDHelpers::Nil; + ast_create->to_inner_uuid = UUIDHelpers::Nil; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + + if (columns_mask[src_index++]) + { + String engine_full; + + if (ast_create && ast_create->storage) + { + engine_full = format({context, *ast_create->storage}); + + static const char * const extra_head = " ENGINE = "; + if (startsWith(engine_full, extra_head)) + engine_full = engine_full.substr(strlen(extra_head)); + } + + res_columns[res_index++]->insert(engine_full); + } + + if (columns_mask[src_index++]) + { + String as_select; + if (ast_create && ast_create->select) + as_select = format({context, *ast_create->select}); + res_columns[res_index++]->insert(as_select); + } + } + else + src_index += 3; + + ASTPtr expression_ptr; + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto policy = table ? table->getStoragePolicy() : nullptr; + if (policy) + res_columns[res_index++]->insert(policy->getName()); + else + res_columns[res_index++]->insertDefault(); + } + + auto settings = context->getSettingsRef(); + settings.select_sequential_consistency = 0; + if (columns_mask[src_index++]) + { + auto total_rows = table ? table->totalRows(settings) : std::nullopt; + if (total_rows) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes = table->totalBytes(settings); + if (total_bytes) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); + if (total_bytes_uncompressed) + res_columns[res_index++]->insert(*total_bytes_uncompressed); + else + res_columns[res_index++]->insertDefault(); + } + + auto table_merge_tree = std::dynamic_pointer_cast(table); + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + { + res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); + } + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; + if (lifetime_rows) + res_columns[res_index++]->insert(*lifetime_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; + if (lifetime_bytes) + res_columns[res_index++]->insert(*lifetime_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot) + res_columns[res_index++]->insert(metadata_snapshot->comment); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table) + res_columns[res_index++]->insert(table->storesDataOnDisk()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) + { + auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); + + Array dependencies_databases; + Array dependencies_tables; + dependencies_databases.reserve(dependencies.size()); + dependencies_tables.reserve(dependencies.size()); + for (const auto & dependency : dependencies) + { + dependencies_databases.push_back(dependency.database_name); + dependencies_tables.push_back(dependency.table_name); + } + + Array dependents_databases; + Array dependents_tables; + dependents_databases.reserve(dependents.size()); + dependents_tables.reserve(dependents.size()); + for (const auto & dependent : dependents) + { + dependents_databases.push_back(dependent.database_name); + dependents_tables.push_back(dependent.table_name); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_tables); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_tables); + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + return Chunk(std::move(res_columns), num_rows); +} + +Chunk DetachedTablesBlockSource::generate() +{ + if (done) + return {}; + + MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + continue; + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!detached_tables_it || !detached_tables_it->isValid()) + detached_tables_it = database->getDetachedTablesIterator(context, {}, false); + + for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) + { + const auto detached_table_name = detached_tables_it->table(); + + if (!detached_tables.contains(detached_table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) + continue; + + fillResultColumnsByDetachedTableIterator(result_columns); + ++rows_count; + } + } + + const UInt64 num_rows = result_columns.at(0)->size(); + done = true; + return Chunk(std::move(result_columns), num_rows); +} + +void DetachedTablesBlockSource::fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const +{ + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->database()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->table()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->uuid()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->metadataPath()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->isPermanently()); +} +} diff --git a/src/Storages/System/TablesBlockSource.h b/src/Storages/System/TablesBlockSource.h new file mode 100644 index 00000000000..a9f2226dca8 --- /dev/null +++ b/src/Storages/System/TablesBlockSource.h @@ -0,0 +1,92 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class TablesBlockSource : public ISource +{ +public: + TablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = tables_->size(); + tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + tables.insert(tables_->getDataAt(idx).toString()); + } + + String getName() const override { return "Tables"; } + +protected: + Chunk generate() override; + +private: + std::vector columns_mask; + UInt64 max_block_size; + ColumnPtr databases; + NameSet tables; + size_t database_idx = 0; + DatabaseTablesIteratorPtr tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; +}; + +class DetachedTablesBlockSource : public ISource +{ +public: + DetachedTablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr detached_tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = detached_tables_->size(); + detached_tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + { + detached_tables.insert(detached_tables_->getDataAt(idx).toString()); + } + } + + String getName() const override { return "DetachedTables"; } + +protected: + Chunk generate() override; + +private: + const std::vector columns_mask; + const UInt64 max_block_size; + const ColumnPtr databases; + NameSet detached_tables; + DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; + + void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const; +}; +} From e06387e6da4a89d9aab145dd057b6a99348e3ee8 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 07:37:23 +0000 Subject: [PATCH 015/178] fix stle --- .../test_system_detached_tables/test.py | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index bbac5e391f8..45bc70147b6 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -18,11 +18,19 @@ def test_system_detached_tables(): node.query("CREATE TABLE test_table (n Int64) ENGINE=MergeTree ORDER BY n;") node.query("CREATE TABLE test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n;") - test_table_uuid = node.query("SELECT uuid FROM system.tables WHERE table='test_table'").rstrip('\n') - test_table_metadata_path = node.query("SELECT metadata_path FROM system.tables WHERE table='test_table'").rstrip('\n') + test_table_uuid = node.query( + "SELECT uuid FROM system.tables WHERE table='test_table'" + ).rstrip('\n') + test_table_metadata_path = node.query( + "SELECT metadata_path FROM system.tables WHERE table='test_table'" + ).rstrip('\n') - test_table_perm_uuid = node.query("SELECT uuid FROM system.tables WHERE table='test_table_perm'").rstrip('\n') - test_table_perm_metadata_path = node.query("SELECT metadata_path FROM system.tables WHERE table='test_table_perm'").rstrip('\n') + test_table_perm_uuid = node.query( + "SELECT uuid FROM system.tables WHERE table='test_table_perm'" + ).rstrip('\n') + test_table_perm_metadata_path = node.query( + "SELECT metadata_path FROM system.tables WHERE table='test_table_perm'" + ).rstrip('\n') assert "" == node.query("SELECT * FROM system.detached_tables") @@ -31,12 +39,17 @@ def test_system_detached_tables(): querry = "SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables FORMAT Values" result = node.query(querry) - assert result == f"('default','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" - - node.restart_clickhouse() - - result = node.query(querry) - assert result == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + assert ( + result + == f"('default','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + ) + node.restart_clickhouse() + + result = node.query(querry) + assert ( + result + == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + ) node.restart_clickhouse() From 1c9ccbae1142d585f7534441a6abc5e75acfdce3 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 07:49:07 +0000 Subject: [PATCH 016/178] small up --- src/Databases/DatabasesCommon.cpp | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 470625ddd8b..4cf914fea56 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -270,37 +270,36 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(ContextPtr /* context_ */, con StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name) { - StoragePtr res; - auto it = tables.find(table_name); if (it == tables.end()) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuote(database_name), backQuote(table_name)); - res = it->second; + + auto table_storage = it->second; snapshot_detached_tables.emplace( - it->first, + table_name, SnapshotDetachedTable{ .database = it->second->getStorageID().getDatabaseName(), - .table = it->first, + .table = table_name, .uuid = it->second->getStorageID().uuid, - .metadata_path = getObjectMetadataPath(it->first), + .metadata_path = getObjectMetadataPath(table_name), .is_permanently = false}); tables.erase(it); - res->is_detached = true; + table_storage->is_detached = true; - if (res->isSystemStorage() == false) - CurrentMetrics::sub(getAttachedCounterForStorage(res), 1); + if (table_storage->isSystemStorage() == false) + CurrentMetrics::sub(getAttachedCounterForStorage(table_storage), 1); - auto table_id = res->getStorageID(); + auto table_id = table_storage->getStorageID(); if (table_id.hasUUID()) { assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getUUID() != UUIDHelpers::Nil); DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid); } - return res; + return table_storage; } void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) From 215802e74eb841449af6cb80f25b66a3610707dd Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 08:28:35 +0000 Subject: [PATCH 017/178] fix style --- .../test_system_detached_tables/test.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index 45bc70147b6..dbccac92523 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -20,17 +20,17 @@ def test_system_detached_tables(): test_table_uuid = node.query( "SELECT uuid FROM system.tables WHERE table='test_table'" - ).rstrip('\n') + ).rstrip("\n") test_table_metadata_path = node.query( "SELECT metadata_path FROM system.tables WHERE table='test_table'" - ).rstrip('\n') + ).rstrip("\n") test_table_perm_uuid = node.query( "SELECT uuid FROM system.tables WHERE table='test_table_perm'" - ).rstrip('\n') + ).rstrip("\n") test_table_perm_metadata_path = node.query( "SELECT metadata_path FROM system.tables WHERE table='test_table_perm'" - ).rstrip('\n') + ).rstrip("\n") assert "" == node.query("SELECT * FROM system.detached_tables") @@ -40,7 +40,7 @@ def test_system_detached_tables(): querry = "SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables FORMAT Values" result = node.query(querry) assert ( - result + result == f"('default','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" ) node.restart_clickhouse() @@ -54,4 +54,7 @@ def test_system_detached_tables(): node.restart_clickhouse() result = node.query(querry) - assert result == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + assert ( + result + == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + ) From bda41ab03d650978cb6765746d786b25de9ed72c Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 10:52:04 +0000 Subject: [PATCH 018/178] fix tests for CI --- src/Databases/DatabasesCommon.cpp | 8 ++++---- src/Databases/IDatabase.h | 4 +++- src/Databases/TablesLoader.cpp | 10 +++++----- .../0_stateless/03172_system_detached_tables.sql | 12 ++++++------ 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 4cf914fea56..fe0baf30e57 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -2,10 +2,6 @@ #include #include -#include -#include -#include -#include #include #include #include @@ -17,6 +13,10 @@ #include #include #include +#include +#include +#include +#include namespace DB diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index e856dd3d2f0..ce5a52b1b0f 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -138,7 +138,9 @@ protected: } public: - explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin()) { } + explicit DatabaseDetachedTablesSnapshotIterator(const SnapshotDetachedTables & tables_) : snapshot(tables_), it(snapshot.begin()) + { + } explicit DatabaseDetachedTablesSnapshotIterator(SnapshotDetachedTables && tables_) : snapshot(std::move(tables_)), it(snapshot.begin()) { diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 6aa13b7b759..1f70f46da60 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -1,14 +1,14 @@ -#include -#include +#include #include #include -#include +#include +#include #include +#include #include #include -#include #include -#include +#include namespace DB diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql index 558dfe4cf18..e361d2f34ca 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -4,23 +4,23 @@ DROP DATABASE IF EXISTS test_system_detached_tables; CREATE DATABASE IF NOT EXISTS test_system_detached_tables; CREATE TABLE test_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables; +SELECT * FROM system.detached_tables WHERE database='test_system_detached_tables'; DETACH TABLE test_system_detached_tables.test_table; -SELECT database, table, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; ATTACH TABLE test_system_detached_tables.test_table; CREATE TABLE test_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables; +SELECT * FROM system.detached_tables WHERE database='test_system_detached_tables'; DETACH TABLE test_system_detached_tables.test_table_perm PERMANENTLY; -SELECT database, table, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; DETACH TABLE test_system_detached_tables.test_table SYNC; -SELECT database, table, is_permanently FROM system.detached_tables; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; -SELECT database, table, is_permanently FROM system.detached_tables WHERE table='test_table'; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables' AND table='test_table'; DROP DATABASE test_system_detached_tables; From 6d1f37ad3db0078514dd76e857ef10474afd2ed4 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 15:37:27 +0000 Subject: [PATCH 019/178] support lazy db --- src/Databases/DatabaseLazy.cpp | 16 ++++++ .../03172_system_detached_tables.reference | 16 ++++-- .../03172_system_detached_tables.sql | 55 ++++++++++++++----- 3 files changed, 67 insertions(+), 20 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 233db07cd68..3c398e50371 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -187,6 +187,13 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); + LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name)); + if (snapshot_detached_tables.contains(table_name)) + { + LOG_DEBUG(log, "Clean info about detached table {} from snapshot.", backQuote(table_name)); + snapshot_detached_tables.erase(table_name); + } + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } @@ -203,6 +210,15 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); tables_cache.erase(it); + LOG_DEBUG(log, "Add info for detached table {} to snapshot.", backQuote(table_name)); + snapshot_detached_tables.emplace( + table_name, + SnapshotDetachedTable{ + .database = res->getStorageID().database_name, + .table = res->getStorageID().table_name, + .uuid = res->getStorageID().uuid, + .metadata_path = getObjectMetadataPath(table_name), + .is_permanently = false}); CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); } diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference index 1cf9e0275c9..ce19daa4cdc 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.reference +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -1,5 +1,11 @@ -test_system_detached_tables test_table 0 -test_system_detached_tables test_table_perm 1 -test_system_detached_tables test_table 0 -test_system_detached_tables test_table_perm 1 -test_system_detached_tables test_table 0 +database atomic tests +test03172_system_detached_tables test_table 0 +test03172_system_detached_tables test_table_perm 1 +test03172_system_detached_tables test_table 0 +test03172_system_detached_tables test_table_perm 1 +test03172_system_detached_tables test_table 0 +----------------------- +database lazy tests +test03172_system_detached_tables_lazy test_table 0 +test03172_system_detached_tables_lazy test_table_perm 1 +DROP TABLE diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql index e361d2f34ca..916821d996b 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -1,28 +1,53 @@ -- Tags: no-parallel -DROP DATABASE IF EXISTS test_system_detached_tables; -CREATE DATABASE IF NOT EXISTS test_system_detached_tables; +SELECT 'database atomic tests'; +DROP DATABASE IF EXISTS test03172_system_detached_tables; +CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables; -CREATE TABLE test_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test_system_detached_tables'; +CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; -DETACH TABLE test_system_detached_tables.test_table; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; +DETACH TABLE test03172_system_detached_tables.test_table; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; -ATTACH TABLE test_system_detached_tables.test_table; +ATTACH TABLE test03172_system_detached_tables.test_table; -CREATE TABLE test_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; -SELECT * FROM system.detached_tables WHERE database='test_system_detached_tables'; +CREATE TABLE test03172_system_detached_tables.test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n; +SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; -DETACH TABLE test_system_detached_tables.test_table_perm PERMANENTLY; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; +DETACH TABLE test03172_system_detached_tables.test_table_perm PERMANENTLY; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; -DETACH TABLE test_system_detached_tables.test_table SYNC; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables'; +DETACH TABLE test03172_system_detached_tables.test_table SYNC; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables'; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test_system_detached_tables' AND table='test_table'; +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables' AND table='test_table'; -DROP DATABASE test_system_detached_tables; +DROP DATABASE test03172_system_detached_tables SYNC; +SELECT '-----------------------'; +SELECT 'database lazy tests'; +DROP DATABASE IF EXISTS test03172_system_detached_tables_lazy; +CREATE DATABASE test03172_system_detached_tables_lazy Engine=Lazy(10); +CREATE TABLE test03172_system_detached_tables_lazy.test_table (number UInt64) engine=Log; +INSERT INTO test03172_system_detached_tables_lazy.test_table SELECT * FROM numbers(100); +DETACH TABLE test03172_system_detached_tables_lazy.test_table; + +CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt64) engine=Log; +INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100); +DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY; + +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; + +ATTACH TABLE test03172_system_detached_tables_lazy.test_table; +ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm; + +SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; + +SELECT 'DROP TABLE'; +DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC; +DROP TABLE test03172_system_detached_tables_lazy.test_table_perm SYNC; + +DROP DATABASE test03172_system_detached_tables_lazy SYNC; From 062198b95642a52ec0c67bbc25e7d5da45814b33 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 27 Jun 2024 15:44:37 +0000 Subject: [PATCH 020/178] add checking and exception --- src/Databases/DatabaseOnDisk.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 25e5347e017..67c9114e793 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -22,6 +22,7 @@ #include #include #include +#include "Common/Exception.h" #include #include #include @@ -309,7 +310,11 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri FS::createFile(detached_permanently_flag); std::lock_guard lock(mutex); - snapshot_detached_tables.at(table_name).is_permanently = true; + if (!snapshot_detached_tables.contains(table_name)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); + } + snapshot_detached_tables[table_name].is_permanently = true; } catch (Exception & e) { From 840edd99fa1bc2325ab5f875c2c9a65ddf67c579 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 27 Jun 2024 21:53:23 +0000 Subject: [PATCH 021/178] Apply https://github.com/ClickHouse/libunwind/pull/27 too --- contrib/libunwind | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libunwind b/contrib/libunwind index 010ea7c5f9d..8f28e64d158 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 010ea7c5f9de966b37a6fcda855d52bdfe0a6ac6 +Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 From 818e8597a22cf87c560d7986818f8a094cd7db1f Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 28 Jun 2024 12:27:40 +0000 Subject: [PATCH 022/178] Revert "general logic for tables" This reverts commit de8f1fc6e3adb6c8f346b2fda4e34c2240e1bf8e. --- src/Storages/System/ReadFromSystemTables.cpp | 157 ---- src/Storages/System/ReadFromSystemTables.h | 66 -- .../System/StorageSystemDetachedTables.cpp | 237 ++++++- src/Storages/System/StorageSystemTables.cpp | 669 +++++++++++++++++- src/Storages/System/TablesBlockSource.cpp | 564 --------------- src/Storages/System/TablesBlockSource.h | 92 --- 6 files changed, 891 insertions(+), 894 deletions(-) delete mode 100644 src/Storages/System/ReadFromSystemTables.cpp delete mode 100644 src/Storages/System/ReadFromSystemTables.h delete mode 100644 src/Storages/System/TablesBlockSource.cpp delete mode 100644 src/Storages/System/TablesBlockSource.h diff --git a/src/Storages/System/ReadFromSystemTables.cpp b/src/Storages/System/ReadFromSystemTables.cpp deleted file mode 100644 index 638cab1b63b..00000000000 --- a/src/Storages/System/ReadFromSystemTables.cpp +++ /dev/null @@ -1,157 +0,0 @@ -#include "ReadFromSystemTables.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables( - const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool need_detached_tables) -{ - Block sample{ - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - if (need_detached_tables) - { - auto table_it = database->getDetachedTablesIterator(context, {}, false); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->table()); - } - } - else - { - auto table_it = database->getTablesIterator(context); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - } - - Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -} - -ReadFromSystemTables::ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) -{ -} - -void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - applyFiltersImpl(std::move(added_filter_nodes), false); -} - -void ReadFromSystemTables::applyFiltersImpl(ActionDAGNodes added_filter_nodes, const bool need_detached_tables_) -{ - SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context, need_detached_tables_); -} - -void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.init(createPipe()); -} - -ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - const size_t max_block_size_) - : ReadFromSystemTables( - column_names_, query_info_, storage_snapshot_, context_, std::move(sample_block), std::move(columns_mask_), max_block_size_) -{ -} - -void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) -{ - applyFiltersImpl(std::move(added_filter_nodes), true); -} - -void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.init(createPipe()); -} -} diff --git a/src/Storages/System/ReadFromSystemTables.h b/src/Storages/System/ReadFromSystemTables.h deleted file mode 100644 index f8f34363159..00000000000 --- a/src/Storages/System/ReadFromSystemTables.h +++ /dev/null @@ -1,66 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ReadFromSystemTables : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromSystemTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_); - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - -protected: - void applyFiltersImpl(ActionDAGNodes added_filter_nodes, bool need_detached_tables_); - - template - Pipe createPipe() - { - return Pipe(std::make_shared( - std::move(columns_mask), - getOutputStream().header, - max_block_size, - std::move(filtered_databases_column), - std::move(filtered_tables_column), - context)); - } - -private: - std::vector columns_mask; - size_t max_block_size; - - ColumnPtr filtered_databases_column; - ColumnPtr filtered_tables_column; -}; - -class ReadFromSystemDetachedTables : public ReadFromSystemTables -{ -public: - std::string getName() const override { return "ReadFromSystemDetachedTables"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - ReadFromSystemDetachedTables( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - std::vector columns_mask_, - size_t max_block_size_); - - void applyFilters(ActionDAGNodes added_filter_nodes) override; -}; -} diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 99c15fdae0e..9805fa2a8cf 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include @@ -23,6 +22,204 @@ namespace DB { +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +class DetachedTablesBlockSource : public ISource +{ +public: + DetachedTablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr detached_tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = detached_tables_->size(); + detached_tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + { + detached_tables.insert(detached_tables_->getDataAt(idx).toString()); + } + } + + String getName() const override { return "DetachedTables"; } + +protected: + Chunk generate() override + { + if (done) + return {}; + + MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + continue; + + const bool need_to_check_access_for_tables + = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!detached_tables_it || !detached_tables_it->isValid()) + detached_tables_it = database->getDetachedTablesIterator(context, {}, false); + + for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) + { + const auto detached_table_name = detached_tables_it->table(); + + if (!detached_tables.contains(detached_table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) + continue; + + fillResultColumnsByDetachedTableIterator(result_columns); + ++rows_count; + } + } + + const UInt64 num_rows = result_columns.at(0)->size(); + done = true; + return Chunk(std::move(result_columns), num_rows); + } + +private: + const std::vector columns_mask; + const UInt64 max_block_size; + const ColumnPtr databases; + NameSet detached_tables; + DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; + + void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const + { + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->database()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->table()); + + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->uuid()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->metadataPath()); + + if (columns_mask[src_index++]) + result_columns[res_index++]->insert(detached_tables_it->isPermanently()); + } +}; + +} + +class ReadFromSystemDetachedTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemDetachedTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_); + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + StorageSystemDetachedTables::StorageSystemDetachedTables(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -60,4 +257,42 @@ void StorageSystemDetachedTables::read( query_plan.addStep(std::move(reading)); } + +ReadFromSystemDetachedTables::ReadFromSystemDetachedTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) +{ +} + +void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); +} + +void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + auto pipe = Pipe(std::make_shared( + std::move(columns_mask), + getOutputStream().header, + max_block_size, + std::move(filtered_databases_column), + std::move(filtered_tables_column), + context)); + pipeline.init(std::move(pipe)); +} } diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 40190f53fd3..783b899c978 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,30 +1,29 @@ -#include #include -#include -#include -#include +#include #include #include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include #include #include #include #include #include +#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include +#include #include @@ -106,6 +105,628 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } + +namespace +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) +{ + Block sample { + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") + }; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + + Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} + + +class TablesBlockSource : public ISource +{ +public: + TablesBlockSource( + std::vector columns_mask_, + Block header, + UInt64 max_block_size_, + ColumnPtr databases_, + ColumnPtr tables_, + ContextPtr context_) + : ISource(std::move(header)) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(Context::createCopy(context_)) + { + size_t size = tables_->size(); + tables.reserve(size); + for (size_t idx = 0; idx < size; ++idx) + tables.insert(tables_->getDataAt(idx).toString()); + } + + String getName() const override { return "Tables"; } + +protected: + Chunk generate() override + { + if (done) + return {}; + + MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); + + const auto access = context->getAccess(); + const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + size_t rows_count = 0; + while (rows_count < max_block_size) + { + if (tables_it && !tables_it->isValid()) + ++database_idx; + + while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) + { + database_name = databases->getDataAt(database_idx).toString(); + database = DatabaseCatalog::instance().tryGetDatabase(database_name); + + if (!database) + { + /// Database was deleted just now or the user has no access. + ++database_idx; + continue; + } + + break; + } + + /// This is for temporary tables. They are output in single block regardless to max_block_size. + if (database_idx >= databases->size()) + { + if (context->hasSessionContext()) + { + Tables external_tables = context->getSessionContext()->getExternalTables(); + + for (auto & table : external_tables) + { + size_t src_index = 0; + size_t res_index = 0; + + // database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // name + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.first); + + // uuid + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getStorageID().uuid); + + // engine + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + // is_temporary + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(1u); + + // data_paths + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_path + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_modification_time + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // metadata_version + // Temporary tables does not support replication + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_database + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // dependencies_table + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + + // create_table_query + if (columns_mask[src_index++]) + { + auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); + ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + } + + // engine_full + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table.second->getName()); + + const auto & settings = context->getSettingsRef(); + while (src_index < columns_mask.size()) + { + // total_rows + if (src_index == 19 && columns_mask[src_index]) + { + if (auto total_rows = table.second->totalRows(settings)) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + // total_bytes + else if (src_index == 20 && columns_mask[src_index]) + { + if (auto total_bytes = table.second->totalBytes(settings)) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + /// Fill the rest columns with defaults + else if (columns_mask[src_index]) + res_columns[res_index++]->insertDefault(); + src_index++; + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + done = true; + return Chunk(std::move(res_columns), num_rows); + } + + const bool need_to_check_access_for_tables = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); + + if (!tables_it || !tables_it->isValid()) + tables_it = database->getTablesIterator(context); + + const bool need_table = needTable(database, getPort().getHeader()); + + for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) + { + auto table_name = tables_it->name(); + if (!tables.contains(table_name)) + continue; + + if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + continue; + + StoragePtr table = nullptr; + TableLockHolder lock; + if (need_table) + { + table = tables_it->table(); + if (!table) + // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) + continue; + + /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) + /// and it's not protected internally by other mutexes + static const size_t DATA_PATHS_INDEX = 5; + if (columns_mask[DATA_PATHS_INDEX]) + { + lock = table->tryLockForShare(context->getCurrentQueryId(), + context->getSettingsRef().lock_acquire_timeout); + if (!lock) + // Table was dropped while acquiring the lock, skipping table + continue; + } + } + ++rows_count; + + size_t src_index = 0; + size_t res_index = 0; + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(table_name); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(tables_it->uuid()); + + if (columns_mask[src_index++]) + { + chassert(table != nullptr); + res_columns[res_index++]->insert(table->getName()); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(0u); // is_temporary + + if (columns_mask[src_index++]) + { + chassert(lock != nullptr); + Array table_paths_array; + auto paths = table->getDataPaths(); + table_paths_array.reserve(paths.size()); + for (const String & path : paths) + table_paths_array.push_back(path); + res_columns[res_index++]->insert(table_paths_array); + /// We don't need the lock anymore + lock = nullptr; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); + + StorageMetadataPtr metadata_snapshot; + if (table) + metadata_snapshot = table->getInMemoryMetadataPtr(); + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && table->supportsReplication()) + res_columns[res_index++]->insert(metadata_snapshot->metadata_version); + else + res_columns[res_index++]->insertDefault(); + } + + { + Array views_table_name_array; + Array views_database_name_array; + if (columns_mask[src_index] || columns_mask[src_index + 1]) + { + const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); + + views_table_name_array.reserve(view_ids.size()); + views_database_name_array.reserve(view_ids.size()); + for (const auto & view_id : view_ids) + { + views_table_name_array.push_back(view_id.table_name); + views_database_name_array.push_back(view_id.database_name); + } + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_database_name_array); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(views_table_name_array); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) + { + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); + auto * ast_create = ast ? ast->as() : nullptr; + + if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) + { + ast_create->uuid = UUIDHelpers::Nil; + ast_create->to_inner_uuid = UUIDHelpers::Nil; + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); + + if (columns_mask[src_index++]) + { + String engine_full; + + if (ast_create && ast_create->storage) + { + engine_full = format({context, *ast_create->storage}); + + static const char * const extra_head = " ENGINE = "; + if (startsWith(engine_full, extra_head)) + engine_full = engine_full.substr(strlen(extra_head)); + } + + res_columns[res_index++]->insert(engine_full); + } + + if (columns_mask[src_index++]) + { + String as_select; + if (ast_create && ast_create->select) + as_select = format({context, *ast_create->select}); + res_columns[res_index++]->insert(as_select); + } + } + else + src_index += 3; + + ASTPtr expression_ptr; + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) + res_columns[res_index++]->insert(format({context, *expression_ptr})); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto policy = table ? table->getStoragePolicy() : nullptr; + if (policy) + res_columns[res_index++]->insert(policy->getName()); + else + res_columns[res_index++]->insertDefault(); + } + + auto settings = context->getSettingsRef(); + settings.select_sequential_consistency = 0; + if (columns_mask[src_index++]) + { + auto total_rows = table ? table->totalRows(settings) : std::nullopt; + if (total_rows) + res_columns[res_index++]->insert(*total_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes = table->totalBytes(settings); + if (total_bytes) + res_columns[res_index++]->insert(*total_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); + if (total_bytes_uncompressed) + res_columns[res_index++]->insert(*total_bytes_uncompressed); + else + res_columns[res_index++]->insertDefault(); + } + + auto table_merge_tree = std::dynamic_pointer_cast(table); + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table_merge_tree) + { + res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); + } + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; + if (lifetime_rows) + res_columns[res_index++]->insert(*lifetime_rows); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; + if (lifetime_bytes) + res_columns[res_index++]->insert(*lifetime_bytes); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (metadata_snapshot) + res_columns[res_index++]->insert(metadata_snapshot->comment); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index++]) + { + if (table) + res_columns[res_index++]->insert(table->storesDataOnDisk()); + else + res_columns[res_index++]->insertDefault(); + } + + if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) + { + auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); + + Array dependencies_databases; + Array dependencies_tables; + dependencies_databases.reserve(dependencies.size()); + dependencies_tables.reserve(dependencies.size()); + for (const auto & dependency : dependencies) + { + dependencies_databases.push_back(dependency.database_name); + dependencies_tables.push_back(dependency.table_name); + } + + Array dependents_databases; + Array dependents_tables; + dependents_databases.reserve(dependents.size()); + dependents_tables.reserve(dependents.size()); + for (const auto & dependent : dependents) + { + dependents_databases.push_back(dependent.database_name); + dependents_tables.push_back(dependent.table_name); + } + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependencies_tables); + + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_databases); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(dependents_tables); + + } + } + } + + UInt64 num_rows = res_columns.at(0)->size(); + return Chunk(std::move(res_columns), num_rows); + } +private: + std::vector columns_mask; + UInt64 max_block_size; + ColumnPtr databases; + NameSet tables; + size_t database_idx = 0; + DatabaseTablesIteratorPtr tables_it; + ContextPtr context; + bool done = false; + DatabasePtr database; + std::string database_name; +}; + +} + +class ReadFromSystemTables : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromSystemTables"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + ReadFromSystemTables( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + std::vector columns_mask_, + size_t max_block_size_) + : SourceStepWithFilter( + DataStream{.header = std::move(sample_block)}, + column_names_, + query_info_, + storage_snapshot_, + context_) + , columns_mask(std::move(columns_mask_)) + , max_block_size(max_block_size_) + { + } + + void applyFilters(ActionDAGNodes added_filter_nodes) override; + +private: + std::vector columns_mask; + size_t max_block_size; + + ColumnPtr filtered_databases_column; + ColumnPtr filtered_tables_column; +}; + void StorageSystemTables::read( QueryPlan & query_plan, const Names & column_names, @@ -126,4 +747,24 @@ void StorageSystemTables::read( query_plan.addStep(std::move(reading)); } + +void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + filtered_databases_column = getFilteredDatabases(predicate, context); + filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); +} + +void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipe pipe(std::make_shared( + std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases_column), std::move(filtered_tables_column), context)); + pipeline.init(std::move(pipe)); +} + } diff --git a/src/Storages/System/TablesBlockSource.cpp b/src/Storages/System/TablesBlockSource.cpp deleted file mode 100644 index d5481856f1b..00000000000 --- a/src/Storages/System/TablesBlockSource.cpp +++ /dev/null @@ -1,564 +0,0 @@ -#include "TablesBlockSource.h" - -#include - -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = {"database", "name", "uuid", "metadata_modification_time"}; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - -} - -Chunk TablesBlockSource::generate() -{ - if (done) - return {}; - - MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - size_t rows_count = 0; - while (rows_count < max_block_size) - { - if (tables_it && !tables_it->isValid()) - ++database_idx; - - while (database_idx < databases->size() && (!tables_it || !tables_it->isValid())) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - { - /// Database was deleted just now or the user has no access. - ++database_idx; - continue; - } - - break; - } - - /// This is for temporary tables. They are output in single block regardless to max_block_size. - if (database_idx >= databases->size()) - { - if (context->hasSessionContext()) - { - Tables external_tables = context->getSessionContext()->getExternalTables(); - - for (auto & table : external_tables) - { - size_t src_index = 0; - size_t res_index = 0; - - // database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // name - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.first); - - // uuid - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getStorageID().uuid); - - // engine - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - // is_temporary - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(1u); - - // data_paths - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_path - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_modification_time - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // metadata_version - // Temporary tables does not support replication - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_database - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // dependencies_table - if (columns_mask[src_index++]) - res_columns[res_index++]->insertDefault(); - - // create_table_query - if (columns_mask[src_index++]) - { - auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); - ASTPtr ast - = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - } - - // engine_full - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table.second->getName()); - - const auto & settings = context->getSettingsRef(); - while (src_index < columns_mask.size()) - { - // total_rows - if (src_index == 19 && columns_mask[src_index]) - { - if (auto total_rows = table.second->totalRows(settings)) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - // total_bytes - else if (src_index == 20 && columns_mask[src_index]) - { - if (auto total_bytes = table.second->totalBytes(settings)) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - /// Fill the rest columns with defaults - else if (columns_mask[src_index]) - res_columns[res_index++]->insertDefault(); - src_index++; - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - done = true; - return Chunk(std::move(res_columns), num_rows); - } - - const bool need_to_check_access_for_tables - = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(context); - - const bool need_table = needTable(database, getPort().getHeader()); - - for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) - { - auto table_name = tables_it->name(); - if (!tables.contains(table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) - continue; - - StoragePtr table = nullptr; - TableLockHolder lock; - if (need_table) - { - table = tables_it->table(); - if (!table) - // Table might have just been removed or detached for Lazy engine (see DatabaseLazy::tryGetTable()) - continue; - - /// The only column that requires us to hold a shared lock is data_paths as rename might alter them (on ordinary tables) - /// and it's not protected internally by other mutexes - static const size_t DATA_PATHS_INDEX = 5; - if (columns_mask[DATA_PATHS_INDEX]) - { - lock = table->tryLockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout); - if (!lock) - // Table was dropped while acquiring the lock, skipping table - continue; - } - } - ++rows_count; - - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(table_name); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(tables_it->uuid()); - - if (columns_mask[src_index++]) - { - chassert(table != nullptr); - res_columns[res_index++]->insert(table->getName()); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(0u); // is_temporary - - if (columns_mask[src_index++]) - { - chassert(lock != nullptr); - Array table_paths_array; - auto paths = table->getDataPaths(); - table_paths_array.reserve(paths.size()); - for (const String & path : paths) - table_paths_array.push_back(path); - res_columns[res_index++]->insert(table_paths_array); - /// We don't need the lock anymore - lock = nullptr; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(table_name))); - - StorageMetadataPtr metadata_snapshot; - if (table) - metadata_snapshot = table->getInMemoryMetadataPtr(); - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && table->supportsReplication()) - res_columns[res_index++]->insert(metadata_snapshot->metadata_version); - else - res_columns[res_index++]->insertDefault(); - } - - { - Array views_table_name_array; - Array views_database_name_array; - if (columns_mask[src_index] || columns_mask[src_index + 1]) - { - const auto view_ids = DatabaseCatalog::instance().getDependentViews(StorageID(database_name, table_name)); - - views_table_name_array.reserve(view_ids.size()); - views_database_name_array.reserve(view_ids.size()); - for (const auto & view_id : view_ids) - { - views_table_name_array.push_back(view_id.table_name); - views_database_name_array.push_back(view_id.database_name); - } - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_database_name_array); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(views_table_name_array); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2]) - { - ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); - auto * ast_create = ast ? ast->as() : nullptr; - - if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil) - { - ast_create->uuid = UUIDHelpers::Nil; - ast_create->to_inner_uuid = UUIDHelpers::Nil; - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(ast ? format({context, *ast}) : ""); - - if (columns_mask[src_index++]) - { - String engine_full; - - if (ast_create && ast_create->storage) - { - engine_full = format({context, *ast_create->storage}); - - static const char * const extra_head = " ENGINE = "; - if (startsWith(engine_full, extra_head)) - engine_full = engine_full.substr(strlen(extra_head)); - } - - res_columns[res_index++]->insert(engine_full); - } - - if (columns_mask[src_index++]) - { - String as_select; - if (ast_create && ast_create->select) - as_select = format({context, *ast_create->select}); - res_columns[res_index++]->insert(as_select); - } - } - else - src_index += 3; - - ASTPtr expression_ptr; - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) - res_columns[res_index++]->insert(format({context, *expression_ptr})); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto policy = table ? table->getStoragePolicy() : nullptr; - if (policy) - res_columns[res_index++]->insert(policy->getName()); - else - res_columns[res_index++]->insertDefault(); - } - - auto settings = context->getSettingsRef(); - settings.select_sequential_consistency = 0; - if (columns_mask[src_index++]) - { - auto total_rows = table ? table->totalRows(settings) : std::nullopt; - if (total_rows) - res_columns[res_index++]->insert(*total_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes = table->totalBytes(settings); - if (total_bytes) - res_columns[res_index++]->insert(*total_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto total_bytes_uncompressed = table->totalBytesUncompressed(settings); - if (total_bytes_uncompressed) - res_columns[res_index++]->insert(*total_bytes_uncompressed); - else - res_columns[res_index++]->insertDefault(); - } - - auto table_merge_tree = std::dynamic_pointer_cast(table); - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getAllPartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - res_columns[res_index++]->insert(table_merge_tree->getActivePartsCount()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table_merge_tree) - { - res_columns[res_index++]->insert(table_merge_tree->getTotalMarksCount()); - } - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_rows = table ? table->lifetimeRows() : std::nullopt; - if (lifetime_rows) - res_columns[res_index++]->insert(*lifetime_rows); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - auto lifetime_bytes = table ? table->lifetimeBytes() : std::nullopt; - if (lifetime_bytes) - res_columns[res_index++]->insert(*lifetime_bytes); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (metadata_snapshot) - res_columns[res_index++]->insert(metadata_snapshot->comment); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index++]) - { - if (table) - res_columns[res_index++]->insert(table->storesDataOnDisk()); - else - res_columns[res_index++]->insertDefault(); - } - - if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) - { - auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); - auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); - - Array dependencies_databases; - Array dependencies_tables; - dependencies_databases.reserve(dependencies.size()); - dependencies_tables.reserve(dependencies.size()); - for (const auto & dependency : dependencies) - { - dependencies_databases.push_back(dependency.database_name); - dependencies_tables.push_back(dependency.table_name); - } - - Array dependents_databases; - Array dependents_tables; - dependents_databases.reserve(dependents.size()); - dependents_tables.reserve(dependents.size()); - for (const auto & dependent : dependents) - { - dependents_databases.push_back(dependent.database_name); - dependents_tables.push_back(dependent.table_name); - } - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependencies_tables); - - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_databases); - if (columns_mask[src_index++]) - res_columns[res_index++]->insert(dependents_tables); - } - } - } - - UInt64 num_rows = res_columns.at(0)->size(); - return Chunk(std::move(res_columns), num_rows); -} - -Chunk DetachedTablesBlockSource::generate() -{ - if (done) - return {}; - - MutableColumns result_columns = getPort().getHeader().cloneEmptyColumns(); - - const auto access = context->getAccess(); - const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - - for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) - { - database_name = databases->getDataAt(database_idx).toString(); - database = DatabaseCatalog::instance().tryGetDatabase(database_name); - - if (!database) - continue; - - const bool need_to_check_access_for_tables - = need_to_check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); - - if (!detached_tables_it || !detached_tables_it->isValid()) - detached_tables_it = database->getDetachedTablesIterator(context, {}, false); - - for (; rows_count < max_block_size && detached_tables_it->isValid(); detached_tables_it->next()) - { - const auto detached_table_name = detached_tables_it->table(); - - if (!detached_tables.contains(detached_table_name)) - continue; - - if (need_to_check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, detached_table_name)) - continue; - - fillResultColumnsByDetachedTableIterator(result_columns); - ++rows_count; - } - } - - const UInt64 num_rows = result_columns.at(0)->size(); - done = true; - return Chunk(std::move(result_columns), num_rows); -} - -void DetachedTablesBlockSource::fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const -{ - size_t src_index = 0; - size_t res_index = 0; - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->database()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->table()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->uuid()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->metadataPath()); - - if (columns_mask[src_index++]) - result_columns[res_index++]->insert(detached_tables_it->isPermanently()); -} -} diff --git a/src/Storages/System/TablesBlockSource.h b/src/Storages/System/TablesBlockSource.h deleted file mode 100644 index a9f2226dca8..00000000000 --- a/src/Storages/System/TablesBlockSource.h +++ /dev/null @@ -1,92 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class TablesBlockSource : public ISource -{ -public: - TablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = tables_->size(); - tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - tables.insert(tables_->getDataAt(idx).toString()); - } - - String getName() const override { return "Tables"; } - -protected: - Chunk generate() override; - -private: - std::vector columns_mask; - UInt64 max_block_size; - ColumnPtr databases; - NameSet tables; - size_t database_idx = 0; - DatabaseTablesIteratorPtr tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; -}; - -class DetachedTablesBlockSource : public ISource -{ -public: - DetachedTablesBlockSource( - std::vector columns_mask_, - Block header, - UInt64 max_block_size_, - ColumnPtr databases_, - ColumnPtr detached_tables_, - ContextPtr context_) - : ISource(std::move(header)) - , columns_mask(std::move(columns_mask_)) - , max_block_size(max_block_size_) - , databases(std::move(databases_)) - , context(Context::createCopy(context_)) - { - size_t size = detached_tables_->size(); - detached_tables.reserve(size); - for (size_t idx = 0; idx < size; ++idx) - { - detached_tables.insert(detached_tables_->getDataAt(idx).toString()); - } - } - - String getName() const override { return "DetachedTables"; } - -protected: - Chunk generate() override; - -private: - const std::vector columns_mask; - const UInt64 max_block_size; - const ColumnPtr databases; - NameSet detached_tables; - DatabaseDetachedTablesSnapshotIteratorPtr detached_tables_it; - ContextPtr context; - bool done = false; - DatabasePtr database; - std::string database_name; - - void fillResultColumnsByDetachedTableIterator(MutableColumns & result_columns) const; -}; -} From 51d293beb9dbbe915c6bedb92613833db0240450 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 28 Jun 2024 15:29:28 +0000 Subject: [PATCH 023/178] general logic in detail --- .../System/StorageSystemDetachedTables.cpp | 68 +----- src/Storages/System/StorageSystemTables.cpp | 230 ++++++++++-------- src/Storages/System/StorageSystemTables.h | 9 + 3 files changed, 139 insertions(+), 168 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 9805fa2a8cf..560056b891a 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -25,69 +26,6 @@ namespace DB namespace { -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample{ - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - auto table_it = database->getDetachedTablesIterator(context, {}, false); - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->table()); - } - } - - Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - class DetachedTablesBlockSource : public ISource { public: @@ -280,8 +218,8 @@ void ReadFromSystemDetachedTables::applyFilters(ActionDAGNodes added_filter_node if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); + filtered_databases_column = detail::getFilteredDatabases(predicate, context); + filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, true); } void ReadFromSystemDetachedTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 783b899c978..dd7f1a6fdc8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -1,29 +1,30 @@ +#include + +#include #include -#include +#include #include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include -#include -#include #include #include #include #include #include -#include +#include +#include +#include +#include +#include +#include #include @@ -31,6 +32,117 @@ namespace DB { +namespace +{ + +/// Avoid heavy operation on tables if we only queried columns that we can get without table object. +/// Otherwise it will require table initialization for Lazy database. +bool needTable(const DatabasePtr & database, const Block & header) +{ + if (database->getEngineName() != "Lazy") + return true; + + static const std::set columns_without_table = {"database", "name", "uuid", "metadata_modification_time"}; + for (const auto & column : header.getColumnsWithTypeAndName()) + { + if (columns_without_table.find(column.name) == columns_without_table.end()) + return true; + } + return false; +} + +template +void fillColumns(T table_it, MutableColumnPtr & database_column, MutableColumnPtr & engine_column) +{ + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } +} + +} + +namespace detail +{ +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) +{ + MutableColumnPtr column = ColumnString::create(); + + const auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & database_name : databases | boost::adaptors::map_keys) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + column->insert(database_name); + } + + Block block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "database")}; + VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); + return block.getByPosition(0).column; +} + +ColumnPtr getFilteredTables( + const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, const bool is_detached) +{ + Block sample{ + ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), + ColumnWithTypeAndName(nullptr, std::make_shared(), "engine")}; + + MutableColumnPtr database_column = ColumnString::create(); + MutableColumnPtr engine_column; + + auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); + if (dag) + { + bool filter_by_engine = false; + for (const auto * input : dag->getInputs()) + if (input->result_name == "engine") + filter_by_engine = true; + + if (filter_by_engine) + engine_column = ColumnString::create(); + } + + for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) + { + const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); + DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); + if (!database) + continue; + + if (is_detached) + { + auto table_it = database->getDetachedTablesIterator(context, {}, false); + for (; table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->table()); + } + } + else + { + for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) + { + database_column->insert(table_it->name()); + if (engine_column) + engine_column->insert(table_it->table()->getName()); + } + } + } + + Block block{ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; + if (engine_column) + block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); + + if (dag) + VirtualColumnUtils::filterBlockWithDAG(dag, block, context); + + return block.getByPosition(0).column; +} + +} StorageSystemTables::StorageSystemTables(const StorageID & table_id_) : IStorage(table_id_) @@ -105,92 +217,6 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) setInMemoryMetadata(storage_metadata); } - -namespace -{ - -ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context) -{ - MutableColumnPtr column = ColumnString::create(); - - const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & database_name : databases | boost::adaptors::map_keys) - { - if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) - continue; /// We don't want to show the internal database for temporary tables in system.tables - - column->insert(database_name); - } - - Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithPredicate(predicate, block, context); - return block.getByPosition(0).column; -} - -ColumnPtr getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context) -{ - Block sample { - ColumnWithTypeAndName(nullptr, std::make_shared(), "name"), - ColumnWithTypeAndName(nullptr, std::make_shared(), "engine") - }; - - MutableColumnPtr database_column = ColumnString::create(); - MutableColumnPtr engine_column; - - auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &sample); - if (dag) - { - bool filter_by_engine = false; - for (const auto * input : dag->getInputs()) - if (input->result_name == "engine") - filter_by_engine = true; - - if (filter_by_engine) - engine_column = ColumnString::create(); - } - - for (size_t database_idx = 0; database_idx < filtered_databases_column->size(); ++database_idx) - { - const auto & database_name = filtered_databases_column->getDataAt(database_idx).toString(); - DatabasePtr database = DatabaseCatalog::instance().tryGetDatabase(database_name); - if (!database) - continue; - - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } - } - - Block block {ColumnWithTypeAndName(std::move(database_column), std::make_shared(), "name")}; - if (engine_column) - block.insert(ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine")); - - if (dag) - VirtualColumnUtils::filterBlockWithDAG(dag, block, context); - - return block.getByPosition(0).column; -} - -/// Avoid heavy operation on tables if we only queried columns that we can get without table object. -/// Otherwise it will require table initialization for Lazy database. -bool needTable(const DatabasePtr & database, const Block & header) -{ - if (database->getEngineName() != "Lazy") - return true; - - static const std::set columns_without_table = { "database", "name", "uuid", "metadata_modification_time" }; - for (const auto & column : header.getColumnsWithTypeAndName()) - { - if (columns_without_table.find(column.name) == columns_without_table.end()) - return true; - } - return false; -} - - class TablesBlockSource : public ISource { public: @@ -690,8 +716,6 @@ private: std::string database_name; }; -} - class ReadFromSystemTables : public SourceStepWithFilter { public: @@ -756,8 +780,8 @@ void ReadFromSystemTables::applyFilters(ActionDAGNodes added_filter_nodes) if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); - filtered_databases_column = getFilteredDatabases(predicate, context); - filtered_tables_column = getFilteredTables(predicate, filtered_databases_column, context); + filtered_databases_column = detail::getFilteredDatabases(predicate, context); + filtered_tables_column = detail::getFilteredTables(predicate, filtered_databases_column, context, false); } void ReadFromSystemTables::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 87cdf1b6a8e..d6e3996b8e3 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -8,6 +8,15 @@ namespace DB class Context; +namespace detail +{ + +ColumnPtr getFilteredDatabases(const ActionsDAG::Node * predicate, ContextPtr context); +ColumnPtr +getFilteredTables(const ActionsDAG::Node * predicate, const ColumnPtr & filtered_databases_column, ContextPtr context, bool is_detached); + +} + /** Implements the system table `tables`, which allows you to get information about all tables. */ From 7f528cc1a031616bd7965a50ff98c2f662fba581 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Jul 2024 06:02:41 +0000 Subject: [PATCH 024/178] fix client, fix tests --- programs/client/Client.cpp | 13 ++- .../certs/ca-cert.pem | 32 ++++++ .../certs/ca-key.pem | 52 +++++++++ .../certs/client-cert.pem | 30 +++++ .../certs/client-key.pem | 52 +++++++++ .../certs/client-req.pem | 27 +++++ .../certs/generate_certs.sh | 13 +++ .../certs/self-cert.pem | 32 ++++++ .../certs/self-key.pem | 52 +++++++++ .../configs/ssl_config.xml | 17 +++ .../test_accept_invalid_certificate/test.py | 93 +++++++++++++++ ...9_clickhouse_accept_invalid_certificate.sh | 107 ------------------ .../01889_clickhouse_client_config_format.sh | 33 +++--- .../02550_client_connections_credentials.sh | 15 +-- 14 files changed, 429 insertions(+), 139 deletions(-) create mode 100644 tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem create mode 100644 tests/integration/test_accept_invalid_certificate/certs/ca-key.pem create mode 100644 tests/integration/test_accept_invalid_certificate/certs/client-cert.pem create mode 100644 tests/integration/test_accept_invalid_certificate/certs/client-key.pem create mode 100644 tests/integration/test_accept_invalid_certificate/certs/client-req.pem create mode 100755 tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh create mode 100644 tests/integration/test_accept_invalid_certificate/certs/self-cert.pem create mode 100644 tests/integration/test_accept_invalid_certificate/certs/self-key.pem create mode 100644 tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml create mode 100644 tests/integration/test_accept_invalid_certificate/test.py delete mode 100644 tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f5e86b756fb..a7689bf7a79 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -178,9 +178,8 @@ void Client::parseConnectionsCredentials(Poco::Util::AbstractConfiguration & con history_file = home_path + "/" + history_file.substr(1); config.setString("history_file", history_file); } - if (config.has(prefix + "accept-invalid-certificate")) { - config.setString("accept-invalid-certificate", prefix + "accept-invalid-certificate"); - } + if (config.has(prefix + ".accept-invalid-certificate")) + config.setBool("accept-invalid-certificate", config.getBool(prefix + ".accept-invalid-certificate")); } if (!connection_name.empty() && !connection_found) @@ -268,6 +267,12 @@ void Client::initialize(Poco::Util::Application & self) else if (config().has("connection")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exist"); + if (config().has("accept-invalid-certificate")) + { + config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler"); + config().setString("openSSL.client.verificationMode", "none"); + } + /** getenv is thread-safe in Linux glibc and in all sane libc implementations. * But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer. * @@ -1105,7 +1110,7 @@ void Client::processOptions(const OptionsDescription & options_description, config().setBool("no-warnings", true); if (options.count("fake-drop")) config().setString("ignore_drop_queries_probability", "1"); - if (options.count("accept-invalid-certificate") || config().has("accept-invalid-certificate")) + if (options.count("accept-invalid-certificate")) { config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler"); config().setString("openSSL.client.verificationMode", "none"); diff --git a/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem new file mode 100644 index 00000000000..c4f88831126 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem @@ -0,0 +1,32 @@ +-----BEGIN CERTIFICATE----- +MIIFhTCCA22gAwIBAgIUZHeZ4ME7rL1RYGJZGPgOTHl7btMwDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwNzAx +MDUxMTI0WhcNMzQwNjI5MDUxMTI0WjBSMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMQsw +CQYDVQQDDAJjYTCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALf623IC +1r6Bzwa4dTufRPvFWQinT4uIBv7UvAk8tu8kmLSMOOC2COALNRCSu0s1SFcO6i5G +CgmaWdoTXuw+NYH9n5rU06N991NBfCoLbf0eSoie7bRdTCKkqWWKaw+dCvKL5vWj +LEZRmdh8ZN0lxa05eIsLxErvgWqE8hOOQ7jgRqYLAagp6XkMdB+YRkXFtbVcze83 +ZNxFqNjdgwYYybaqDZKGeOnRDRof7P/cyMkbGNUI7+JYA5eGmDgiOTGbaCZx5AUK +DKmkdmy5IJBWS0rYR8Ol9Sjs6NSYkmYiLFEWFNSvI88K99+Ab3DEi9JMEqMeNTJ5 +1sx48e2OfMG7ywoGj3D3zf7SUxAfwW+7Ls+v7c/kB9hvspA9iVsqjAKcSPAPnJXk +zR5r+zcPwjb/dvGGGB+D7PhvreEpj0n7PRFEB07jviA1E8zkwdk1L9FefYMG2oRl +8J7miWxoyqUOtA6AYhbN36+I0iJkjxIjXLLzB/dhOzerNTzRHcJtmgSyIpcPIV42 +IsJBttLOW0Mpkfw8z88xji+1VWHm+H8sOWCzbfjncScu/z5Xhb92C2T+AG+RpakB +LwyTsovf+xv1gGF/1gmmdKSoV/JZuHSczMmmDLJ9vXRnIpK0AgL16lfs6o0C2Pe2 +OHJv0oViokU5mW+Mge3U9kUrh6BVITk8UjUDAgMBAAGjUzBRMB0GA1UdDgQWBBRA +x0+tgav3Q2gRt34AtKZiGfFH7zAfBgNVHSMEGDAWgBRAx0+tgav3Q2gRt34AtKZi +GfFH7zAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4ICAQBcV7s7au6r +bEkHaW9RbGe/7iHesOcH2uyiALGeiCF+pfN9QbfkpJJbMRDhKr+sVqJH6lS+Iys5 +MpPaRHOInNs5ajiPQcIxtxAnw2HEsy6LUVvs6RUHrS7Bxlc406YQCY/sPyItAt8t +U/qhx5VNj6Mx1smyKZHFHZIZ0eYieriJkK6NB2E7Ji6EPGe7lkh3kEoVTfaSVCkP +p3d4U603kzCk8PGkYlhjWUCVMhDRwgEy6FXHmU156HivGpYurKcab7cMUfr50MyK +1Mx/NaHOYHhIIQ87t9wF2yQaMskEcyK9XOorp+HFtnTdci+6+J3u2Kmyckq81uy9 +uZreH8bUKgmntOhSqtl4tWGMWxX78c7rX2cGIbSJsZzzjmUdhvnpYeuhCCrUPzY3 +kIt9Q6q7DCKG+f+D4bHQ292emEfKYb9LgftyOX3pcAJ+d3c56Cs7c0RRVFAZyVnX +1O9ZvHrypmii5RARNuyVtttgDpuGBrRCa2kSNlXD9PJJitI2kfoWrCI/88Pw8H/y +bLO9/ZcCOsxBGmfrQDMUSC8rE8JLUSKV80otWwtHcxcnfpsobs64/Rz6aOLYEiS4 +/0M6xyUYdOsEGUh1uyX5qxto8iOWop4S9DLy1QVx+N8/fC6gjhMb4Vg2oTBnHlGr +JcxdYP8jVe6tUgvRKAyNlccPIq20udfGTw== +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem b/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem new file mode 100644 index 00000000000..66d548d0cf5 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQC3+ttyAta+gc8G +uHU7n0T7xVkIp0+LiAb+1LwJPLbvJJi0jDjgtgjgCzUQkrtLNUhXDuouRgoJmlna +E17sPjWB/Z+a1NOjffdTQXwqC239HkqInu20XUwipKllimsPnQryi+b1oyxGUZnY +fGTdJcWtOXiLC8RK74FqhPITjkO44EamCwGoKel5DHQfmEZFxbW1XM3vN2TcRajY +3YMGGMm2qg2Shnjp0Q0aH+z/3MjJGxjVCO/iWAOXhpg4Ijkxm2gmceQFCgyppHZs +uSCQVktK2EfDpfUo7OjUmJJmIixRFhTUryPPCvffgG9wxIvSTBKjHjUyedbMePHt +jnzBu8sKBo9w983+0lMQH8Fvuy7Pr+3P5AfYb7KQPYlbKowCnEjwD5yV5M0ea/s3 +D8I2/3bxhhgfg+z4b63hKY9J+z0RRAdO474gNRPM5MHZNS/RXn2DBtqEZfCe5ols +aMqlDrQOgGIWzd+viNIiZI8SI1yy8wf3YTs3qzU80R3CbZoEsiKXDyFeNiLCQbbS +zltDKZH8PM/PMY4vtVVh5vh/LDlgs23453EnLv8+V4W/dgtk/gBvkaWpAS8Mk7KL +3/sb9YBhf9YJpnSkqFfyWbh0nMzJpgyyfb10ZyKStAIC9epX7OqNAtj3tjhyb9KF +YqJFOZlvjIHt1PZFK4egVSE5PFI1AwIDAQABAoICACPKkBVqevddznYqvVfHgMaQ +GmWGEwH8iQvbO+d6IcZ9Mp3xJ4GDuE9Jrt2WBgy4BWY2VxWAd6RtnCxCF3XTFlq/ +CeU3oSKNt2dBdpPBXHjhAzP34xtzCQQ2T2QbHP/RF9eeouRHha36YvkMHh0to3qz +EwHN3LaNSHp9+ZAXXZJiQyw7OOvLl4l+hDq/R3B8aYCqWL/AYYU+of3qffpaKSe5 +hmFqeqxWCHv8GQGfpIoHFPogvnYklDJOs188PS8q4gSUpHzceyEnhwhYrX6W4xfg +4nkr2HlJOXReaCCpZkCyEIRdvBQjuJwI4uUnv2G+feOw1T2IGWL25m3GCfwweeuZ +AWqqbj5Xnl1+VgvaNm68aHBVlfZ14rIJ/Hu1yJV/E+5m8+3xewAor0jshWceC3OS +lhrZp+EyrwxsKj17s2IBhk4EIOLjfosoDqdw3E2DrtqKrnI63MLUTaEqHWyXnF82 +Mn880DR/E3Byj2kKtdIVesAdO0ARtIRqedPunpcvhp1eMZ2wrLJXAHMDBJlUZF64 +ZAyO3ll79PhBn/QShGu+KQ682Ns1T3j3J14n2zqBKsU1GzSLBOZO8QXfFELsv6Xj +NGYPg2SbkRwbYHEPr6G9ehsxKaTpMAtPesdz1devKDI4nSpd2qE9ZfREH+CW0MkK +bpiL20d4blIG1DP84qodAoIBAQDEBXbn0IEeyL3j50/AUJY/K4seeDm1ltJmhoNU +HC0dwt2FOHrD1J/Cx6S19zuEVt1qMwJXELxOVzixykKPy73ETW+Qfcpo2J51THrl +fSEUpoheD37Cwjv9vzb6Qb4ZXDr4acLIB06mAUKfo0Pez2x6IybWqS/dZoD1bfh5 +wiSCITXQB4n6mI7chfDDyTqgf3C5U5b2UABLDdIKQZHY5kX55Vfr0Lz+Gt2t6uUy +hUiN4C+Qrx1LoHY4x7g1SG9lKASG5zv5dksc+a6uYsS4N8TM3ZQz7BnZrCQPNiWz +K70p5pMjdVTHmQqThoww9y+rRiHMU3qUSZ0tX4nrY99/MiFNAoIBAQDwRi0GzkeA +ECWfr2R+vh5lnxXBeAvM46v+8CzwJHNxtGAhB1ojod7GxOxHiMnWDXubRiNVlhx0 +OHeCq11hSQh2+a7s8xwxYbC3paYf1ZQq81YmTGd8v+/QX9b21Kq9dpx+G2C3eikZ +rHMuSifrSEwZMxdw/gKvQ+x66PTNeK8fIgAtZsE96VX3u7sCCNhbBElWWg5OwwMf +0RH5Pi/O8UXxkZs2SUKFaXUFMxjYn9r9636QI8LaLsCvcbPjrhshPtHWHqgrZzia +8gD00ty3mcydRU//xNfHruew2iRZUVVB8E0GQ80eQCsHrI0YkP876at55DKbyZLi +pGisswDdyoePAoIBAQCfOFUSDMGksKfVY8Da1FqGKydQc5Vkk4KJdr9b5Zsb7nmi +yXKmeKHgQ+vdRlmYsDofFhnTgKuD8sMBXuncBUE6Hy/8rMBpk3J6ksnMeGwTawWf +aXSsW7sA7H2KnzX7QXhA6R8m/RB8NM/Z/XKWm+Nf/geBG2Hes3ESaeYHeJPRA4rD +VCxWWe7I3x7m2R/+2CVG08fugKMMU8iFPidqh/jU994HlMC+ojHoH8FV3P9HBo98 +NVFVaZw9mLRey1x4TR4IOfD0O4rdmU2zJlTl0Z7QTTchSajhrzqHBJNwyplXXIut +pJNdoUj+9a4P4+hbdZbkcfZCVt0NyumrQq96kMyNAoIBAQC8Jcemd6tHnd1IYPip +wshcvSk1GYLsCzAo7vvRhEmeNSe9VVuBYYToU1XOYk/ivSYYneCXaXeYjosPGI+D +2cO2HqiaxXhDNwNBm4YtkXp385p3XQcAifAJVLMpD7kvqYy7t0aSBiffAcoj6k++ +Dxv/86EcituhUOgCCx9/Mky3bVGEO92AVJJL7oZu/mwlQYMICnb/t/dfV8DiUo7v +VsF1e5Z1+QINJFpNGwkOwIakpV6zEj3Hyip6d+fslu6Jpv570//OFQMfWffjvqa1 +Ysr2MEuHtiVeKRkmxHsIMgc4HirCiB872/WWCclm/kvPJ5huV4JFuZ+6uBF7gqRg +CPMDAoIBAGs0KCL3a/eXIPWhFcOgtpcNTb3o4gK/s38+GLIVbdP7MUNEkMeWq+Pf +PBUca2p1M8CkbpRi0OAvg1gewAR/+bCdyzoipl1+3v3goYC3J5TAs5NT/siBcjrm +QcTKEHmTeVu82Kpi1JVtcfS2TWuSmT6vSB2A2H3R4GEPQQXnXeS/FnRA0I85MFzq +zxMkar5VY7Eg0Z3/bN5dNqAYSzqjgMMhRMPN1W54gjL2ev65Mzt1jYf6xz7gpJOh +6Fw2EOrM23c4OckZx2dZ7Je6ns2U82D81ffhXRwWfQxoKXvMVMccBJmHTbkVEZlE +VAq9BWwNistc+YfRka1OOeq37MgpXq8= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem new file mode 100644 index 00000000000..2dd497409c3 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem @@ -0,0 +1,30 @@ +-----BEGIN CERTIFICATE----- +MIIFLzCCAxcCFA6o2DM1UGQ1NNCQzp0n4hb3iSd9MA0GCSqGSIb3DQEBCwUAMFIx +CzAJBgNVBAYTAlJVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRl +cm5ldCBXaWRnaXRzIFB0eSBMdGQxCzAJBgNVBAMMAmNhMB4XDTI0MDcwMTA1MTEy +NloXDTM0MDYyOTA1MTEyNlowVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUt +U3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UE +AwwGY2xpZW50MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAl9i6heiD +2inNs0lKt7XjK0QyXKyIg63uVO39ZpLY5sj1noZnbFmRo5MYC4MMIrlD3wGnqUo/ +mdev2eg8qYry0CmzhLscVEha7LN+7wmdftYSGkXOl9E08KIv9Uw2bHap4KEIRvJ6 +298bdsuv1l4ljWo3VDkPCczXIaUi94/o1uNu0sbPRluBP8kjFLhwV8Ph/7mxsofu +V5tDYNOZ8VozA7aHAnJqXBCFZuW6igIH3XaROY9LOe2FD+qxtpqJHp7E+ZNsXXH5 +SXMQzE2/X7eKHA7zvdG4z5GLgmz3GEvq3ZBJUGUe6DCnrxqBXUeLTqyummohGE57 +LuqzrH9etFj/91NaZW9uwN42pxGJSkeZaJ5BDnzO3VYxqvvDnFs/2TFbJiZ065vI +NpA0PnuAkGjztaNAm6qXNTXsWo8bdi8TDgIwKxoo5KIlWsxnxKB82tfBovWd+JLa +xLlWNKbKgNb6xc9OmNCyhYkRXhqIsBrpLy6n2yppyYEiCBzPrqwJ2p3Y88yj3dsc +xiiu0ngFSroE2pLwCgi1rQY1x9VO4XQrrq6vBvPhi530+cdXdVTnnyBhaOMX4bWX +OOjI97xJmR/YbOvpgKw9v/na82t2fgKoqIs124iZ21QOiffOIfwwg2AKj8EaWJ9/ +Era8sZMsq/g56PWYwT1J85cCMvbNwGOXjusCAwEAATANBgkqhkiG9w0BAQsFAAOC +AgEAAPSps9k8HEZrfBbzH5SHASNTUc7F81h2EYhaEuQrXoan39u3syCKYMSJDt0P +AVCYbIvXmlHeaBba9YXtZVZJpCM53b+x01297x1bXzRklYxnpuD2IauuQgiaJBXV +VlCjEy7GHBAmRQwV7llkoxx+qHsA0b+oHKvTRc9FJzk/Fx4Eb496gNMvUk47f+15 +WBmWoQolqwbH1Ivhca7BpTvTSeNGkJ14cbmuhwryLstHJCPep+vp2TYS9bHFZIPY +g3APnWv1qShuA7Oj68GLpRVJmImgyPMK0i8LDGcLhrDiG+HYEr89MiRWDvBHun7O +Sp5y8zDnDZSrN+EtRp8DypHeB7R0Ukmn76NP6NsigWCPMlLaHNVsvVAwWwFNae9u +OxsoW07tJo3lZjDx+tFrUb7IRyn/9PLDIwEs4SvDP+O97o0JkCszwfRhkDnjY7Qj +XCgXoCBalIadgnGshbEjFLH0ZSwAoCIC/35X0Zo7cd1+Fz4Xq1tgnV5ec6ci/EHi +qsOmGfTqZz5HJlmfyxUOYicizV6SjNZSFWY07ebYRewx6ymdDFhqrbnJIEUGJYlU +5qDY1nemoqh9QRIhm1+73hAKHtmcaa1u72gj1QBtRD5eG3ToM3GHgdbUUPvFpenx +UPfzw/wbzITSqPoYr1/XIGq9e1muTqHbFNlGVByIW0ejeZQ= +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-key.pem b/tests/integration/test_accept_invalid_certificate/certs/client-key.pem new file mode 100644 index 00000000000..09c04c1e815 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQCX2LqF6IPaKc2z +SUq3teMrRDJcrIiDre5U7f1mktjmyPWehmdsWZGjkxgLgwwiuUPfAaepSj+Z16/Z +6DypivLQKbOEuxxUSFrss37vCZ1+1hIaRc6X0TTwoi/1TDZsdqngoQhG8nrb3xt2 +y6/WXiWNajdUOQ8JzNchpSL3j+jW427Sxs9GW4E/ySMUuHBXw+H/ubGyh+5Xm0Ng +05nxWjMDtocCcmpcEIVm5bqKAgfddpE5j0s57YUP6rG2mokensT5k2xdcflJcxDM +Tb9ft4ocDvO90bjPkYuCbPcYS+rdkElQZR7oMKevGoFdR4tOrK6aaiEYTnsu6rOs +f160WP/3U1plb27A3janEYlKR5lonkEOfM7dVjGq+8OcWz/ZMVsmJnTrm8g2kDQ+ +e4CQaPO1o0Cbqpc1Nexajxt2LxMOAjArGijkoiVazGfEoHza18Gi9Z34ktrEuVY0 +psqA1vrFz06Y0LKFiRFeGoiwGukvLqfbKmnJgSIIHM+urAnandjzzKPd2xzGKK7S +eAVKugTakvAKCLWtBjXH1U7hdCuurq8G8+GLnfT5x1d1VOefIGFo4xfhtZc46Mj3 +vEmZH9hs6+mArD2/+drza3Z+AqioizXbiJnbVA6J984h/DCDYAqPwRpYn38Stryx +kyyr+Dno9ZjBPUnzlwIy9s3AY5eO6wIDAQABAoICAAl8R6ICPpCcEXrUb5qn2Ujy +IJQdT+D0yJqGTD6QCDvsLDstGcWMcx1him5785IxJ9YpZBbnidO0euXJo8R3Ql8S +aGHK4ZtfFyeKAYNoJ0mSIx03LdiRAEOJ7QDf2LI017yJS9IkeW5ql0fTas0za2Jm +nCBZZrNNzvv7a/BEn8NHkfMQV5Fgy9ESvP3qcHhsNHly/aeD5ZEmbZcmWeb82orS +nxEnR0hUSnI0JpibsycT90DimAOh3LDXfd3E0Iz9VsA8urMpqtpDYzZ1tShed7TP +tyiaxS1+PUNIkEc9xRv2i2FQyhVmpmZdoCXfNi84nUtWhJYE6HkrmgRwnnLQp5rr +4stSQLo9X/5ML/bo2cj1eAiH+P6bAWaZhkMYPGkhBYwwgYqZF0Jn2Ay07BPF5gqq +rDY0mjF+Nm/NOqHfQjWHSj7KYYHm6qU1s5cAnDWLfznXGWZHpoI4UqAZtpAwGWEh +f/nS2LyepFW8RE6mbRDe+PkOM2455QdM8MuQc8drGInJ57A5e7y1LxOvFaUP+QtG +fqXok95XciMfttcR8nfYkUyC5QX+JmDZRf1U/DUxTJtbKXRKVG8WaFagkViiCniw +Q4RbrS4Utu6h+8HXdI9mK2pIHlJPp202b9gPry1j3kya8/jx5cTFWCcPj4IlgY51 +ZC1+vXJUezwq60gWJrRtAoIBAQDLsR9M+ZdBC5sl5fiVGsznAvfEVv/Cf7pmeXqt +q4BZT7kB8zfdfRRMzmZiQIX0zTbBn7Q7kWULcl8HgPbMEGZorc24CFuCq8k0n2ji +0aJsTJlfr7kUfTZ5c3HN1fGhDrnGoCL1eElt/Es4hZTIDnbn3i+RoCqwhCqhC0Lo +L7f6iHrVIcKyoaz+Cye9QKA89PVHMNM0f41zFJTVpSzGfWefG4vNoeU6nxqwkWVJ +Guhkt1EsJUScHueAa7hUYlELDy99tlqOPLdG7/GG6LmE6pM7R1E6DishnL1VgOll +KL6fouoz8nTve59eCDQECNzXN8lxPvx6TdufabGJTSZ3iNjPAoIBAQC+10DFlO3Y +XLHfX9LWK4rdkghx9AoeCFIVZO2G29H4zyDQn7hAMISWeVCqdAKhPQMJmAXyhQcc +PLEUdKVwifbPNO/IxsWL8SOfbI28XoZLYkQLual6JPa10jemFsinVRTPLU7RA1/T +7t/mWvyaAHmODWUPKSs1ltzh5dNwvnSNou6h72TytwlMIk5h8Z8ZtMIY2v5Ca2Mx +I7Wi8/5RebpszBbuLYy0GBqK5oqQov1YH0WK3ae4leaPWlZ49nyNYpx5Bvngm+d8 +4F+HvcxdpckRPfenl9nBW+0UHVXZORepQ+g0CNJxAQTomXy5VeEnrpiylrec4mE7 +YD15GKDxeHclAoIBAQC6efcr4zwDwnpRIc6HwrdfriniR7tx9rlRY1lacS3ZldDL +5jx4uy/AeRzmSifYy56beHfqKNuWtSgmmHxYqZL3tlYlaxFL+bUERgQh0sWd3Cp1 +nUwVY+RhH71KMdk6F18gZi5bB7pBxL2gizqARC27bH+kFDvAyd60/LhOFJGb/SJm +yLeUDctv8Ap7ihSoeaGrgRl6cGOwvWDcjpEFTn10/iTpJpPTLSGkEBfn64hh3Cml +/qBnRSuvN+7S4gdL1qUls7IHkbJAcR5u4dQlyRLtLAdC3QHDjdkP1CrX/4s0XlOH +JyrCrCxkS6JDJ1H2fg4fbQh7d5few0ZTapwSxuelAoIBAQCloou8MTuR6zoShsem +TRp0npen18ikMlKT8QpbXxTJ33Q6vjWbuMCLHXYOyyuxKBQn0oNu71ixgFetSmSA +3a2SP/7SAvtYL43cg8ossNncOciXaKgYtFZQjsjv5Zx351dtTYS30jbs6cn41AWj +4D+gNC3/T5sKIwtJIUE3/IWKnNyDPhUeuBBgvPHM/no5RHKmQgQmP7BS6QxwB1c2 +/fD5FtXcGTqi4LzAy8F+MYizYEjqcuFsSuZWi3C8AisuR2VJge2Eravn7ZHcbbYf +nZF63o7BLgG4WgHV0HQ/WdcYeTDjYLjNjbXl77mZ3ljSKS6s6jUo/C6p6yjPnrmp +ewXlAoIBAHX9aLtoj06zU5dtrMVjhYip6Z5LEtHsXCvOmnAl91uUXKwtMze6Y7xc +4ZggVWBR9lFyq9bAAuIXkRuE1jbvEwl5sLZY5Hf6GrIUR2uD7ZYFbogBTgmWCX9S +vYRpofEwquX0AUt+a4nO09wPBX1BJMViAEZSAWGfCfzVz8rXYIgcmu471wrpQHv4 +/whNA90JX0v6tuOASgR2dGorU//0ejGcm0bcYSuivKPxBfmVNeZNgIJXLM0ZHql1 +qwLnWXS0e0kADWhp/GAWVo0xzefgD3mDmyG2Qbz8MLaDkwPU/L073ZZNT1AHFZZE +xfVijyydtPqa/JDgqcUFBxLtiKGTbbI= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-req.pem b/tests/integration/test_accept_invalid_certificate/certs/client-req.pem new file mode 100644 index 00000000000..507db7bae6c --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-req.pem @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE REQUEST----- +MIIEmzCCAoMCAQAwVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx +ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGY2xp +ZW50MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAl9i6heiD2inNs0lK +t7XjK0QyXKyIg63uVO39ZpLY5sj1noZnbFmRo5MYC4MMIrlD3wGnqUo/mdev2eg8 +qYry0CmzhLscVEha7LN+7wmdftYSGkXOl9E08KIv9Uw2bHap4KEIRvJ6298bdsuv +1l4ljWo3VDkPCczXIaUi94/o1uNu0sbPRluBP8kjFLhwV8Ph/7mxsofuV5tDYNOZ +8VozA7aHAnJqXBCFZuW6igIH3XaROY9LOe2FD+qxtpqJHp7E+ZNsXXH5SXMQzE2/ +X7eKHA7zvdG4z5GLgmz3GEvq3ZBJUGUe6DCnrxqBXUeLTqyummohGE57LuqzrH9e +tFj/91NaZW9uwN42pxGJSkeZaJ5BDnzO3VYxqvvDnFs/2TFbJiZ065vINpA0PnuA +kGjztaNAm6qXNTXsWo8bdi8TDgIwKxoo5KIlWsxnxKB82tfBovWd+JLaxLlWNKbK +gNb6xc9OmNCyhYkRXhqIsBrpLy6n2yppyYEiCBzPrqwJ2p3Y88yj3dscxiiu0ngF +SroE2pLwCgi1rQY1x9VO4XQrrq6vBvPhi530+cdXdVTnnyBhaOMX4bWXOOjI97xJ +mR/YbOvpgKw9v/na82t2fgKoqIs124iZ21QOiffOIfwwg2AKj8EaWJ9/Era8sZMs +q/g56PWYwT1J85cCMvbNwGOXjusCAwEAAaAAMA0GCSqGSIb3DQEBCwUAA4ICAQBr +ZWJGghyeCUESKu9v9dgQYD4Bq5YviYY7BAjdQsn5/Ti+tqkAKHEn/L+264QJMrTU +muTKg09o7n7r+g4sHmZhT2QEtwGFwaEGvz2zRd4po29WlGQAN5a8yzXIb8b9n+C4 +Z5APeAzjBV9ooqbJCURlgeDP247XTcU8yIvvX0YpLmXYRQ5Na8TTfdBdlD3Y23mD +aPe+2YGkK1Go4iNMoyTEoJ0cj+89WMA9o+BiGcptALYMzKBM26f/vz9hb0J49RDV +MkccL8TKNDJQrLFCIcwpLRw08uZZdqJ/cy01hrtkMkp+SSBQO9qSmalDrTLgs4Pp +3ml88OIsQHMkdBUjUAw5FyAJoik4iXab/sLoMmMJ/lqH0in3dT4KeEh8Xe2iAfNU +97/6lAuZOrn2EPvqBJI/gfjwPj9wTgjlvLpmcjhBvfqkuM/dd7zTmwRoM1klRMC2 +07x2YNoG7cGXjAYk1h5HDYVJqdphgQLTS1nRqmiQ8QnPryqlZcI7T/GgpOOulHce +lgMp6FFlGaQMysCbdYJQk4JMQXjwlj5ms/6rZSkgwmrv6WhCY7ImDj2E/7oE5XJd +BqXHFwOQdCSgQOVH+QgmxceYolI73cMNF3Frq8/IztAOtMyBF0bpRpI1U6a3ALf4 +cK2Dkd/TLg8ZwsVBjPaRrtXlntO6tB+tn3suukCKcg== +-----END CERTIFICATE REQUEST----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh b/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh new file mode 100755 index 00000000000..610a1281f8b --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +# 1. Generate CA's private key and self-signed certificate +openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout ca-key.pem -out ca-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" + +# 2. Generate self-signed certificate and private key for using as wrong server certificate (because it's not signed by CA) +openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout self-key.pem -out self-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" + +# 3. Generate client's private key and certificate signing request (CSR) +openssl req -newkey rsa:4096 -nodes -batch -keyout client-key.pem -out client-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client" + +# 4. Use CA's private key to sign client's CSR and get back the signed certificate +openssl x509 -req -days 3650 -in client-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client-cert.pem diff --git a/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem new file mode 100644 index 00000000000..7b3832a3b6b --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem @@ -0,0 +1,32 @@ +-----BEGIN CERTIFICATE----- +MIIFjTCCA3WgAwIBAgIUFpPVJytX13EWXwaLqg/rOTdu4UgwDQYJKoZIhvcNAQEL +BQAwVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGc2VydmVyMB4XDTI0 +MDcwMTA1MTEyNVoXDTM0MDYyOTA1MTEyNVowVjELMAkGA1UEBhMCUlUxEzARBgNV +BAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0 +ZDEPMA0GA1UEAwwGc2VydmVyMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKC +AgEAsQO7f1Ar0ThZjh00KeuckyJDf/oPQ+67EX2Y82W4/dl8KmHksmIKp2+CJFwg +Y7/c1ti+DjxVXr4tZynzUeDYqujTrAidUMz0zrvjXNeeyoe2hgc2X6YFtVhyEtMQ +6q8VMaqomjuQDIyxUUdt2xHZWyxHPntDrzsmCCDH5Md27CZ4Z5jsSOdgnlrCkjd9 +NtuIzxTlrExe3whIlTL0DURTvAVV/tPoVpVSvHTWCdziaCWm1hG5T6ZCN2W0pBgV +k+UNNtXr1ksr/Ma0Z84ShKjpN+soFOIZ3GDDGY8Xr/74jZo2c4025TpeIPcPBE0Z +b2K24l0UNnQbgm3ryI7NRrZ7psNnhvnWZru4w5Ykr8RKBf3n1AZOt8+dXHNvJQeU +/CUA6pKpc+sIxzAtWtS2ABqdOTJTXj2GyYNsz38dI2uzecUzT7oEtHWq9sr376EO +xMv1wXsUBli8INhPricfojH2/ACnRPysFglAKTx0IpcwqYoV5pGkk0iK1DNkQMv4 +Kf7cD2hLWv9+Fgbrjfu/gNCL0bSJBjaTHngACePsLEQkmGomscDySYXa+PHeB8Ln +QCWsE6MkeDa5g3gUOPV/vC6qv0ZBvQj1p3sOeyIQATDBYX+KheCO+N0ZCCdlT5Sx +thKNDusultu6/lyjGum0c/5oGgoDKTIkibgljekKupBRAykCAwEAAaNTMFEwHQYD +VR0OBBYEFEnPwBjLDPzta3nBrUEH+Gz1nBJRMB8GA1UdIwQYMBaAFEnPwBjLDPzt +a3nBrUEH+Gz1nBJRMA8GA1UdEwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggIB +AIW829ODEBvBp+vUwt3hbsJdBx9DzwcBBJ+Mew95YVveiosU+3p1HaWCkc4srgOi +hDNbpvfbqBIKVAZzesJbuTEeQ+Aj4M1lcFed9tvidVnmqDgHJApOI61GihT9MDjC +dH6vhcsorE8E5F58FcMJ0QDynVOuCA/gHkS3ucMxBsNKGfYNe0rOGMw23CzHdnz7 +UonOKhrhIQawI3ELLQxo2LWZEmMsluznE4MB+RGdn3k/UzOY9x4bauv4hrHjYvEm +x/zp2iWhZXvvTZ9u9xaHXEzbSU3WFCcXE/l9SDPM7ip9fabUrvaTJkmM190bD9sk +VcEFOAFBVxfx2fvmBIYqvSkVbcWdAT6x8k+MVNfq99/1aIdh+DTB3XUB7FkE/fnU +tXr/1sz0guwfNI/cHvhHisjmbgMhZWeeVV1YZHOJCsK7eq9rGq5Uul8nYrTdYKlN +u0OcrBXYBB6ny7ljCTahyPPmFZh/4Mgr0xROn2HyhTCpDHjnD1M5jT783fvWqu06 +cpOjRIPYNtA/UQAWUZWrTuJTD6Kju9tDI8i1NtMWIXd7w0LDXsXeHpRoDrVvOIPT +P0ivUpt4dFHnwR8Dos4NI/j9mkdGCblUxDsd/7or8gjedOoWjWfGCNlYa72aq0gf +KjKGPISC3BGTNtwRTe+E7k38r+Y6YO/Tnu9RvBdDM3vF +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/self-key.pem b/tests/integration/test_accept_invalid_certificate/certs/self-key.pem new file mode 100644 index 00000000000..10b299a8c0c --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/self-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQCxA7t/UCvROFmO +HTQp65yTIkN/+g9D7rsRfZjzZbj92XwqYeSyYgqnb4IkXCBjv9zW2L4OPFVevi1n +KfNR4Niq6NOsCJ1QzPTOu+Nc157Kh7aGBzZfpgW1WHIS0xDqrxUxqqiaO5AMjLFR +R23bEdlbLEc+e0OvOyYIIMfkx3bsJnhnmOxI52CeWsKSN30224jPFOWsTF7fCEiV +MvQNRFO8BVX+0+hWlVK8dNYJ3OJoJabWEblPpkI3ZbSkGBWT5Q021evWSyv8xrRn +zhKEqOk36ygU4hncYMMZjxev/viNmjZzjTblOl4g9w8ETRlvYrbiXRQ2dBuCbevI +js1Gtnumw2eG+dZmu7jDliSvxEoF/efUBk63z51cc28lB5T8JQDqkqlz6wjHMC1a +1LYAGp05MlNePYbJg2zPfx0ja7N5xTNPugS0dar2yvfvoQ7Ey/XBexQGWLwg2E+u +Jx+iMfb8AKdE/KwWCUApPHQilzCpihXmkaSTSIrUM2RAy/gp/twPaEta/34WBuuN ++7+A0IvRtIkGNpMeeAAJ4+wsRCSYaiaxwPJJhdr48d4HwudAJawToyR4NrmDeBQ4 +9X+8Lqq/RkG9CPWnew57IhABMMFhf4qF4I743RkIJ2VPlLG2Eo0O6y6W27r+XKMa +6bRz/mgaCgMpMiSJuCWN6Qq6kFEDKQIDAQABAoICAA4BaZhifipeTFeo395hPKY3 +k8z7EAd5SrvQ4C/XNefVNAnLwmM1dqUAzEnsMB6JqQN8FVgzAIjagI/PtedzOYbB +nTMuC08VUwP4bGjay+mNC1gCF2nsojmoTH5IzOcvVdH0MBrj6OXVS+z5EIFrdsh2 +nGcegaB1DEagXAXcnQAvmwniA5qmaUfzjmmF58EAuwuyGj9OW5icoyw1jTHE3OCH +hjeqCeav3I1PVpAOi1YKL0UeFIzExCsglPEBfCjoz+mCy4IiYydA4Y1FChoM80bw +CDUpc+UH6D02y9JVA317XMuPuhmGS2RNp+vYiHv+Uide40vbq+w0nkSOvUHXC7lo +UXf26sZWHcQP8TxewjZFcajApk03lCAb/u4skt7OQxbDOcvrcdbCsSpZPS16s3CA +OWA9fS7WJDmFx/dhzm4xNaXpz9z5MMkUfnFqgceG3H1leamL66J+eOzlquQ8LdsJ +/wTvCi1SsR3JmqT+TpqlXTJci7A+I/b0Yt6a0s2HunQ5LA1Xomffhn/00IxUeqyi +4MKh7wnoCaRlwZVORgQfKnosSkbkixDabwr9BMnMryE+NdQI3JKCvchdB0idyk1z +igTgZYpaxZg9hwTlOITGYSsprN7tiwjNQG+Kivatn1NxnxhD8lpDoUqluR7mZs1e +CwOLrB+irCULr7tcRYtxAoIBAQDFV7E1Oz8j8NkvkLztIFJ0ej7zEyNp3HUJb4bd +SRvF2AL6sqSw0xVLK8HbrlfPo1Nex8nZUfY3v2DmPS4+S3mrtk+8XoRkvNn0VdL4 +FqLfygYsURI8xVp4zTsAmR+7grPJbb/u3XG7CBY3T4XGHSbMmYAu7+W3A2t8Q+0U +fptju7Gc9gEACcSfap45CzEnFbabxNq3FJJ9V92N/xVQcOzNLxPxgn6tVhPeQnXV +0WHJQdxB/8DivgOObiTCB7on9tNbSHisTYtGwn6Kz0961ewYU7Xj4epj7A35nWhI +E10FDAf3T7cQa9ubSFZwOiSe0voAbiWbN3/M+zgkZDp2+wlnAoIBAQDloTtzgQaA +//VPNhB2d7iOgAl5cUSWqY0HLJobw1ETsdbkGRTgZrFXe4XlXwHB4S+zVEMmyR3p +HY0sYoAdpmd3Ggo4efJEx9IxkD6UZ71we02szvt4MdV5mwjHOAQfEJffktVkeqEJ +gdekSmaHTeryBf3ESpefq+OH1qaj/IjivySLI4c8fPcxNg0nMmDsdMzRaDOHDDa2 +nUpRBeMFkwiiPJpALPPp/4be0d9mrahPd4zBoBieZwaP6Zhwksi0NaM4WKEfx4Fd +TzH0K7UfLDMJ13wuTsNPzjPlDA+R4E/O3qVZxtHzuqc7fCksE7inYZghIRSjQHmv +mmLPaFdaIWTvAoIBAQC50BQfVyXgpYsaamI5tFFvMxvKrJ6PsvDQbZ7DBW+NHhPQ +Eh1YEajfGthyDPrJm7OIbv30DmSToTOQFtNP/KkXsZXqe+SVJezjT1xSFJ5PsV2W +pM9TQVAqprODoKpsWGkIpLowI3pO7RJeChafsBs0UuSLTX8yj5nJa/vzbDLL+ZMH +g+r1Kj1lDXLlq1AgySUkDnsSH+C+/Ni0NxbOo0WU/Vr98C1u8LnMcS+aoIg+hDDs +Q/BOjzOyDKu3tqblslbYgKh/NJSLj96uvPzNYFfpoAiIBdQ5Oa7+9eOW2gSgF+/4 +QarRX31r/PTGvzh0A5s6hHwMZnmDfJgkS+tw1HbJAoIBAQDfauQBGYROdrsieqPB +3/H9E64kLoRhYk4IsOiwInT8CIfgHlcljkT1rQU1ih6XmJVkkZNUZVFowovcp2mn +b4QzaW6QTHXfXduwxXtdwSGcExwcCra41yOw0ZrU2jeIUgiySpa80GNmcKJzAsCG +3NS+7f1dwoWLoR4byEkSPF6i3v6bMdIelCeeBcz+WhHbfjjZddzMhKGnQ5pPAr9D +aS95SiV6U0JuZWO/38rQiB2+U77rlOzfEbZcxxa+UNJeI+HOrMz3spC3TJp5yxzK +W6K87J+yHogBWcs8C2QKtohv6khO9RbJYM3dHoygif+bO2Vctmbx/4VlyXxh+E8l +LmCZAoIBAFR+dd7cW0pDkOsPsGZbWwDPNRwo2lxmqf2B5w9zoGNFoc4qV5k9tJfu +kPcwol6h2P4s2olYTgYC6hjPmE17U1TyP2DFeOw5IdxzNqy7dZndmaTs4qFLnDv3 +HIbkX42p7v++EU9eKAhYKJzJpuianMZ+v5azV1wNS5TDuN2TFULyIi+1UEw0ONuJ +rA97idb2PrO2sT/eBj02uiOcPhZH6CHSMxDp7QG+S1AVZBgutTQQdeAsb6pE/iW0 +u3IzGNxTYjTmF+u7iNdOPiaTFpdnltDviADESMJGH7PIk7r7PXMI9ID+LbBFGJ7Q +duW6PGuZ9JMBbxbNNvYayVgPeAeUsUI= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml b/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml new file mode 100644 index 00000000000..a1dd1a24693 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml @@ -0,0 +1,17 @@ + + + 9440 + + + + + /etc/clickhouse-server/config.d/self-cert.pem + /etc/clickhouse-server/config.d/self-key.pem + /etc/clickhouse-server/config.d/ca-cert.pem + none + + + diff --git a/tests/integration/test_accept_invalid_certificate/test.py b/tests/integration/test_accept_invalid_certificate/test.py new file mode 100644 index 00000000000..13431f39854 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/test.py @@ -0,0 +1,93 @@ +import pytest +from helpers.client import Client +from helpers.cluster import ClickHouseCluster +import os.path +from os import remove + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ssl_config.xml", + "certs/self-key.pem", + "certs/self-cert.pem", + "certs/ca-cert.pem", + ], +) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + +config_default = """ +""" + +config_accept = """ + 1 +""" + +config_connection_accept = """ + + + {ip_address} + 1 + + +""" + + +def execute_query_native(node, query, config): + config_path = f"{SCRIPT_DIR}/configs/client.xml" + + file = open(config_path, "w") + file.write(config) + file.close() + + client = Client( + node.ip_address, + 9440, + command=cluster.client_bin_path, + secure=True, + config=config_path, + ) + + try: + result = client.query(query) + remove(config_path) + return result + except: + remove(config_path) + raise + + +def test_default(): + with pytest.raises(Exception) as err: + execute_query_native( + instance, "SELECT 1", config_default + ) + assert "certificate verify failed" in str(err.value) + +def test_accept(): + assert ( + execute_query_native( + instance, "SELECT 1", config_accept + ) + == "1\n" + ) + +def test_connection_accept(): + assert ( + execute_query_native( + instance, "SELECT 1", config_connection_accept.format(ip_address=f"{instance.ip_address}") + ) + == "1\n" + ) diff --git a/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh b/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh deleted file mode 100644 index 00378abdb8e..00000000000 --- a/tests/queries/0_stateless/01189_clickhouse_accept_invalid_certificate.sh +++ /dev/null @@ -1,107 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-random-settings - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# use $CLICKHOUSE_DATABASE so that clickhouse-test will replace it with default to match .reference -config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE -xml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.xml -XML_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.XML -conf_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.conf -yml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yml -yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml -ini_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.ini - -function cleanup() { - rm "${config:?}" - rm "${xml_config:?}" - rm "${XML_config:?}" - rm "${conf_config:?}" - rm "${yml_config:?}" - rm "${yaml_config:?}" - rm "${ini_config:?}" -} -trap cleanup EXIT - -cat >"$config" < - - - - RejectCertificateHandler - - - - -EOL -cat >"$conf_config" < - - - - RejectCertificateHandler - - - - -EOL -cat >"$xml_config" < - - - - RejectCertificateHandler - - - - -EOL -cat >"$XML_config" < - - - - RejectCertificateHandler - - - - -EOL -cat >"$yml_config" <"$yaml_config" <"$ini_config" <RejectCertificateHandler') -q "select getSetting('invalidCertificateHandler')" - -echo 'yml' -$CLICKHOUSE_CLIENT --config "$yml_config" -q "select getSetting('invalidCertificateHandler')" -echo 'yaml' -$CLICKHOUSE_CLIENT --config "$yaml_config" -q "select getSetting('invalidCertificateHandler')" -echo 'ini' -$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('invalidCertificateHandler')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 4af4e27653c..11d3c994230 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -14,44 +14,45 @@ yml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yml yaml_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.yaml ini_config=$CLICKHOUSE_TMP/config_$CLICKHOUSE_DATABASE.ini -function cleanup() { - rm "${config:?}" - rm "${xml_config:?}" - rm "${XML_config:?}" - rm "${conf_config:?}" - rm "${yml_config:?}" - rm "${yaml_config:?}" - rm "${ini_config:?}" +function cleanup() +{ + rm "${config:?}" + rm "${xml_config:?}" + rm "${XML_config:?}" + rm "${conf_config:?}" + rm "${yml_config:?}" + rm "${yaml_config:?}" + rm "${ini_config:?}" } trap cleanup EXIT -cat >"$config" < "$config" < 2 EOL -cat >"$conf_config" < "$conf_config" < 2 EOL -cat >"$xml_config" < "$xml_config" < 2 EOL -cat >"$XML_config" < "$XML_config" < 2 EOL -cat >"$yml_config" < "$yml_config" <"$yaml_config" < "$yaml_config" <"$ini_config" < "$ini_config" <&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" +$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file diff --git a/tests/queries/0_stateless/02550_client_connections_credentials.sh b/tests/queries/0_stateless/02550_client_connections_credentials.sh index b7319d22d1b..d89fca0fe97 100755 --- a/tests/queries/0_stateless/02550_client_connections_credentials.sh +++ b/tests/queries/0_stateless/02550_client_connections_credentials.sh @@ -14,7 +14,7 @@ CLICKHOUSE_PORT_TCP="" . "$CUR_DIR"/../shell_config.sh CONFIG=$CLICKHOUSE_TMP/client.xml -cat >$CONFIG < $CONFIG < $TEST_HOST $TEST_PORT @@ -66,21 +66,12 @@ cat >$CONFIG <$TEST_HOST /no/such/dir/.history - - - test_accept_invalid_certificate - $TEST_HOST - $CLICKHOUSE_DATABASE - - RejectCertificateHandler - - EOL CONFIG_ROOT_OVERRIDES=$CLICKHOUSE_TMP/client_user_pass.xml -cat >$CONFIG_ROOT_OVERRIDES < $CONFIG_ROOT_OVERRIDES < $TEST_HOST $TEST_PORT @@ -140,4 +131,4 @@ $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection incorrect_auth - $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection default -q 'select currentUser()' $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection default --user foo -q 'select currentUser()' |& grep -F -o 'foo: Authentication failed: password is incorrect, or there is no user with such name.' -rm -f "${CONFIG:?}" +rm -f "${CONFIG:?}" \ No newline at end of file From 8673add9909211b3493e004ff6683d3124830f67 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Jul 2024 06:06:03 +0000 Subject: [PATCH 025/178] restore unrelated tests --- .../0_stateless/01889_clickhouse_client_config_format.sh | 2 +- .../queries/0_stateless/02550_client_connections_credentials.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh index 11d3c994230..4e970f17d3a 100755 --- a/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh +++ b/tests/queries/0_stateless/01889_clickhouse_client_config_format.sh @@ -75,4 +75,4 @@ $CLICKHOUSE_CLIENT --config "$yml_config" -q "select getSetting('max_threads')" echo 'yaml' $CLICKHOUSE_CLIENT --config "$yaml_config" -q "select getSetting('max_threads')" echo 'ini' -$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" \ No newline at end of file +$CLICKHOUSE_CLIENT --config "$ini_config" -q "select getSetting('max_threads')" 2>&1 |& sed -e "s#$CLICKHOUSE_TMP##" -e "s#DB::Exception: ##" diff --git a/tests/queries/0_stateless/02550_client_connections_credentials.sh b/tests/queries/0_stateless/02550_client_connections_credentials.sh index d89fca0fe97..a88f3fc7880 100755 --- a/tests/queries/0_stateless/02550_client_connections_credentials.sh +++ b/tests/queries/0_stateless/02550_client_connections_credentials.sh @@ -131,4 +131,4 @@ $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection incorrect_auth - $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection default -q 'select currentUser()' $CLICKHOUSE_CLIENT --config $CONFIG_ROOT_OVERRIDES --connection default --user foo -q 'select currentUser()' |& grep -F -o 'foo: Authentication failed: password is incorrect, or there is no user with such name.' -rm -f "${CONFIG:?}" \ No newline at end of file +rm -f "${CONFIG:?}" From 7c796113e83373063675a2c4189fe3ce115897f0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Jul 2024 06:34:05 +0000 Subject: [PATCH 026/178] fix style --- .../test_accept_invalid_certificate/test.py | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_accept_invalid_certificate/test.py b/tests/integration/test_accept_invalid_certificate/test.py index 13431f39854..87229d75f90 100644 --- a/tests/integration/test_accept_invalid_certificate/test.py +++ b/tests/integration/test_accept_invalid_certificate/test.py @@ -19,6 +19,7 @@ instance = cluster.add_instance( ], ) + @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: @@ -28,6 +29,7 @@ def started_cluster(): finally: cluster.shutdown() + config_default = """ """ @@ -71,23 +73,20 @@ def execute_query_native(node, query, config): def test_default(): with pytest.raises(Exception) as err: - execute_query_native( - instance, "SELECT 1", config_default - ) + execute_query_native(instance, "SELECT 1", config_default) assert "certificate verify failed" in str(err.value) + def test_accept(): - assert ( - execute_query_native( - instance, "SELECT 1", config_accept - ) - == "1\n" - ) + assert execute_query_native(instance, "SELECT 1", config_accept) == "1\n" + def test_connection_accept(): assert ( execute_query_native( - instance, "SELECT 1", config_connection_accept.format(ip_address=f"{instance.ip_address}") + instance, + "SELECT 1", + config_connection_accept.format(ip_address=f"{instance.ip_address}"), ) == "1\n" ) From f1105374cb30d6eb3323e25fe9a4bdcbe9e5ba1f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Jul 2024 12:28:39 +0000 Subject: [PATCH 027/178] fix style --- tests/integration/test_accept_invalid_certificate/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_accept_invalid_certificate/__init__.py diff --git a/tests/integration/test_accept_invalid_certificate/__init__.py b/tests/integration/test_accept_invalid_certificate/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From f76bd8ed776bc34829a7ff2de394936dc6035d01 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Tue, 2 Jul 2024 14:58:04 +0000 Subject: [PATCH 028/178] apply some comments --- src/Databases/DatabaseAtomic.cpp | 6 ------ src/Databases/DatabaseAtomic.h | 3 --- src/Databases/DatabaseOnDisk.cpp | 9 ++++++--- src/Databases/TablesLoader.cpp | 10 +++++----- src/Storages/System/StorageSystemTables.cpp | 12 ------------ .../0_stateless/03172_system_detached_tables.sql | 2 +- 6 files changed, 12 insertions(+), 30 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ddc30627cde..b8d7315fc0f 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -437,12 +437,6 @@ DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::Fil return std::make_unique(std::move(typeid_cast(*base_iter))); } -DatabaseDetachedTablesSnapshotIteratorPtr DatabaseAtomic::getDetachedTablesIterator( - ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name, const bool skip_not_loaded) const -{ - return DatabaseOrdinary::getDetachedTablesIterator(local_context, filter_by_table_name, skip_not_loaded); -} - UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const { if (auto table = tryGetTable(table_name, getContext())) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 25c5cdfbde1..4a4ccfa2573 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -49,9 +49,6 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; - DatabaseDetachedTablesSnapshotIteratorPtr - getDetachedTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; - void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 67c9114e793..61de0727ba6 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -22,8 +22,8 @@ #include #include #include -#include "Common/Exception.h" #include +#include #include #include #include @@ -310,11 +310,14 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const Stri FS::createFile(detached_permanently_flag); std::lock_guard lock(mutex); - if (!snapshot_detached_tables.contains(table_name)) + if (const auto it = snapshot_detached_tables.find(table_name); it == snapshot_detached_tables.end()) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Snapshot doesn't contain info about detached table={}", table_name); } - snapshot_detached_tables[table_name].is_permanently = true; + else + { + it->second.is_permanently = true; + } } catch (Exception & e) { diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index 1f70f46da60..6aa13b7b759 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -1,14 +1,14 @@ -#include +#include +#include #include #include -#include -#include -#include #include +#include #include #include -#include #include +#include +#include namespace DB diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index dd7f1a6fdc8..43b761d84b1 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -50,18 +50,6 @@ bool needTable(const DatabasePtr & database, const Block & header) } return false; } - -template -void fillColumns(T table_it, MutableColumnPtr & database_column, MutableColumnPtr & engine_column) -{ - for (; table_it->isValid(); table_it->next()) - { - database_column->insert(table_it->name()); - if (engine_column) - engine_column->insert(table_it->table()->getName()); - } -} - } namespace detail diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql index 916821d996b..64f30a9d275 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -2,7 +2,7 @@ SELECT 'database atomic tests'; DROP DATABASE IF EXISTS test03172_system_detached_tables; -CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables; +CREATE DATABASE IF NOT EXISTS test03172_system_detached_tables ENGINE=Atomic; CREATE TABLE test03172_system_detached_tables.test_table (n Int64) ENGINE=MergeTree ORDER BY n; SELECT * FROM system.detached_tables WHERE database='test03172_system_detached_tables'; From 2f083fd1a709dd236ffaf37a7a059cafef8c4a41 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Jul 2024 15:08:14 +0000 Subject: [PATCH 029/178] Fix resolving dynamic subcolumns in analyzer --- src/Analyzer/Resolve/IdentifierResolver.cpp | 35 ++++++++++++++++--- .../03198_dynamic_read_subcolumns.reference | 8 +++++ .../03198_dynamic_read_subcolumns.sql | 18 ++++++++++ 3 files changed, 56 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference create mode 100644 tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 692a31b66ba..4347f07e829 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -678,9 +679,33 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( bool match_full_identifier = false; const auto & identifier_full_name = identifier_without_column_qualifier.getFullName(); - auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name); - bool can_resolve_directly_from_storage = it != table_expression_data.column_name_to_column_node.end(); - if (can_resolve_directly_from_storage && table_expression_data.subcolumn_names.contains(identifier_full_name)) + + ColumnNodePtr result_column_node; + bool can_resolve_directly_from_storage = false; + bool is_subcolumn = false; + if (auto it = table_expression_data.column_name_to_column_node.find(identifier_full_name); it != table_expression_data.column_name_to_column_node.end()) + { + can_resolve_directly_from_storage = true; + is_subcolumn = table_expression_data.subcolumn_names.contains(identifier_full_name); + result_column_node = it->second; + } + /// Check if it's a dynamic subcolumn + else + { + auto [column_name, dynamic_subcolumn_name] = Nested::splitName(identifier_full_name); + auto jt = table_expression_data.column_name_to_column_node.find(column_name); + if (jt != table_expression_data.column_name_to_column_node.end() && jt->second->getColumnType()->hasDynamicSubcolumns()) + { + if (auto dynamic_subcolumn_type = jt->second->getColumnType()->tryGetSubcolumnType(dynamic_subcolumn_name)) + { + result_column_node = std::make_shared(NameAndTypePair{identifier_full_name, dynamic_subcolumn_type}, jt->second->getColumnSource()); + can_resolve_directly_from_storage = true; + is_subcolumn = true; + } + } + } + + if (can_resolve_directly_from_storage && is_subcolumn) { /** In the case when we have an ARRAY JOIN, we should not resolve subcolumns directly from storage. * For example, consider the following SQL query: @@ -696,11 +721,11 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( if (can_resolve_directly_from_storage) { match_full_identifier = true; - result_expression = it->second; + result_expression = result_column_node; } else { - it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); + auto it = table_expression_data.column_name_to_column_node.find(identifier_without_column_qualifier.at(0)); if (it != table_expression_data.column_name_to_column_node.end()) result_expression = it->second; } diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference new file mode 100644 index 00000000000..59d65220c83 --- /dev/null +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference @@ -0,0 +1,8 @@ +foo +\N +\N +foo +\N +\N +6 +6 diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql new file mode 100644 index 00000000000..7f88b4cd347 --- /dev/null +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -0,0 +1,18 @@ +SET allow_experimental_dynamic_type = 1; +DROP TABLE IF EXISTS test_dynamic; +CREATE TABLE test_dynamic (id UInt64, d Dynamic) ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; +INSERT INTO test_dynamic VALUES (1, 'foo'), (2, 1111), (3, [1, 2, 3]); +SYSTEM DROP MARK CACHE; +SELECT d.String FROM test_dynamic SETTINGS allow_experimental_analyzer = 1; +SYSTEM DROP MARK CACHE; +SELECT d.String FROM test_dynamic SETTINGS allow_experimental_analyzer = 0; +SYSTEM FLUSH LOGS; +SELECT + ProfileEvents['FileOpen'] +FROM system.query_log +WHERE (type = 2) AND (query LIKE 'SELECT d.String %test_dynamic%') AND (current_database = currentDatabase()) +ORDER BY event_time_microseconds DESC +LIMIT 2; + +DROP TABLE test_dynamic; + From ac315785d3ab89756b160ef1610bf5e4125c332b Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Jul 2024 23:41:57 +0000 Subject: [PATCH 030/178] Update tests --- .../03040_dynamic_type_alters_1.reference | 108 +++++++++--------- .../03040_dynamic_type_alters_1.sh | 2 +- 2 files changed, 55 insertions(+), 55 deletions(-) diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference index ca98ec0963c..b41645f0c4a 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference @@ -30,16 +30,16 @@ alter modify column 1 0 0 \N \N \N \N \N 1 1 \N \N \N \N \N 2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N +3 3 3 \N 3 \N \N +4 4 4 \N 4 \N \N +5 5 5 \N 5 \N \N 6 6 str_6 str_6 \N \N \N 7 7 str_7 str_7 \N \N \N 8 8 str_8 str_8 \N \N \N 9 9 \N \N \N \N \N 10 10 \N \N \N \N \N 11 11 \N \N \N \N \N -12 12 12 12 \N \N \N +12 12 12 \N 12 \N \N 13 13 str_13 str_13 \N \N \N 14 14 \N \N \N \N \N insert after alter modify column 1 @@ -48,16 +48,16 @@ insert after alter modify column 1 0 0 \N \N \N \N \N 1 1 \N \N \N \N \N 2 2 \N \N \N \N \N -3 3 3 3 \N \N \N -4 4 4 4 \N \N \N -5 5 5 5 \N \N \N +3 3 3 \N 3 \N \N +4 4 4 \N 4 \N \N +5 5 5 \N 5 \N \N 6 6 str_6 str_6 \N \N \N 7 7 str_7 str_7 \N \N \N 8 8 str_8 str_8 \N \N \N 9 9 \N \N \N \N \N 10 10 \N \N \N \N \N 11 11 \N \N \N \N \N -12 12 12 12 \N \N \N +12 12 12 \N 12 \N \N 13 13 str_13 str_13 \N \N \N 14 14 \N \N \N \N \N 15 15 \N \N \N \N \N @@ -120,57 +120,57 @@ alter modify column 3 5 UInt64 8 String 9 None -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N \N 3 \N \N -4 4 4 \N \N \N 4 \N \N -5 5 5 \N \N \N 5 \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N \N 12 \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N +0 0 \N \N \N \N \N \N \N +1 1 \N \N \N \N \N \N \N +2 2 \N \N \N \N \N \N \N +3 3 \N \N \N \N 3 \N \N +4 4 \N \N \N \N 4 \N \N +5 5 \N \N \N \N 5 \N \N +6 6 \N \N \N str_6 \N \N \N +7 7 \N \N \N str_7 \N \N \N +8 8 \N \N \N str_8 \N \N \N +9 9 \N \N \N \N \N \N \N +10 10 \N \N \N \N \N \N \N +11 11 \N \N \N \N \N \N \N +12 12 \N \N \N \N 12 \N \N +13 13 \N \N \N str_13 \N \N \N +14 14 \N \N \N \N \N \N \N +15 15 \N \N \N \N \N \N \N +16 16 \N \N \N 16 \N \N \N +17 17 \N \N \N str_17 \N \N \N +18 18 \N \N \N 1970-01-19 \N \N \N +19 19 \N \N \N \N \N \N \N +20 20 \N \N \N \N 20 \N \N +21 21 \N \N \N str_21 \N \N \N +22 22 \N \N \N \N \N 1970-01-23 \N insert after alter modify column 3 1 Date 5 UInt64 8 String 12 None -0 0 0 \N \N \N \N \N \N -1 1 1 \N \N \N \N \N \N -2 2 2 \N \N \N \N \N \N -3 3 3 \N \N \N 3 \N \N -4 4 4 \N \N \N 4 \N \N -5 5 5 \N \N \N 5 \N \N -6 6 6 \N \N str_6 \N \N \N -7 7 7 \N \N str_7 \N \N \N -8 8 8 \N \N str_8 \N \N \N -9 9 9 \N \N \N \N \N \N -10 10 10 \N \N \N \N \N \N -11 11 11 \N \N \N \N \N \N -12 12 12 \N \N \N 12 \N \N -13 13 13 \N \N str_13 \N \N \N -14 14 14 \N \N \N \N \N \N -15 15 15 \N \N \N \N \N \N -16 16 16 \N \N 16 \N \N \N -17 17 17 \N \N str_17 \N \N \N -18 18 18 \N \N 1970-01-19 \N \N \N -19 19 19 \N \N \N \N \N \N -20 20 20 \N \N \N 20 \N \N -21 21 21 \N \N str_21 \N \N \N -22 22 22 \N \N \N \N 1970-01-23 \N +0 0 \N \N \N \N \N \N \N +1 1 \N \N \N \N \N \N \N +2 2 \N \N \N \N \N \N \N +3 3 \N \N \N \N 3 \N \N +4 4 \N \N \N \N 4 \N \N +5 5 \N \N \N \N 5 \N \N +6 6 \N \N \N str_6 \N \N \N +7 7 \N \N \N str_7 \N \N \N +8 8 \N \N \N str_8 \N \N \N +9 9 \N \N \N \N \N \N \N +10 10 \N \N \N \N \N \N \N +11 11 \N \N \N \N \N \N \N +12 12 \N \N \N \N 12 \N \N +13 13 \N \N \N str_13 \N \N \N +14 14 \N \N \N \N \N \N \N +15 15 \N \N \N \N \N \N \N +16 16 \N \N \N 16 \N \N \N +17 17 \N \N \N str_17 \N \N \N +18 18 \N \N \N 1970-01-19 \N \N \N +19 19 \N \N \N \N \N \N \N +20 20 \N \N \N \N 20 \N \N +21 21 \N \N \N str_21 \N \N \N +22 22 \N \N \N \N \N 1970-01-23 \N 23 \N \N \N \N \N \N \N \N 24 24 24 \N \N \N \N \N \N 25 str_25 \N str_25 \N \N \N \N \N diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh b/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh index 7a73be20a4d..080a7e583bf 100755 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_analyzer=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_analyzer=0" function run() { From 65a34b8f63cc0cd6fda3d454e7505a01f0a7a75a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 07:27:44 +0000 Subject: [PATCH 031/178] Fix test flakiness --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 7f88b4cd347..a9e3d88b7c7 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; CREATE TABLE test_dynamic (id UInt64, d Dynamic) ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; From 192ada5ed93f93c17f26671336c6c62e53a1b669 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 3 Jul 2024 08:37:50 +0000 Subject: [PATCH 032/178] apply comments --- src/Databases/DatabaseAtomic.cpp | 1 - src/Storages/System/StorageSystemDetachedTables.cpp | 13 +++++++++---- .../03172_system_detached_tables.reference | 4 ++-- .../0_stateless/03172_system_detached_tables.sql | 4 ++-- 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b8d7315fc0f..65672f0583c 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -103,7 +103,6 @@ void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, auto table_id = table->getStorageID(); assertDetachedTableNotInUse(table_id.uuid); DatabaseOrdinary::attachTableUnlocked(name, table); - detached_tables.erase(table_id.uuid); table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 560056b891a..158edb7818f 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -31,12 +31,12 @@ class DetachedTablesBlockSource : public ISource public: DetachedTablesBlockSource( std::vector columns_mask_, - Block header, + Block header_, UInt64 max_block_size_, ColumnPtr databases_, ColumnPtr detached_tables_, ContextPtr context_) - : ISource(std::move(header)) + : ISource(std::move(header_)) , columns_mask(std::move(columns_mask_)) , max_block_size(max_block_size_) , databases(std::move(databases_)) @@ -63,7 +63,9 @@ protected: const auto access = context->getAccess(); const bool need_to_check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - for (size_t database_idx = 0, rows_count = 0; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) + size_t database_idx = 0; + size_t rows_count = 0; + for (; database_idx < databases->size() && rows_count < max_block_size; ++database_idx) { database_name = databases->getDataAt(database_idx).toString(); database = DatabaseCatalog::instance().tryGetDatabase(database_name); @@ -92,8 +94,11 @@ protected: } } + if (databases->size() == database_idx && max_block_size != rows_count) + { + done = true; + } const UInt64 num_rows = result_columns.at(0)->size(); - done = true; return Chunk(std::move(result_columns), num_rows); } diff --git a/tests/queries/0_stateless/03172_system_detached_tables.reference b/tests/queries/0_stateless/03172_system_detached_tables.reference index ce19daa4cdc..83d1ff13942 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.reference +++ b/tests/queries/0_stateless/03172_system_detached_tables.reference @@ -6,6 +6,6 @@ test03172_system_detached_tables test_table_perm 1 test03172_system_detached_tables test_table 0 ----------------------- database lazy tests -test03172_system_detached_tables_lazy test_table 0 -test03172_system_detached_tables_lazy test_table_perm 1 +before attach test03172_system_detached_tables_lazy test_table 0 +before attach test03172_system_detached_tables_lazy test_table_perm 1 DROP TABLE diff --git a/tests/queries/0_stateless/03172_system_detached_tables.sql b/tests/queries/0_stateless/03172_system_detached_tables.sql index 64f30a9d275..1a3c2d7cc0f 100644 --- a/tests/queries/0_stateless/03172_system_detached_tables.sql +++ b/tests/queries/0_stateless/03172_system_detached_tables.sql @@ -39,12 +39,12 @@ CREATE TABLE test03172_system_detached_tables_lazy.test_table_perm (number UInt6 INSERT INTO test03172_system_detached_tables_lazy.test_table_perm SELECT * FROM numbers(100); DETACH table test03172_system_detached_tables_lazy.test_table_perm PERMANENTLY; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; +SELECT 'before attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; ATTACH TABLE test03172_system_detached_tables_lazy.test_table; ATTACH TABLE test03172_system_detached_tables_lazy.test_table_perm; -SELECT database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; +SELECT 'after attach', database, table, is_permanently FROM system.detached_tables WHERE database='test03172_system_detached_tables_lazy'; SELECT 'DROP TABLE'; DROP TABLE test03172_system_detached_tables_lazy.test_table SYNC; From 7616f58a0bed80525385879ac3a2a4dcdf2707f4 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 3 Jul 2024 11:36:04 +0000 Subject: [PATCH 033/178] add tests for another db engines --- .../configs/remote_servers.xml | 13 ++++ .../test_system_detached_tables/test.py | 69 +++++++++++++------ 2 files changed, 61 insertions(+), 21 deletions(-) create mode 100644 tests/integration/test_system_detached_tables/configs/remote_servers.xml diff --git a/tests/integration/test_system_detached_tables/configs/remote_servers.xml b/tests/integration/test_system_detached_tables/configs/remote_servers.xml new file mode 100644 index 00000000000..8b579724282 --- /dev/null +++ b/tests/integration/test_system_detached_tables/configs/remote_servers.xml @@ -0,0 +1,13 @@ + + + + + true + + replica1 + 9000 + + + + + \ No newline at end of file diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index dbccac92523..f2bb5d71e85 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -2,8 +2,12 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node_default", stay_alive=True) - +node = cluster.add_instance("replica1", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml"], + macros={"replica": "replica1"}, + stay_alive=True, +) @pytest.fixture(scope="module", autouse=True) def start_cluster(): @@ -13,10 +17,25 @@ def start_cluster(): finally: cluster.shutdown() +@pytest.mark.parametrize( + "db_name,db_engine,table_engine,table_engine_params", + [ + pytest.param( + "test_db_atomic", "Atomic", "MergeTree", "ORDER BY n", id="Atomic db with MergeTree table" + ), + pytest.param( + "test_db_lazy", "Lazy(60)", "Log", "", id="Lazy db with Log table" + ), + pytest.param( + "test_db_repl", "Replicated('/clickhouse/tables/test_table', 'shard1', 'replica1')", "ReplicatedMergeTree", "ORDER BY n", id="Replicated db with ReplicatedMergeTree table" + ), + ], +) +def test_system_detached_tables(start_cluster, db_name, db_engine, table_engine, table_engine_params): + node.query(f"CREATE DATABASE IF NOT EXISTS {db_name} ENGINE={db_engine};") -def test_system_detached_tables(): - node.query("CREATE TABLE test_table (n Int64) ENGINE=MergeTree ORDER BY n;") - node.query("CREATE TABLE test_table_perm (n Int64) ENGINE=MergeTree ORDER BY n;") + node.query(f"CREATE TABLE {db_name}.test_table (n Int64) ENGINE={table_engine} {table_engine_params};") + node.query(f"CREATE TABLE {db_name}.test_table_perm (n Int64) ENGINE={table_engine} {table_engine_params};") test_table_uuid = node.query( "SELECT uuid FROM system.tables WHERE table='test_table'" @@ -32,29 +51,37 @@ def test_system_detached_tables(): "SELECT metadata_path FROM system.tables WHERE table='test_table_perm'" ).rstrip("\n") - assert "" == node.query("SELECT * FROM system.detached_tables") + assert "" == node.query(f"SELECT * FROM system.detached_tables WHERE database='{db_name}'") - node.query("DETACH TABLE test_table") - node.query("DETACH TABLE test_table_perm PERMANENTLY") + node.query(f"SET database_replicated_always_detach_permanently=1; DETACH TABLE {db_name}.test_table") + node.query(f"DETACH TABLE {db_name}.test_table_perm PERMANENTLY") - querry = "SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables FORMAT Values" + querry = f"SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables WHERE database='{db_name}' FORMAT Values" result = node.query(querry) - assert ( - result - == f"('default','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" - ) + + if db_engine.startswith("Repl"): + expected_before_restart = f"('{db_name}','test_table',1,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + else: + expected_before_restart = f"('{db_name}','test_table',0,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + + assert result == expected_before_restart + + if db_engine.startswith("Lazy"): + return + node.restart_clickhouse() + if db_engine.startswith("Repl"): + expected_after_restart = expected_before_restart + else: + expected_after_restart = f"('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" + result = node.query(querry) - assert ( - result - == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" - ) + assert result == expected_after_restart node.restart_clickhouse() result = node.query(querry) - assert ( - result - == f"('default','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" - ) + assert result == expected_after_restart + + node.query(f"DROP DATABASE {db_name}") From 1bb1e0189207c458550ab677983e3efeebad23b1 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 3 Jul 2024 12:12:09 +0000 Subject: [PATCH 034/178] fix style --- .../test_system_detached_tables/test.py | 42 ++++++++++++++----- 1 file changed, 32 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index f2bb5d71e85..dfb55dfd99b 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -2,7 +2,8 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("replica1", +node = cluster.add_instance( + "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"], macros={"replica": "replica1"}, @@ -17,25 +18,42 @@ def start_cluster(): finally: cluster.shutdown() + @pytest.mark.parametrize( "db_name,db_engine,table_engine,table_engine_params", [ pytest.param( - "test_db_atomic", "Atomic", "MergeTree", "ORDER BY n", id="Atomic db with MergeTree table" + "test_db_atomic", + "Atomic", + "MergeTree", + "ORDER BY n", + id="Atomic db with MergeTree table", ), pytest.param( - "test_db_lazy", "Lazy(60)", "Log", "", id="Lazy db with Log table" + "test_db_lazy", + "Lazy(60)", + "Log", + "", + id="Lazy db with Log table", ), pytest.param( - "test_db_repl", "Replicated('/clickhouse/tables/test_table', 'shard1', 'replica1')", "ReplicatedMergeTree", "ORDER BY n", id="Replicated db with ReplicatedMergeTree table" + "test_db_repl", + "Replicated('/clickhouse/tables/test_table','shard1', 'replica1')", + "ReplicatedMergeTree", + "ORDER BY n", + id="Replicated db with ReplicatedMergeTree table", ), ], ) def test_system_detached_tables(start_cluster, db_name, db_engine, table_engine, table_engine_params): node.query(f"CREATE DATABASE IF NOT EXISTS {db_name} ENGINE={db_engine};") - node.query(f"CREATE TABLE {db_name}.test_table (n Int64) ENGINE={table_engine} {table_engine_params};") - node.query(f"CREATE TABLE {db_name}.test_table_perm (n Int64) ENGINE={table_engine} {table_engine_params};") + node.query( + f"CREATE TABLE {db_name}.test_table (n Int64) ENGINE={table_engine} {table_engine_params};" + ) + node.query( + f"CREATE TABLE {db_name}.test_table_perm (n Int64) ENGINE={table_engine} {table_engine_params};" + ) test_table_uuid = node.query( "SELECT uuid FROM system.tables WHERE table='test_table'" @@ -51,14 +69,18 @@ def test_system_detached_tables(start_cluster, db_name, db_engine, table_engine, "SELECT metadata_path FROM system.tables WHERE table='test_table_perm'" ).rstrip("\n") - assert "" == node.query(f"SELECT * FROM system.detached_tables WHERE database='{db_name}'") + assert "" == node.query( + f"SELECT * FROM system.detached_tables WHERE database='{db_name}'" + ) - node.query(f"SET database_replicated_always_detach_permanently=1; DETACH TABLE {db_name}.test_table") + node.query( + f"SET database_replicated_always_detach_permanently=1; DETACH TABLE {db_name}.test_table" + ) node.query(f"DETACH TABLE {db_name}.test_table_perm PERMANENTLY") querry = f"SELECT database, table, is_permanently, uuid, metadata_path FROM system.detached_tables WHERE database='{db_name}' FORMAT Values" result = node.query(querry) - + if db_engine.startswith("Repl"): expected_before_restart = f"('{db_name}','test_table',1,'{test_table_uuid}','{test_table_metadata_path}'),('{db_name}','test_table_perm',1,'{test_table_perm_uuid}','{test_table_perm_metadata_path}')" else: @@ -68,7 +90,7 @@ def test_system_detached_tables(start_cluster, db_name, db_engine, table_engine, if db_engine.startswith("Lazy"): return - + node.restart_clickhouse() if db_engine.startswith("Repl"): From f964b0b6ab48bd92a784a5c719817e2c82fa1f4a Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 3 Jul 2024 12:22:32 +0000 Subject: [PATCH 035/178] fix style again --- tests/integration/test_system_detached_tables/test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_system_detached_tables/test.py b/tests/integration/test_system_detached_tables/test.py index dfb55dfd99b..2eb870efcbc 100644 --- a/tests/integration/test_system_detached_tables/test.py +++ b/tests/integration/test_system_detached_tables/test.py @@ -10,6 +10,7 @@ node = cluster.add_instance( stay_alive=True, ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -38,14 +39,16 @@ def start_cluster(): ), pytest.param( "test_db_repl", - "Replicated('/clickhouse/tables/test_table','shard1', 'replica1')", - "ReplicatedMergeTree", + "Replicated('/clickhouse/tables/test_table','shard1', 'replica1')", + "ReplicatedMergeTree", "ORDER BY n", id="Replicated db with ReplicatedMergeTree table", ), ], ) -def test_system_detached_tables(start_cluster, db_name, db_engine, table_engine, table_engine_params): +def test_system_detached_tables( + start_cluster, db_name, db_engine, table_engine, table_engine_params +): node.query(f"CREATE DATABASE IF NOT EXISTS {db_name} ENGINE={db_engine};") node.query( From be3d707a5b519e9d810bfcf65b10379b00323762 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 16:37:01 +0000 Subject: [PATCH 036/178] Update test --- .../0_stateless/03198_dynamic_read_subcolumns.reference | 9 +++++++++ .../0_stateless/03198_dynamic_read_subcolumns.sql | 3 ++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference index 59d65220c83..f9fe0c2ad77 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.reference @@ -1,3 +1,12 @@ +QUERY id: 0 + PROJECTION COLUMNS + d.String Nullable(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: d.String, result_type: Nullable(String), source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test_dynamic + SETTINGS allow_experimental_analyzer=1 foo \N \N diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index a9e3d88b7c7..27fca179580 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,9 +1,10 @@ --- Tags: no-random-settings +-- Tags: no-random-settings, no-s3-storage SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; CREATE TABLE test_dynamic (id UInt64, d Dynamic) ENGINE = MergeTree ORDER BY id SETTINGS min_bytes_for_wide_part = 0; INSERT INTO test_dynamic VALUES (1, 'foo'), (2, 1111), (3, [1, 2, 3]); +EXPLAIN QUERY TREE SELECT d.String FROM test_dynamic SETTINGS allow_experimental_analyzer = 1; SYSTEM DROP MARK CACHE; SELECT d.String FROM test_dynamic SETTINGS allow_experimental_analyzer = 1; SYSTEM DROP MARK CACHE; From a2de85427473eb3ec982a3fc79cb85fe0ab3cbcb Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 3 Jul 2024 16:37:54 +0000 Subject: [PATCH 037/178] Remove allow_experimental_analyzer from the test --- tests/queries/0_stateless/03040_dynamic_type_alters_1.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh b/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh index 080a7e583bf..1f2a6a31ad7 100755 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_analyzer=0" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1" function run() { From cc3e166dec6fde41c3903d486e4e18ba9b2991b2 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Thu, 4 Jul 2024 12:11:15 +0000 Subject: [PATCH 038/178] apply comment --- src/Storages/System/StorageSystemDetachedTables.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDetachedTables.cpp b/src/Storages/System/StorageSystemDetachedTables.cpp index 158edb7818f..56c5e49b467 100644 --- a/src/Storages/System/StorageSystemDetachedTables.cpp +++ b/src/Storages/System/StorageSystemDetachedTables.cpp @@ -94,7 +94,7 @@ protected: } } - if (databases->size() == database_idx && max_block_size != rows_count) + if (databases->size() == database_idx && (!detached_tables_it || !detached_tables_it->isValid())) { done = true; } From 20f1256a2fce9ea9604d58bb4065302af06422ec Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Jul 2024 18:29:09 +0000 Subject: [PATCH 039/178] Debuging 02956_rocksdb_bulk_sink --- src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 31812406d34..6f0f5f19970 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -199,9 +199,10 @@ void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) if (chunks_to_write.empty()) return; + size_t num_chunks = chunks_to_write.size(); auto [serialized_key_column, serialized_value_column] = serializeChunks(std::move(chunks_to_write)); auto sst_file_path = getTemporarySSTFilePath(); - LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows to SST file {}", serialized_key_column->size(), sst_file_path); + LOG_DEBUG(getLogger("EmbeddedRocksDBBulkSink"), "Writing {} rows from {} chunks to SST file {}", serialized_key_column->size(), num_chunks, sst_file_path); if (auto status = buildSSTFile(sst_file_path, *serialized_key_column, *serialized_value_column); !status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); From f0966f519b5bbc88c63f604ab97603b504023869 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 4 Jul 2024 21:00:51 +0000 Subject: [PATCH 040/178] Revert changes in test reference --- .../03040_dynamic_type_alters_1.reference | 108 +++++++++--------- 1 file changed, 54 insertions(+), 54 deletions(-) diff --git a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference index b41645f0c4a..ca98ec0963c 100644 --- a/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference +++ b/tests/queries/0_stateless/03040_dynamic_type_alters_1.reference @@ -30,16 +30,16 @@ alter modify column 1 0 0 \N \N \N \N \N 1 1 \N \N \N \N \N 2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N +3 3 3 3 \N \N \N +4 4 4 4 \N \N \N +5 5 5 5 \N \N \N 6 6 str_6 str_6 \N \N \N 7 7 str_7 str_7 \N \N \N 8 8 str_8 str_8 \N \N \N 9 9 \N \N \N \N \N 10 10 \N \N \N \N \N 11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N +12 12 12 12 \N \N \N 13 13 str_13 str_13 \N \N \N 14 14 \N \N \N \N \N insert after alter modify column 1 @@ -48,16 +48,16 @@ insert after alter modify column 1 0 0 \N \N \N \N \N 1 1 \N \N \N \N \N 2 2 \N \N \N \N \N -3 3 3 \N 3 \N \N -4 4 4 \N 4 \N \N -5 5 5 \N 5 \N \N +3 3 3 3 \N \N \N +4 4 4 4 \N \N \N +5 5 5 5 \N \N \N 6 6 str_6 str_6 \N \N \N 7 7 str_7 str_7 \N \N \N 8 8 str_8 str_8 \N \N \N 9 9 \N \N \N \N \N 10 10 \N \N \N \N \N 11 11 \N \N \N \N \N -12 12 12 \N 12 \N \N +12 12 12 12 \N \N \N 13 13 str_13 str_13 \N \N \N 14 14 \N \N \N \N \N 15 15 \N \N \N \N \N @@ -120,57 +120,57 @@ alter modify column 3 5 UInt64 8 String 9 None -0 0 \N \N \N \N \N \N \N -1 1 \N \N \N \N \N \N \N -2 2 \N \N \N \N \N \N \N -3 3 \N \N \N \N 3 \N \N -4 4 \N \N \N \N 4 \N \N -5 5 \N \N \N \N 5 \N \N -6 6 \N \N \N str_6 \N \N \N -7 7 \N \N \N str_7 \N \N \N -8 8 \N \N \N str_8 \N \N \N -9 9 \N \N \N \N \N \N \N -10 10 \N \N \N \N \N \N \N -11 11 \N \N \N \N \N \N \N -12 12 \N \N \N \N 12 \N \N -13 13 \N \N \N str_13 \N \N \N -14 14 \N \N \N \N \N \N \N -15 15 \N \N \N \N \N \N \N -16 16 \N \N \N 16 \N \N \N -17 17 \N \N \N str_17 \N \N \N -18 18 \N \N \N 1970-01-19 \N \N \N -19 19 \N \N \N \N \N \N \N -20 20 \N \N \N \N 20 \N \N -21 21 \N \N \N str_21 \N \N \N -22 22 \N \N \N \N \N 1970-01-23 \N +0 0 0 \N \N \N \N \N \N +1 1 1 \N \N \N \N \N \N +2 2 2 \N \N \N \N \N \N +3 3 3 \N \N \N 3 \N \N +4 4 4 \N \N \N 4 \N \N +5 5 5 \N \N \N 5 \N \N +6 6 6 \N \N str_6 \N \N \N +7 7 7 \N \N str_7 \N \N \N +8 8 8 \N \N str_8 \N \N \N +9 9 9 \N \N \N \N \N \N +10 10 10 \N \N \N \N \N \N +11 11 11 \N \N \N \N \N \N +12 12 12 \N \N \N 12 \N \N +13 13 13 \N \N str_13 \N \N \N +14 14 14 \N \N \N \N \N \N +15 15 15 \N \N \N \N \N \N +16 16 16 \N \N 16 \N \N \N +17 17 17 \N \N str_17 \N \N \N +18 18 18 \N \N 1970-01-19 \N \N \N +19 19 19 \N \N \N \N \N \N +20 20 20 \N \N \N 20 \N \N +21 21 21 \N \N str_21 \N \N \N +22 22 22 \N \N \N \N 1970-01-23 \N insert after alter modify column 3 1 Date 5 UInt64 8 String 12 None -0 0 \N \N \N \N \N \N \N -1 1 \N \N \N \N \N \N \N -2 2 \N \N \N \N \N \N \N -3 3 \N \N \N \N 3 \N \N -4 4 \N \N \N \N 4 \N \N -5 5 \N \N \N \N 5 \N \N -6 6 \N \N \N str_6 \N \N \N -7 7 \N \N \N str_7 \N \N \N -8 8 \N \N \N str_8 \N \N \N -9 9 \N \N \N \N \N \N \N -10 10 \N \N \N \N \N \N \N -11 11 \N \N \N \N \N \N \N -12 12 \N \N \N \N 12 \N \N -13 13 \N \N \N str_13 \N \N \N -14 14 \N \N \N \N \N \N \N -15 15 \N \N \N \N \N \N \N -16 16 \N \N \N 16 \N \N \N -17 17 \N \N \N str_17 \N \N \N -18 18 \N \N \N 1970-01-19 \N \N \N -19 19 \N \N \N \N \N \N \N -20 20 \N \N \N \N 20 \N \N -21 21 \N \N \N str_21 \N \N \N -22 22 \N \N \N \N \N 1970-01-23 \N +0 0 0 \N \N \N \N \N \N +1 1 1 \N \N \N \N \N \N +2 2 2 \N \N \N \N \N \N +3 3 3 \N \N \N 3 \N \N +4 4 4 \N \N \N 4 \N \N +5 5 5 \N \N \N 5 \N \N +6 6 6 \N \N str_6 \N \N \N +7 7 7 \N \N str_7 \N \N \N +8 8 8 \N \N str_8 \N \N \N +9 9 9 \N \N \N \N \N \N +10 10 10 \N \N \N \N \N \N +11 11 11 \N \N \N \N \N \N +12 12 12 \N \N \N 12 \N \N +13 13 13 \N \N str_13 \N \N \N +14 14 14 \N \N \N \N \N \N +15 15 15 \N \N \N \N \N \N +16 16 16 \N \N 16 \N \N \N +17 17 17 \N \N str_17 \N \N \N +18 18 18 \N \N 1970-01-19 \N \N \N +19 19 19 \N \N \N \N \N \N +20 20 20 \N \N \N 20 \N \N +21 21 21 \N \N str_21 \N \N \N +22 22 22 \N \N \N \N 1970-01-23 \N 23 \N \N \N \N \N \N \N \N 24 24 24 \N \N \N \N \N \N 25 str_25 \N str_25 \N \N \N \N \N From 5e4a244faf99299f0ab4b4f105cc08561c00857b Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Jul 2024 16:52:51 +0200 Subject: [PATCH 041/178] Revert "Revert "insertion deduplication on retries for materialised views"" --- src/Columns/ColumnObject.cpp | 6 + src/Columns/ColumnObject.h | 2 +- src/Common/CollectionOfDerived.h | 184 ++++ src/Core/Settings.h | 6 +- src/Interpreters/AsynchronousInsertQueue.cpp | 19 +- src/Interpreters/InterpreterCheckQuery.cpp | 18 +- src/Interpreters/InterpreterCreateQuery.cpp | 9 +- src/Interpreters/InterpreterExplainQuery.cpp | 8 +- src/Interpreters/InterpreterInsertQuery.cpp | 678 ++++++------ src/Interpreters/InterpreterInsertQuery.h | 17 +- src/Interpreters/Squashing.cpp | 124 +-- src/Interpreters/Squashing.h | 46 +- src/Interpreters/SystemLog.cpp | 8 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Processors/Chunk.cpp | 20 +- src/Processors/Chunk.h | 58 +- .../PullingAsyncPipelineExecutor.cpp | 9 +- .../Executors/PullingPipelineExecutor.cpp | 9 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 4 +- src/Processors/IAccumulatingTransform.cpp | 5 +- .../FinishAggregatingInOrderAlgorithm.cpp | 10 +- .../Algorithms/MergeTreePartLevelInfo.h | 12 +- .../Algorithms/ReplacingSortedAlgorithm.cpp | 2 +- .../Algorithms/ReplacingSortedAlgorithm.h | 7 +- src/Processors/Merges/IMergingTransform.cpp | 2 +- src/Processors/Merges/IMergingTransform.h | 2 +- src/Processors/Sinks/RemoteSink.h | 2 +- src/Processors/Sinks/SinkToStorage.cpp | 5 +- src/Processors/Sinks/SinkToStorage.h | 5 +- src/Processors/Sources/BlocksSource.h | 5 +- src/Processors/Sources/RemoteSource.cpp | 2 +- .../Sources/SourceFromSingleChunk.cpp | 6 +- .../AggregatingInOrderTransform.cpp | 11 +- .../Transforms/AggregatingInOrderTransform.h | 5 +- .../Transforms/AggregatingTransform.cpp | 16 +- .../Transforms/AggregatingTransform.h | 3 +- .../Transforms/ApplySquashingTransform.h | 14 +- .../Transforms/CountingTransform.cpp | 5 +- .../DeduplicationTokenTransforms.cpp | 236 +++++ .../Transforms/DeduplicationTokenTransforms.h | 237 +++++ .../Transforms/ExpressionTransform.cpp | 2 + .../Transforms/JoiningTransform.cpp | 9 +- src/Processors/Transforms/JoiningTransform.h | 6 +- .../Transforms/MaterializingTransform.cpp | 1 + .../Transforms/MemoryBoundMerging.h | 6 +- ...gingAggregatedMemoryEfficientTransform.cpp | 36 +- ...ergingAggregatedMemoryEfficientTransform.h | 5 +- .../Transforms/MergingAggregatedTransform.cpp | 10 +- .../Transforms/PlanSquashingTransform.cpp | 15 +- .../Transforms/PlanSquashingTransform.h | 3 +- .../Transforms/SelectByIndicesTransform.h | 3 +- .../Transforms/SquashingTransform.cpp | 18 +- .../Transforms/TotalsHavingTransform.cpp | 6 +- .../Transforms/buildPushingToViewsChain.cpp | 127 ++- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- src/QueryPipeline/QueryPlanResourceHolder.cpp | 8 +- src/QueryPipeline/QueryPlanResourceHolder.h | 3 + src/Server/TCPHandler.cpp | 20 +- src/Storages/Distributed/DistributedSink.cpp | 20 +- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 9 +- src/Storages/Kafka/StorageKafka.cpp | 8 +- src/Storages/LiveView/LiveViewSink.h | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 18 +- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/MaterializedView/RefreshTask.cpp | 8 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/MergeTreeSelectProcessor.cpp | 6 +- .../MergeTree/MergeTreeSequentialSource.cpp | 5 +- src/Storages/MergeTree/MergeTreeSink.cpp | 68 +- src/Storages/MergeTree/MergeTreeSink.h | 3 +- src/Storages/MergeTree/MutateTask.cpp | 19 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 94 +- .../MergeTree/ReplicatedMergeTreeSink.h | 13 +- src/Storages/MessageQueueSink.cpp | 2 +- src/Storages/MessageQueueSink.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 8 +- .../StorageObjectStorageSink.cpp | 4 +- .../ObjectStorage/StorageObjectStorageSink.h | 2 +- .../StorageObjectStorageQueue.cpp | 8 +- src/Storages/PartitionedSink.cpp | 4 +- src/Storages/PartitionedSink.h | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 8 +- .../PostgreSQLReplicationHandler.cpp | 8 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 8 +- .../RocksDB/EmbeddedRocksDBBulkSink.cpp | 7 +- .../RocksDB/EmbeddedRocksDBBulkSink.h | 2 +- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 2 +- src/Storages/RocksDB/EmbeddedRocksDBSink.h | 2 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 3 +- src/Storages/StorageBuffer.cpp | 10 +- src/Storages/StorageDistributed.cpp | 8 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageKeeperMap.cpp | 9 +- src/Storages/StorageLog.cpp | 8 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 5 +- src/Storages/StorageMySQL.cpp | 4 +- src/Storages/StoragePostgreSQL.cpp | 4 +- src/Storages/StorageRedis.cpp | 9 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSet.cpp | 6 +- src/Storages/StorageStripeLog.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- src/Storages/StorageURL.h | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 59 +- src/Storages/WindowView/StorageWindowView.h | 2 +- .../test_force_deduplication/test.py | 73 +- ...view_and_deduplication_zookeeper.reference | 2 +- ...lized_view_and_deduplication_zookeeper.sql | 2 +- ...lized_view_and_too_many_parts_zookeeper.sh | 4 +- .../0_stateless/01275_parallel_mv.reference | 4 +- ...01927_query_views_log_current_database.sql | 1 + ...ication_token_materialized_views.reference | 14 +- ...deduplication_token_materialized_views.sql | 8 +- .../0_stateless/02125_query_views_log.sql | 2 +- ...02912_ingestion_mv_deduplication.reference | 5 +- .../02912_ingestion_mv_deduplication.sql | 5 +- .../0_stateless/03008_deduplication.python | 657 ++++++++++++ ...08_deduplication_cases_from_docs.reference | 41 + .../03008_deduplication_cases_from_docs.sql | 331 ++++++ ...on_insert_into_partitioned_table.reference | 35 + ...lication_insert_into_partitioned_table.sql | 83 ++ ...ert_several_blocks_nonreplicated.reference | 962 ++++++++++++++++++ ...ion_insert_several_blocks_nonreplicated.sh | 59 ++ ...insert_several_blocks_replicated.reference | 962 ++++++++++++++++++ ...cation_insert_several_blocks_replicated.sh | 59 ++ ...tes_several_blocks_nonreplicated.reference | 962 ++++++++++++++++++ ..._generates_several_blocks_nonreplicated.sh | 59 ++ ...erates_several_blocks_replicated.reference | 962 ++++++++++++++++++ ..._mv_generates_several_blocks_replicated.sh | 59 ++ ..._mv_into_one_table_nonreplicated.reference | 706 +++++++++++++ ...several_mv_into_one_table_nonreplicated.sh | 59 ++ ...ral_mv_into_one_table_replicated.reference | 706 +++++++++++++ ...on_several_mv_into_one_table_replicated.sh | 59 ++ .../03035_max_insert_threads_support.sh | 2 +- 138 files changed, 8638 insertions(+), 857 deletions(-) create mode 100644 src/Common/CollectionOfDerived.h create mode 100644 src/Processors/Transforms/DeduplicationTokenTransforms.cpp create mode 100644 src/Processors/Transforms/DeduplicationTokenTransforms.h create mode 100644 tests/queries/0_stateless/03008_deduplication.python create mode 100644 tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference create mode 100644 tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql create mode 100644 tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference create mode 100644 tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql create mode 100644 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh create mode 100644 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh create mode 100644 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh create mode 100644 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh create mode 100644 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh create mode 100644 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 90ef974010c..ded56b60e64 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1093,4 +1093,10 @@ void ColumnObject::finalize() checkObjectHasNoAmbiguosPaths(getKeys()); } +void ColumnObject::updateHashFast(SipHash & hash) const +{ + for (const auto & entry : subcolumns) + for (auto & part : entry->data.data) + part->updateHashFast(hash); +} } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index e2936b27994..b1b8827622f 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -242,7 +242,7 @@ public: const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } - void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } + void updateHashFast(SipHash & hash) const override; void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h new file mode 100644 index 00000000000..97c0c3fbc06 --- /dev/null +++ b/src/Common/CollectionOfDerived.h @@ -0,0 +1,184 @@ +#pragma once + +#include + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/* This is a collections of objects derived from ItemBase. +* Collection contains no more than one instance for each derived type. +* The derived type is used to access the instance. +*/ + +template +class CollectionOfDerivedItems +{ +public: + using Self = CollectionOfDerivedItems; + using ItemPtr = std::shared_ptr; + +private: + struct Rec + { + std::type_index type_idx; + ItemPtr ptr; + + bool operator<(const Rec & other) const + { + return type_idx < other.type_idx; + } + + bool operator<(const std::type_index & value) const + { + return type_idx < value; + } + + bool operator==(const Rec & other) const + { + return type_idx == other.type_idx; + } + }; + using Records = std::vector; + +public: + void swap(Self & other) noexcept + { + records.swap(other.records); + } + + void clear() + { + records.clear(); + } + + bool empty() const + { + return records.empty(); + } + + size_t size() const + { + return records.size(); + } + + Self clone() const + { + Self result; + result.records.reserve(records.size()); + for (const auto & rec : records) + result.records.emplace_back(rec.type_idx, rec.ptr->clone()); + return result; + } + + void append(Self && other) + { + auto middle_idx = records.size(); + std::move(other.records.begin(), other.records.end(), std::back_inserter(records)); + std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end()); + chassert(isUniqTypes()); + } + + template + void add(std::shared_ptr info) + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + return addImpl(std::type_index(typeid(T)), std::move(info)); + } + + template + std::shared_ptr get() const + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + auto it = getImpl(std::type_index(typeid(T))); + if (it == records.cend()) + return nullptr; + auto cast = std::dynamic_pointer_cast(it->ptr); + chassert(cast); + return cast; + } + + template + std::shared_ptr extract() + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + auto it = getImpl(std::type_index(typeid(T))); + if (it == records.cend()) + return nullptr; + auto cast = std::dynamic_pointer_cast(it->ptr); + chassert(cast); + + records.erase(it); + return cast; + } + + std::string debug() const + { + std::string result; + + for (auto & rec : records) + { + result.append(rec.type_idx.name()); + result.append(" "); + } + + return result; + } + +private: + bool isUniqTypes() const + { + auto uniq_it = std::adjacent_find(records.begin(), records.end()); + + return uniq_it == records.end(); + } + + void addImpl(std::type_index type_idx, ItemPtr item) + { + auto it = std::lower_bound(records.begin(), records.end(), type_idx); + + if (it == records.end()) + { + records.emplace_back(type_idx, item); + return; + } + + if (it->type_idx == type_idx) + throw Exception(ErrorCodes::LOGICAL_ERROR, "inserted items must be unique by their type, type {} is inserted twice", type_idx.name()); + + + records.emplace(it, type_idx, item); + + chassert(isUniqTypes()); + } + + Records::const_iterator getImpl(std::type_index type_idx) const + { + auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx); + + if (it == records.cend()) + return records.cend(); + + if (it->type_idx != type_idx) + return records.cend(); + + return it; + } + + Records records; +}; + +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5903dbd32eb..81d0aa0c51d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -36,7 +36,7 @@ class IColumn; M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ - M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ + M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \ M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \ M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \ M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \ @@ -634,9 +634,8 @@ class IColumn; M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \ - M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ + M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ - M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ @@ -953,6 +952,7 @@ class IColumn; #define OBSOLETE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ + MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 1) \ MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \ MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \ MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index d72f3d81549..dd1166a9228 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -301,7 +301,13 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const auto & insert_query = query->as(); insert_query.async_insert_flush = true; - InterpreterInsertQuery interpreter(query, query_context, query_context->getSettingsRef().insert_allow_materialized_columns); + InterpreterInsertQuery interpreter( + query, + query_context, + query_context->getSettingsRef().insert_allow_materialized_columns, + /* no_squash */ false, + /* no_destination */ false, + /* async_insert */ false); auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -781,7 +787,12 @@ try try { interpreter = std::make_unique( - key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); + key.query, + insert_context, + key.settings.insert_allow_materialized_columns, + false, + false, + true); pipeline = interpreter->execute().pipeline; chassert(pipeline.pushing()); @@ -1000,7 +1011,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( } Chunk chunk(executor.getResultColumns(), total_rows); - chunk.setChunkInfo(std::move(chunk_info)); + chunk.getChunkInfos().add(std::move(chunk_info)); return chunk; } @@ -1052,7 +1063,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( } Chunk chunk(std::move(result_columns), total_rows); - chunk.setChunkInfo(std::move(chunk_info)); + chunk.getChunkInfos().add(std::move(chunk_info)); return chunk; } diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 4a84a7bf570..81bb6290acb 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -22,6 +23,7 @@ #include #include +#include #include #include #include @@ -91,7 +93,7 @@ Chunk getChunkFromCheckResult(const String & database, const String & table, con return Chunk(std::move(columns), 1); } -class TableCheckTask : public ChunkInfo +class TableCheckTask : public ChunkInfoCloneable { public: TableCheckTask(StorageID table_id, const std::variant & partition_or_part, ContextPtr context) @@ -110,6 +112,12 @@ public: context->checkAccess(AccessType::SHOW_TABLES, table_->getStorageID()); } + TableCheckTask(const TableCheckTask & other) + : table(other.table) + , check_data_tasks(other.check_data_tasks) + , is_finished(other.is_finished.load()) + {} + std::optional checkNext() const { if (isFinished()) @@ -121,8 +129,8 @@ public: std::this_thread::sleep_for(sleep_time); }); - IStorage::DataValidationTasksPtr check_data_tasks_ = check_data_tasks; - auto result = table->checkDataNext(check_data_tasks_); + IStorage::DataValidationTasksPtr tmp = check_data_tasks; + auto result = table->checkDataNext(tmp); is_finished = !result.has_value(); return result; } @@ -180,7 +188,7 @@ protected: /// source should return at least one row to start pipeline result.addColumn(ColumnUInt8::create(1, 1)); /// actual data stored in chunk info - result.setChunkInfo(std::move(current_check_task)); + result.getChunkInfos().add(std::move(current_check_task)); return result; } @@ -280,7 +288,7 @@ public: protected: void transform(Chunk & chunk) override { - auto table_check_task = std::dynamic_pointer_cast(chunk.getChunkInfo()); + auto table_check_task = chunk.getChunkInfos().get(); auto check_result = table_check_task->checkNext(); if (!check_result) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 0ee2bb6c0e9..ee191c02ff8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1776,8 +1776,13 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) else insert->select = create.select->clone(); - return InterpreterInsertQuery(insert, getContext(), - getContext()->getSettingsRef().insert_allow_materialized_columns).execute(); + return InterpreterInsertQuery( + insert, + getContext(), + getContext()->getSettingsRef().insert_allow_materialized_columns, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false).execute(); } return {}; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 7c7b4b3f95a..26b7e074fdf 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -534,7 +534,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() } else if (dynamic_cast(ast.getExplainedQuery().get())) { - InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); + InterpreterInsertQuery insert( + ast.getExplainedQuery(), + getContext(), + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto io = insert.execute(); printPipeline(io.pipeline.getProcessors(), buf); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f396db70d21..2cbfc55d008 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +40,7 @@ #include #include #include +#include "base/defines.h" namespace ProfileEvents @@ -394,28 +397,358 @@ Chain InterpreterInsertQuery::buildPreSinkChain( return out; } +std::pair, std::vector> InterpreterInsertQuery::buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block) +{ + chassert(presink_streams > 0); + chassert(sink_streams > 0); + + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + + std::vector sink_chains; + std::vector presink_chains; + + for (size_t i = 0; i < sink_streams; ++i) + { + auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, + running_group, /* elapsed_counter_ms= */ nullptr); + + sink_chains.emplace_back(std::move(out)); + } + + for (size_t i = 0; i < presink_streams; ++i) + { + auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); + presink_chains.emplace_back(std::move(out)); + } + + return {std::move(presink_chains), std::move(sink_chains)}; +} + + +QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table) +{ + const Settings & settings = getContext()->getSettingsRef(); + + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); + + bool is_trivial_insert_select = false; + + if (settings.optimize_trivial_insert_select) + { + const auto & select_query = query.select->as(); + const auto & selects = select_query.list_of_selects->children; + const auto & union_modes = select_query.list_of_modes; + + /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries + const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; + + is_trivial_insert_select = + std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) + && std::all_of(selects.begin(), selects.end(), isTrivialSelect); + } + + ContextPtr select_context = getContext(); + + if (is_trivial_insert_select) + { + /** When doing trivial INSERT INTO ... SELECT ... FROM table, + * don't need to process SELECT with more than max_insert_threads + * and it's reasonable to set block size for SELECT to the desired block size for INSERT + * to avoid unnecessary squashing. + */ + + Settings new_settings = select_context->getSettings(); + + new_settings.max_threads = std::max(1, settings.max_insert_threads); + + if (table->prefersLargeBlocks()) + { + if (settings.min_insert_block_size_rows) + new_settings.max_block_size = settings.min_insert_block_size_rows; + if (settings.min_insert_block_size_bytes) + new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; + } + + auto context_for_trivial_select = Context::createCopy(context); + context_for_trivial_select->setSettings(new_settings); + context_for_trivial_select->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); + + select_context = context_for_trivial_select; + } + + QueryPipelineBuilder pipeline; + + { + auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); + + if (settings.allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, select_context, select_query_options); + pipeline = interpreter_select_analyzer.buildQueryPipeline(); + } + else + { + InterpreterSelectWithUnionQuery interpreter_select(query.select, select_context, select_query_options); + pipeline = interpreter_select.buildQueryPipeline(); + } + } + + pipeline.dropTotalsAndExtremes(); + + /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. + if (getContext()->getSettingsRef().insert_null_as_default) + { + const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); + const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); + const auto & output_columns = metadata_snapshot->getColumns(); + + if (input_columns.size() == query_columns.size()) + { + for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) + { + /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with + /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. + if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) + && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) + && !isVariant(query_columns[col_idx].type) + && !isDynamic(query_columns[col_idx].type) + && output_columns.has(query_columns[col_idx].name)) + { + query_sample_block.setColumn( + col_idx, + ColumnWithTypeAndName( + makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), + makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), + query_columns[col_idx].name)); + } + } + } + } + + auto actions_dag = ActionsDAG::makeConvertingActions( + pipeline.getHeader().getColumnsWithTypeAndName(), + query_sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header, actions); + }); + + /// We need to convert Sparse columns to full, because it's destination storage + /// may not support it or may have different settings for applying Sparse serialization. + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + auto context_ptr = getContext(); + auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + + return counting; + }); + + size_t num_select_threads = pipeline.getNumThreads(); + + pipeline.resize(1); + + if (shouldAddSquashingFroStorage(table)) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, + table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); + }); + } + + pipeline.addSimpleTransform([&](const Block &in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + + if (!settings.insert_deduplication_token.value.empty()) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(settings.insert_deduplication_token.value, in_header); + }); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + } + + /// Number of streams works like this: + /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever + /// InterpreterSelectQuery ends up with. + /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. + /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. + /// * If the table supports parallel inserts, use max_insert_threads for writing to IStorage. + /// Otherwise ResizeProcessor them down to 1 stream. + + size_t presink_streams_size = std::max(settings.max_insert_threads, pipeline.getNumStreams()); + + size_t sink_streams_size = table->supportsParallelInsert() ? std::max(1, settings.max_insert_threads) : 1; + + if (!settings.parallel_view_processing) + { + auto table_id = table->getStorageID(); + auto views = DatabaseCatalog::instance().getDependentViews(table_id); + + if (table->isView() || !views.empty()) + sink_streams_size = 1; + } + + auto [presink_chains, sink_chains] = buildPreAndSinkChains( + presink_streams_size, sink_streams_size, + table, metadata_snapshot, query_sample_block); + + pipeline.resize(presink_chains.size()); + + if (shouldAddSquashingFroStorage(table)) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, + table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); + }); + } + + for (auto & chain : presink_chains) + pipeline.addResources(chain.detachResources()); + pipeline.addChains(std::move(presink_chains)); + + pipeline.resize(sink_streams_size); + + for (auto & chain : sink_chains) + pipeline.addResources(chain.detachResources()); + pipeline.addChains(std::move(sink_chains)); + + if (!settings.parallel_view_processing) + { + /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. + if (pipeline.getNumThreads() > num_select_threads) + pipeline.setMaxThreads(num_select_threads); + } + else if (pipeline.getNumThreads() < settings.max_threads) + { + /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, + /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. + /// + /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. + pipeline.setMaxThreads(settings.max_threads); + } + + pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr + { + return std::make_shared(cur_header); + }); + + return QueryPipelineBuilder::getPipeline(std::move(pipeline)); +} + + +QueryPipeline InterpreterInsertQuery::buildInsertPipeline(ASTInsertQuery & query, StoragePtr table) +{ + const Settings & settings = getContext()->getSettingsRef(); + + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); + + Chain chain; + + { + auto [presink_chains, sink_chains] = buildPreAndSinkChains( + /* presink_streams */1, /* sink_streams */1, + table, metadata_snapshot, query_sample_block); + + chain = std::move(presink_chains.front()); + chain.appendChain(std::move(sink_chains.front())); + } + + if (!settings.insert_deduplication_token.value.empty()) + { + chain.addSource(std::make_shared(chain.getInputHeader())); + chain.addSource(std::make_shared(settings.insert_deduplication_token.value, chain.getInputHeader())); + } + + chain.addSource(std::make_shared(chain.getInputHeader())); + + if (shouldAddSquashingFroStorage(table)) + { + bool table_prefers_large_blocks = table->prefersLargeBlocks(); + + auto squashing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(squashing)); + + auto balancing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(balancing)); + } + + auto context_ptr = getContext(); + auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + chain.addSource(std::move(counting)); + + QueryPipeline pipeline = QueryPipeline(std::move(chain)); + + pipeline.setNumThreads(std::min(pipeline.getNumThreads(), settings.max_threads)); + pipeline.setConcurrencyControl(settings.use_concurrency_control); + + if (query.hasInlinedData() && !async_insert) + { + /// can execute without additional data + auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); + for (auto && buffer : owned_buffers) + format->addBuffer(std::move(buffer)); + + auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); + pipeline.complete(std::move(pipe)); + } + + return pipeline; +} + + BlockIO InterpreterInsertQuery::execute() { const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); - QueryPipelineBuilder pipeline; - std::optional distributed_pipeline; - QueryPlanResourceHolder resources; StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); - StoragePtr inner_table; - if (const auto * mv = dynamic_cast(table.get())) - inner_table = mv->getTargetTable(); - if (query.partition_by && !table->supportsPartitionBy()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage"); auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); /// For table functions we check access while executing @@ -423,320 +756,43 @@ BlockIO InterpreterInsertQuery::execute() if (!query.table_function) getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); - if (query.select && settings.parallel_distributed_insert_select) - // Distributed INSERT SELECT - distributed_pipeline = table->distributedWrite(query, getContext()); - - std::vector presink_chains; - std::vector sink_chains; - if (!distributed_pipeline) + if (!allow_materialized) { - /// Number of streams works like this: - /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever - /// InterpreterSelectQuery ends up with. - /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. - /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. - /// * If the table supports parallel inserts, use the same streams for writing to IStorage. - /// Otherwise ResizeProcessor them down to 1 stream. - /// * If it's not an INSERT SELECT, forget all that and use one stream. - size_t pre_streams_size = 1; - size_t sink_streams_size = 1; - - if (query.select) - { - bool is_trivial_insert_select = false; - - if (settings.optimize_trivial_insert_select) - { - const auto & select_query = query.select->as(); - const auto & selects = select_query.list_of_selects->children; - const auto & union_modes = select_query.list_of_modes; - - /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries - const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; - - is_trivial_insert_select = - std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) - && std::all_of(selects.begin(), selects.end(), isTrivialSelect); - } - - if (is_trivial_insert_select) - { - /** When doing trivial INSERT INTO ... SELECT ... FROM table, - * don't need to process SELECT with more than max_insert_threads - * and it's reasonable to set block size for SELECT to the desired block size for INSERT - * to avoid unnecessary squashing. - */ - - Settings new_settings = getContext()->getSettings(); - - new_settings.max_threads = std::max(1, settings.max_insert_threads); - - if (table->prefersLargeBlocks()) - { - if (settings.min_insert_block_size_rows) - new_settings.max_block_size = settings.min_insert_block_size_rows; - if (settings.min_insert_block_size_bytes) - new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; - } - - auto new_context = Context::createCopy(context); - new_context->setSettings(new_settings); - new_context->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); - - auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - - if (settings.allow_experimental_analyzer) - { - InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, new_context, select_query_options); - pipeline = interpreter_select_analyzer.buildQueryPipeline(); - } - else - { - InterpreterSelectWithUnionQuery interpreter_select(query.select, new_context, select_query_options); - pipeline = interpreter_select.buildQueryPipeline(); - } - } - else - { - /// Passing 1 as subquery_depth will disable limiting size of intermediate result. - auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - - if (settings.allow_experimental_analyzer) - { - InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, getContext(), select_query_options); - pipeline = interpreter_select_analyzer.buildQueryPipeline(); - } - else - { - InterpreterSelectWithUnionQuery interpreter_select(query.select, getContext(), select_query_options); - pipeline = interpreter_select.buildQueryPipeline(); - } - } - - pipeline.dropTotalsAndExtremes(); - - if (settings.max_insert_threads > 1) - { - auto table_id = table->getStorageID(); - auto views = DatabaseCatalog::instance().getDependentViews(table_id); - - /// It breaks some views-related tests and we have dedicated `parallel_view_processing` for views, so let's just skip them. - /// Also it doesn't make sense to reshuffle data if storage doesn't support parallel inserts. - const bool resize_to_max_insert_threads = !table->isView() && views.empty() && table->supportsParallelInsert(); - pre_streams_size = resize_to_max_insert_threads ? settings.max_insert_threads - : std::min(settings.max_insert_threads, pipeline.getNumStreams()); - - /// Deduplication when passing insert_deduplication_token breaks if using more than one thread - if (!settings.insert_deduplication_token.toString().empty()) - { - LOG_DEBUG( - getLogger("InsertQuery"), - "Insert-select query using insert_deduplication_token, setting streams to 1 to avoid deduplication issues"); - pre_streams_size = 1; - } - - if (table->supportsParallelInsert()) - sink_streams_size = pre_streams_size; - } - - pipeline.resize(pre_streams_size); - - /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. - if (getContext()->getSettingsRef().insert_null_as_default) - { - const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); - const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); - const auto & output_columns = metadata_snapshot->getColumns(); - - if (input_columns.size() == query_columns.size()) - { - for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) - { - /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with - /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. - if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) - && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) - && !isVariant(query_columns[col_idx].type) - && !isDynamic(query_columns[col_idx].type) - && output_columns.has(query_columns[col_idx].name)) - query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), query_columns[col_idx].name)); - } - } - } - } - - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(getContext()); - for (size_t i = 0; i < sink_streams_size; ++i) - { - auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, - running_group, /* elapsed_counter_ms= */ nullptr); - sink_chains.emplace_back(std::move(out)); - } - for (size_t i = 0; i < pre_streams_size; ++i) - { - auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); - presink_chains.emplace_back(std::move(out)); - } + for (const auto & column : metadata_snapshot->getColumns()) + if (column.default_desc.kind == ColumnDefaultKind::Materialized && query_sample_block.has(column.name)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); } BlockIO res; - /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? - if (distributed_pipeline) + if (query.select) { - res.pipeline = std::move(*distributed_pipeline); - } - else if (query.select) - { - const auto & header = presink_chains.at(0).getInputHeader(); - auto actions_dag = ActionsDAG::makeConvertingActions( - pipeline.getHeader().getColumnsWithTypeAndName(), - header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + if (settings.parallel_distributed_insert_select) { - return std::make_shared(in_header, actions); - }); - - /// We need to convert Sparse columns to full, because it's destination storage - /// may not support it or may have different settings for applying Sparse serialization. - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(in_header); - }); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - auto context_ptr = getContext(); - auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - - return counting; - }); - - if (shouldAddSquashingFroStorage(table)) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - size_t threads = presink_chains.size(); - - pipeline.resize(1); - - pipeline.addTransform(std::make_shared( - header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - - pipeline.resize(threads); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + auto distributed = table->distributedWrite(query, getContext()); + if (distributed) { - return std::make_shared( - in_header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - }); + res.pipeline = std::move(*distributed); + } + else + { + res.pipeline = buildInsertSelectPipeline(query, table); + } } - - size_t num_select_threads = pipeline.getNumThreads(); - - for (auto & chain : presink_chains) - resources = chain.detachResources(); - for (auto & chain : sink_chains) - resources = chain.detachResources(); - - pipeline.addChains(std::move(presink_chains)); - pipeline.resize(sink_chains.size()); - pipeline.addChains(std::move(sink_chains)); - - if (!settings.parallel_view_processing) + else { - /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. - if (pipeline.getNumThreads() > num_select_threads) - pipeline.setMaxThreads(num_select_threads); + res.pipeline = buildInsertSelectPipeline(query, table); } - else if (pipeline.getNumThreads() < settings.max_threads) - { - /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, - /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. - /// - /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. - pipeline.setMaxThreads(settings.max_threads); - } - - pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr - { - return std::make_shared(cur_header); - }); - - if (!allow_materialized) - { - for (const auto & column : metadata_snapshot->getColumns()) - if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); - } - - res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } else { - auto & chain = presink_chains.at(0); - chain.appendChain(std::move(sink_chains.at(0))); - - if (shouldAddSquashingFroStorage(table)) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - auto squashing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(squashing)); - - auto balancing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(balancing)); - } - - auto context_ptr = getContext(); - auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - chain.addSource(std::move(counting)); - - res.pipeline = QueryPipeline(std::move(presink_chains[0])); - res.pipeline.setNumThreads(std::min(res.pipeline.getNumThreads(), settings.max_threads)); - res.pipeline.setConcurrencyControl(settings.use_concurrency_control); - - if (query.hasInlinedData() && !async_insert) - { - /// can execute without additional data - auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); - for (auto && buffer : owned_buffers) - format->addBuffer(std::move(buffer)); - - auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); - res.pipeline.complete(std::move(pipe)); - } + res.pipeline = buildInsertPipeline(query, table); } - res.pipeline.addResources(std::move(resources)); - res.pipeline.addStorageHolder(table); - if (inner_table) - res.pipeline.addStorageHolder(inner_table); + + if (const auto * mv = dynamic_cast(table.get())) + res.pipeline.addStorageHolder(mv->getTargetTable()); return res; } @@ -757,17 +813,27 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, Cont } } + void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const { extendQueryLogElemImpl(elem, context_); } + void registerInterpreterInsertQuery(InterpreterFactory & factory) { auto create_fn = [] (const InterpreterFactory::Arguments & args) { - return std::make_unique(args.query, args.context, args.allow_materialized); + return std::make_unique( + args.query, + args.context, + args.allow_materialized, + /* no_squash */false, + /* no_destination */false, + /* async_insert */false); }; factory.registerInterpreter("InterpreterInsertQuery", create_fn); } + + } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index bf73fb2a319..894c7c42144 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -23,10 +23,10 @@ public: InterpreterInsertQuery( const ASTPtr & query_ptr_, ContextPtr context_, - bool allow_materialized_ = false, - bool no_squash_ = false, - bool no_destination_ = false, - bool async_insert_ = false); + bool allow_materialized_, + bool no_squash_, + bool no_destination, + bool async_insert_); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -73,12 +73,17 @@ private: ASTPtr query_ptr; const bool allow_materialized; - const bool no_squash; - const bool no_destination; + bool no_squash = false; + bool no_destination = false; const bool async_insert; std::vector> owned_buffers; + std::pair, std::vector> buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block); + + QueryPipeline buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table); + QueryPipeline buildInsertPipeline(ASTInsertQuery & query, StoragePtr table); + Chain buildSink( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index f8b6a6542cc..25434d1103e 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -11,24 +12,33 @@ namespace ErrorCodes } Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : header(header_) - , min_block_size_rows(min_block_size_rows_) + : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) + , header(header_) { } Chunk Squashing::flush() { - return convertToChunk(std::move(chunks_to_merge_vec)); + if (!accumulated) + return {}; + + auto result = convertToChunk(accumulated.extract()); + chassert(result); + return result; } Chunk Squashing::squash(Chunk && input_chunk) { - if (!input_chunk.hasChunkInfo()) + if (!input_chunk) return Chunk(); - const auto *info = getInfoFromChunk(input_chunk); - return squash(info->chunks); + auto squash_info = input_chunk.getChunkInfos().extract(); + + if (!squash_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); + + return squash(std::move(squash_info->chunks), std::move(input_chunk.getChunkInfos())); } Chunk Squashing::add(Chunk && input_chunk) @@ -37,48 +47,37 @@ Chunk Squashing::add(Chunk && input_chunk) return {}; /// Just read block is already enough. - if (isEnoughSize(input_chunk.getNumRows(), input_chunk.bytes())) + if (isEnoughSize(input_chunk)) { /// If no accumulated data, return just read block. - if (chunks_to_merge_vec.empty()) + if (!accumulated) { - chunks_to_merge_vec.push_back(std::move(input_chunk)); - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - return res_chunk; + accumulated.add(std::move(input_chunk)); + return convertToChunk(accumulated.extract()); } /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(accumulated.extract()); + accumulated.add(std::move(input_chunk)); return res_chunk; } /// Accumulated block is already enough. - if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) + if (isEnoughSize()) { /// Return accumulated data and place new block to accumulated data. - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(accumulated.extract()); + accumulated.add(std::move(input_chunk)); return res_chunk; } /// Pushing data into accumulating vector - expandCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + accumulated.add(std::move(input_chunk)); /// If accumulated data is big enough, we send it - if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) - { - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - changeCurrentSize(0, 0); - chunks_to_merge_vec.clear(); - return res_chunk; - } + if (isEnoughSize()) + return convertToChunk(accumulated.extract()); + return {}; } @@ -90,14 +89,15 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const auto info = std::make_shared(); info->chunks = std::move(chunks); - chunks.clear(); - - return Chunk(header.cloneEmptyColumns(), 0, info); + // It is imortant that chunk is not empty, it has to have columns even if they are empty + auto aggr_chunk = Chunk(header.getColumns(), 0); + aggr_chunk.getChunkInfos().add(std::move(info)); + chassert(aggr_chunk); + return aggr_chunk; } -Chunk Squashing::squash(std::vector & input_chunks) +Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos) { - Chunk accumulated_chunk; std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) @@ -119,35 +119,17 @@ Chunk Squashing::squash(std::vector & input_chunks) for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) { const auto source_column = columns[j]; - mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); } } - accumulated_chunk.setColumns(std::move(mutable_columns), rows); - return accumulated_chunk; -} -const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk) -{ - const auto& info = chunk.getChunkInfo(); - const auto * agg_info = typeid_cast(info.get()); + Chunk result; + result.setColumns(std::move(mutable_columns), rows); + result.setChunkInfos(infos); + result.getChunkInfos().append(std::move(input_chunks.back().getChunkInfos())); - if (!agg_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); - - return agg_info; -} - -void Squashing::expandCurrentSize(size_t rows, size_t bytes) -{ - accumulated_size.rows += rows; - accumulated_size.bytes += bytes; -} - -void Squashing::changeCurrentSize(size_t rows, size_t bytes) -{ - accumulated_size.rows = rows; - accumulated_size.bytes = bytes; + chassert(result); + return result; } bool Squashing::isEnoughSize(size_t rows, size_t bytes) const @@ -156,4 +138,28 @@ bool Squashing::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } + +bool Squashing::isEnoughSize() const +{ + return isEnoughSize(accumulated.getRows(), accumulated.getBytes()); +}; + +bool Squashing::isEnoughSize(const Chunk & chunk) const +{ + return isEnoughSize(chunk.getNumRows(), chunk.bytes()); +} + +void Squashing::CurrentSize::add(Chunk && chunk) +{ + rows += chunk.getNumRows(); + bytes += chunk.bytes(); + chunks.push_back(std::move(chunk)); +} + +std::vector Squashing::CurrentSize::extract() +{ + auto result = std::move(chunks); + *this = {}; + return result; +} } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d76cca60e41..64a9768a71f 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -8,9 +8,18 @@ namespace DB { -struct ChunksToSquash : public ChunkInfo +class ChunksToSquash : public ChunkInfoCloneable { - mutable std::vector chunks = {}; +public: + ChunksToSquash() = default; + ChunksToSquash(const ChunksToSquash & other) + { + chunks.reserve(other.chunks.size()); + for (const auto & chunk: other.chunks) + chunks.push_back(chunk.clone()); + } + + std::vector chunks = {}; }; /** Merging consecutive passed blocks to specified minimum size. @@ -36,32 +45,35 @@ public: static Chunk squash(Chunk && input_chunk); Chunk flush(); - bool isDataLeft() - { - return !chunks_to_merge_vec.empty(); - } + void setHeader(Block header_) { header = std::move(header_); } + const Block & getHeader() const { return header; } - Block header; private: - struct CurrentSize + class CurrentSize { + std::vector chunks = {}; size_t rows = 0; size_t bytes = 0; + + public: + explicit operator bool () const { return !chunks.empty(); } + size_t getRows() const { return rows; } + size_t getBytes() const { return bytes; } + void add(Chunk && chunk); + std::vector extract(); }; - std::vector chunks_to_merge_vec = {}; - size_t min_block_size_rows; - size_t min_block_size_bytes; + const size_t min_block_size_rows; + const size_t min_block_size_bytes; + Block header; - CurrentSize accumulated_size; + CurrentSize accumulated; - static const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); + static Chunk squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos); - static Chunk squash(std::vector & input_chunks); - - void expandCurrentSize(size_t rows, size_t bytes); - void changeCurrentSize(size_t rows, size_t bytes); + bool isEnoughSize() const; bool isEnoughSize(size_t rows, size_t bytes) const; + bool isEnoughSize(const Chunk & chunk) const; Chunk convertToChunk(std::vector && chunks) const; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 557065b23ff..f386e157b14 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -538,7 +538,13 @@ void SystemLog::flushImpl(const std::vector & to_flush, insert_context->makeQueryContext(); addSettingsForQuery(insert_context, IAST::QueryKind::Insert); - InterpreterInsertQuery interpreter(query_ptr, insert_context); + InterpreterInsertQuery interpreter( + query_ptr, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); BlockIO io = interpreter.execute(); PushingPipelineExecutor executor(io.pipeline); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index a3c5a7ed3ed..6ce6f5e454e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1188,7 +1188,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } } - /// Check for dynamic subcolums in unknown required columns. + /// Check for dynamic subcolumns in unknown required columns. if (!unknown_required_source_columns.empty()) { for (const NameAndTypePair & pair : source_columns_ordinary) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 5f6cf2f7230..4466be5b3a7 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -19,14 +19,6 @@ Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns checkNumRowsIsConsistent(); } -Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(std::move(columns_)) - , num_rows(num_rows_) - , chunk_info(std::move(chunk_info_)) -{ - checkNumRowsIsConsistent(); -} - static Columns unmuteColumns(MutableColumns && mutable_columns) { Columns columns; @@ -43,17 +35,11 @@ Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_) checkNumRowsIsConsistent(); } -Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(unmuteColumns(std::move(columns_))) - , num_rows(num_rows_) - , chunk_info(std::move(chunk_info_)) -{ - checkNumRowsIsConsistent(); -} - Chunk Chunk::clone() const { - return Chunk(getColumns(), getNumRows(), chunk_info); + auto tmp = Chunk(getColumns(), getNumRows()); + tmp.setChunkInfos(chunk_infos.clone()); + return tmp; } void Chunk::setColumns(Columns columns_, UInt64 num_rows_) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 4f753798eaa..1348966c0d3 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -1,7 +1,9 @@ #pragma once +#include #include -#include + +#include namespace DB { @@ -9,11 +11,29 @@ namespace DB class ChunkInfo { public: - virtual ~ChunkInfo() = default; + using Ptr = std::shared_ptr; + ChunkInfo() = default; + ChunkInfo(const ChunkInfo&) = default; + ChunkInfo(ChunkInfo&&) = default; + + virtual Ptr clone() const = 0; + virtual ~ChunkInfo() = default; }; -using ChunkInfoPtr = std::shared_ptr; + +template +class ChunkInfoCloneable : public ChunkInfo +{ +public: + ChunkInfoCloneable() = default; + ChunkInfoCloneable(const ChunkInfoCloneable & other) = default; + + Ptr clone() const override + { + return std::static_pointer_cast(std::make_shared(*static_cast(this))); + } +}; /** * Chunk is a list of columns with the same length. @@ -32,26 +52,26 @@ using ChunkInfoPtr = std::shared_ptr; class Chunk { public: + using ChunkInfoCollection = CollectionOfDerivedItems; + Chunk() = default; Chunk(const Chunk & other) = delete; Chunk(Chunk && other) noexcept : columns(std::move(other.columns)) , num_rows(other.num_rows) - , chunk_info(std::move(other.chunk_info)) + , chunk_infos(std::move(other.chunk_infos)) { other.num_rows = 0; } Chunk(Columns columns_, UInt64 num_rows_); - Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk(MutableColumns columns_, UInt64 num_rows_); - Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk & operator=(const Chunk & other) = delete; Chunk & operator=(Chunk && other) noexcept { columns = std::move(other.columns); - chunk_info = std::move(other.chunk_info); + chunk_infos = std::move(other.chunk_infos); num_rows = other.num_rows; other.num_rows = 0; return *this; @@ -62,15 +82,15 @@ public: void swap(Chunk & other) noexcept { columns.swap(other.columns); - chunk_info.swap(other.chunk_info); std::swap(num_rows, other.num_rows); + chunk_infos.swap(other.chunk_infos); } void clear() { num_rows = 0; columns.clear(); - chunk_info.reset(); + chunk_infos.clear(); } const Columns & getColumns() const { return columns; } @@ -81,9 +101,9 @@ public: /** Get empty columns with the same types as in block. */ MutableColumns cloneEmptyColumns() const; - const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } - bool hasChunkInfo() const { return chunk_info != nullptr; } - void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } + ChunkInfoCollection & getChunkInfos() { return chunk_infos; } + const ChunkInfoCollection & getChunkInfos() const { return chunk_infos; } + void setChunkInfos(ChunkInfoCollection chunk_infos_) { chunk_infos = std::move(chunk_infos_); } UInt64 getNumRows() const { return num_rows; } UInt64 getNumColumns() const { return columns.size(); } @@ -107,7 +127,7 @@ public: private: Columns columns; UInt64 num_rows = 0; - ChunkInfoPtr chunk_info; + ChunkInfoCollection chunk_infos; void checkNumRowsIsConsistent(); }; @@ -117,11 +137,15 @@ using Chunks = std::vector; /// AsyncInsert needs two kinds of information: /// - offsets of different sub-chunks /// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`. -class AsyncInsertInfo : public ChunkInfo +class AsyncInsertInfo : public ChunkInfoCloneable { public: AsyncInsertInfo() = default; - explicit AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) : offsets(offsets_), tokens(tokens_) {} + AsyncInsertInfo(const AsyncInsertInfo & other) = default; + AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) + : offsets(offsets_) + , tokens(tokens_) + {} std::vector offsets; std::vector tokens; @@ -130,9 +154,11 @@ public: using AsyncInsertInfoPtr = std::shared_ptr; /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. -class ChunkMissingValues : public ChunkInfo +class ChunkMissingValues : public ChunkInfoCloneable { public: + ChunkMissingValues(const ChunkMissingValues & other) = default; + using RowsBitMask = std::vector; /// a bit per row for a column const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index d27002197d2..d9fab88fe1f 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -147,13 +147,10 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..25c15d40c9a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -73,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index a5d334f4f1d..9e499e2c400 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -179,7 +179,9 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count); Chunks piece; - piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo()); + piece.emplace_back(std::move(columns), count); + piece.back().setChunkInfos(concatenated.getChunkInfos()); + writeRowGroup(std::move(piece)); } } diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 4136fc5a5f2..46be6e74693 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -8,8 +8,9 @@ namespace ErrorCodes } IAccumulatingTransform::IAccumulatingTransform(Block input_header, Block output_header) - : IProcessor({std::move(input_header)}, {std::move(output_header)}), - input(inputs.front()), output(outputs.front()) + : IProcessor({std::move(input_header)}, {std::move(output_header)}) + , input(inputs.front()) + , output(outputs.front()) { } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 466adf93538..86675bcb237 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -53,13 +53,11 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num if (!input.chunk.hasRows()) return; - const auto & info = input.chunk.getChunkInfo(); - if (!info) + if (input.chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm"); Int64 allocated_bytes = 0; - /// Will be set by AggregatingInOrderTransform during local aggregation; will be nullptr during merging on initiator. - if (const auto * arenas_info = typeid_cast(info.get())) + if (auto arenas_info = input.chunk.getChunkInfos().get()) allocated_bytes = arenas_info->allocated_bytes; states[source_num] = State{input.chunk, description, allocated_bytes}; @@ -136,7 +134,7 @@ Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() info->chunk_num = chunk_num++; Chunk chunk; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); return chunk; } @@ -163,7 +161,7 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() chunks.emplace_back(std::move(new_columns), current_rows); } - chunks.back().setChunkInfo(std::make_shared()); + chunks.back().getChunkInfos().add(std::make_shared()); states[i].current_row = states[i].to_row; /// We assume that sizes in bytes of rows are almost the same. diff --git a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h index bcf4e759024..e4f22deec8d 100644 --- a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h +++ b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h @@ -6,18 +6,22 @@ namespace DB { /// To carry part level if chunk is produced by a merge tree source -class MergeTreePartLevelInfo : public ChunkInfo +class MergeTreePartLevelInfo : public ChunkInfoCloneable { public: MergeTreePartLevelInfo() = delete; - explicit MergeTreePartLevelInfo(ssize_t part_level) : origin_merge_tree_part_level(part_level) { } + explicit MergeTreePartLevelInfo(ssize_t part_level) + : origin_merge_tree_part_level(part_level) + { } + MergeTreePartLevelInfo(const MergeTreePartLevelInfo & other) = default; + size_t origin_merge_tree_part_level = 0; }; inline size_t getPartLevelFromChunk(const Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - if (const auto * part_level_info = typeid_cast(info.get())) + const auto part_level_info = chunk.getChunkInfos().get(); + if (part_level_info) return part_level_info->origin_merge_tree_part_level; return 0; } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 7b2c7d82a01..cd347d371d9 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes static IMergingAlgorithm::Status emitChunk(detail::SharedChunkPtr & chunk, bool finished = false) { - chunk->setChunkInfo(std::make_shared(std::move(chunk->replace_final_selection))); + chunk->getChunkInfos().add(std::make_shared(std::move(chunk->replace_final_selection))); return IMergingAlgorithm::Status(std::move(*chunk), finished); } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index a3ccccf0845..2f23f2a5c4d 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { @@ -14,11 +15,13 @@ namespace DB /** Use in skipping final to keep list of indices of selected row after merging final */ -struct ChunkSelectFinalIndices : public ChunkInfo +struct ChunkSelectFinalIndices : public ChunkInfoCloneable { + explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); + ChunkSelectFinalIndices(const ChunkSelectFinalIndices & other) = default; + const ColumnPtr column_holder; const ColumnUInt64 * select_final_indices = nullptr; - explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); }; /** Merges several sorted inputs into one. diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index fbb47969b2f..b1b0182a113 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -157,7 +157,7 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if ((state.output_chunk || state.output_chunk.hasChunkInfo()) && !is_port_full) + if ((state.output_chunk || !state.output_chunk.getChunkInfos().empty()) && !is_port_full) output.push(std::move(state.output_chunk)); if (!is_initialized) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index c218f622870..be629271736 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -129,7 +129,7 @@ public: IMergingAlgorithm::Status status = algorithm.merge(); - if ((status.chunk && status.chunk.hasRows()) || status.chunk.hasChunkInfo()) + if ((status.chunk && status.chunk.hasRows()) || !status.chunk.getChunkInfos().empty()) { // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); diff --git a/src/Processors/Sinks/RemoteSink.h b/src/Processors/Sinks/RemoteSink.h index 30cf958c072..c05cc1defcb 100644 --- a/src/Processors/Sinks/RemoteSink.h +++ b/src/Processors/Sinks/RemoteSink.h @@ -20,7 +20,7 @@ public: } String getName() const override { return "RemoteSink"; } - void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } + void consume (Chunk & chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.getColumns())); } void onFinish() override { RemoteInserter::onFinish(); } }; diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 5f9f9f9b1a1..36bb70f493f 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -15,9 +15,8 @@ void SinkToStorage::onConsume(Chunk chunk) */ Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); - consume(chunk.clone()); - if (!lastBlockIsDuplicate()) - cur_chunk = std::move(chunk); + consume(chunk); + cur_chunk = std::move(chunk); } SinkToStorage::GenerateResult SinkToStorage::onGenerate() diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 023bbd8b094..c728fa87b1e 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -18,8 +18,7 @@ public: void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } protected: - virtual void consume(Chunk chunk) = 0; - virtual bool lastBlockIsDuplicate() const { return false; } + virtual void consume(Chunk & chunk) = 0; private: std::vector table_locks; @@ -38,7 +37,7 @@ class NullSinkToStorage : public SinkToStorage public: using SinkToStorage::SinkToStorage; std::string getName() const override { return "NullSinkToStorage"; } - void consume(Chunk) override {} + void consume(Chunk &) override {} }; using SinkPtr = std::shared_ptr; diff --git a/src/Processors/Sources/BlocksSource.h b/src/Processors/Sources/BlocksSource.h index ec0dc9609f1..7ac460c14e2 100644 --- a/src/Processors/Sources/BlocksSource.h +++ b/src/Processors/Sources/BlocksSource.h @@ -43,7 +43,10 @@ protected: info->bucket_num = res.info.bucket_num; info->is_overflows = res.info.is_overflows; - return Chunk(res.getColumns(), res.rows(), std::move(info)); + auto chunk = Chunk(res.getColumns(), res.rows()); + chunk.getChunkInfos().add(std::move(info)); + + return chunk; } private: diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..1578bd389c9 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -176,7 +176,7 @@ std::optional RemoteSource::tryGenerate() auto info = std::make_shared(); info->bucket_num = block.info.bucket_num; info->is_overflows = block.info.is_overflows; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); } return chunk; diff --git a/src/Processors/Sources/SourceFromSingleChunk.cpp b/src/Processors/Sources/SourceFromSingleChunk.cpp index 00f40a34361..9abe0504d10 100644 --- a/src/Processors/Sources/SourceFromSingleChunk.cpp +++ b/src/Processors/Sources/SourceFromSingleChunk.cpp @@ -5,7 +5,9 @@ namespace DB { -SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) {} +SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) +{ +} SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) { @@ -20,7 +22,7 @@ SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmp auto info = std::make_shared(); info->bucket_num = data.info.bucket_num; info->is_overflows = data.info.is_overflows; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 9ffe15d0f85..45b0960ec8f 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -332,7 +332,7 @@ void AggregatingInOrderTransform::generate() variants.aggregates_pool = variants.aggregates_pools.at(0).get(); /// Pass info about used memory by aggregate functions further. - to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + to_push_chunk.getChunkInfos().add(std::make_shared(cur_block_bytes)); cur_block_bytes = 0; cur_block_size = 0; @@ -351,11 +351,12 @@ FinalizeAggregatedTransform::FinalizeAggregatedTransform(Block header, Aggregati void FinalizeAggregatedTransform::transform(Chunk & chunk) { if (params->final) - finalizeChunk(chunk, aggregates_mask); - else if (!chunk.getChunkInfo()) { - auto info = std::make_shared(); - chunk.setChunkInfo(std::move(info)); + finalizeChunk(chunk, aggregates_mask); + } + else if (!chunk.getChunkInfos().get()) + { + chunk.getChunkInfos().add(std::make_shared()); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 5d50e97f552..41a0d7fc7f1 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -12,10 +13,12 @@ namespace DB struct InputOrderInfo; using InputOrderInfoPtr = std::shared_ptr; -struct ChunkInfoWithAllocatedBytes : public ChunkInfo +struct ChunkInfoWithAllocatedBytes : public ChunkInfoCloneable { + ChunkInfoWithAllocatedBytes(const ChunkInfoWithAllocatedBytes & other) = default; explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) : allocated_bytes(allocated_bytes_) {} + Int64 allocated_bytes; }; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 65f0612d738..517f035667f 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -35,7 +35,7 @@ Chunk convertToChunk(const Block & block) UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); return chunk; } @@ -44,15 +44,11 @@ namespace { const AggregatedChunkInfo * getInfoFromChunk(const Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk."); - - const auto * agg_info = typeid_cast(info.get()); + auto agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo."); - return agg_info; + return agg_info.get(); } /// Reads chunks from file in native format. Provide chunks with aggregation info. @@ -210,11 +206,7 @@ private: void process(Chunk && chunk) { - if (!chunk.hasChunkInfo()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with chunk info in {}", getName()); - - const auto & info = chunk.getChunkInfo(); - const auto * chunks_to_merge = typeid_cast(info.get()); + auto chunks_to_merge = chunk.getChunkInfos().get(); if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with ChunksToMerge info in {}", getName()); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index e167acde067..95983c39d1e 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -19,7 +20,7 @@ namespace CurrentMetrics namespace DB { -class AggregatedChunkInfo : public ChunkInfo +class AggregatedChunkInfo : public ChunkInfoCloneable { public: bool is_overflows = false; diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 965a084bb13..49a6581e685 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -27,18 +27,12 @@ public: } ExceptionKeepingTransform::work(); - if (finish_chunk) - { - data.chunk = std::move(finish_chunk); - ready_output = true; - } } protected: void onConsume(Chunk chunk) override { - if (auto res_chunk = DB::Squashing::squash(std::move(chunk))) - cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); + cur_chunk = Squashing::squash(std::move(chunk)); } GenerateResult onGenerate() override @@ -48,16 +42,10 @@ protected: res.is_done = true; return res; } - void onFinish() override - { - auto chunk = DB::Squashing::squash({}); - finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); - } private: Squashing squashing; Chunk cur_chunk; - Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/CountingTransform.cpp b/src/Processors/Transforms/CountingTransform.cpp index 3dfb9fe178f..2c6b3bd8638 100644 --- a/src/Processors/Transforms/CountingTransform.cpp +++ b/src/Processors/Transforms/CountingTransform.cpp @@ -1,6 +1,7 @@ - -#include #include + +#include +#include #include #include diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp new file mode 100644 index 00000000000..6786f76cbef --- /dev/null +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -0,0 +1,236 @@ +#include + +#include + +#include +#include +#include + + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void RestoreChunkInfosTransform::transform(Chunk & chunk) +{ + chunk.getChunkInfos().append(chunk_infos.clone()); +} + +namespace DeduplicationToken +{ + +String TokenInfo::getToken() const +{ + if (!isDefined()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is not defined, stage {}, token {}", stage, debugToken()); + + return getTokenImpl(); +} + +String TokenInfo::getTokenImpl() const +{ + String result; + result.reserve(getTotalSize()); + + for (const auto & part : parts) + { + if (!result.empty()) + result.append(":"); + result.append(part); + } + + return result; +} + +String TokenInfo::debugToken() const +{ + return getTokenImpl(); +} + +void TokenInfo::addChunkHash(String part) +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_WITH_HASHES; + + if (stage != DEFINE_SOURCE_WITH_HASHES) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(std::move(part)); +} + +void TokenInfo::finishChunkHashes() +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_WITH_HASHES; + + if (stage != DEFINE_SOURCE_WITH_HASHES) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + stage = DEFINED; +} + +void TokenInfo::setUserToken(const String & token) +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_USER_TOKEN; + + if (stage != DEFINE_SOURCE_USER_TOKEN) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("user-token-{}", token)); +} + +void TokenInfo::setSourceWithUserToken(size_t block_number) +{ + if (stage != DEFINE_SOURCE_USER_TOKEN) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("source-number-{}", block_number)); + + stage = DEFINED; +} + +void TokenInfo::setViewID(const String & id) +{ + if (stage == DEFINED) + stage = DEFINE_VIEW; + + if (stage != DEFINE_VIEW) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("view-id-{}", id)); +} + +void TokenInfo::setViewBlockNumber(size_t block_number) +{ + if (stage != DEFINE_VIEW) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("view-block-{}", block_number)); + + stage = DEFINED; +} + +void TokenInfo::reset() +{ + stage = UNDEFINED; + parts.clear(); +} + +void TokenInfo::addTokenPart(String part) +{ + parts.push_back(std::move(part)); +} + +size_t TokenInfo::getTotalSize() const +{ + if (parts.empty()) + return 0; + + size_t size = 0; + for (const auto & part : parts) + size += part.size(); + + // we reserve more size here to be able to add delimenter between parts. + return size + parts.size() - 1; +} + +#ifdef ABORT_ON_LOGICAL_ERROR +void CheckTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + + if (!token_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk has to have DedupTokenInfo as ChunkInfo, {}", debug); + + LOG_DEBUG(log, "debug: {}, token: {}", debug, token_info->debugToken()); +} +#endif + +String DefineSourceWithChunkHashTransform::getChunkHash(const Chunk & chunk) +{ + SipHash hash; + for (const auto & colunm : chunk.getColumns()) + colunm->updateHashFast(hash); + + const auto hash_value = hash.get128(); + return toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); +} + + +void DefineSourceWithChunkHashTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in DefineSourceWithChunkHashesTransform"); + + if (token_info->isDefined()) + return; + + token_info->addChunkHash(getChunkHash(chunk)); + token_info->finishChunkHashes(); +} + +void SetUserTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetUserTokenTransform"); + token_info->setUserToken(user_token); +} + +void SetSourceBlockNumberTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetSourceBlockNumberTransform"); + token_info->setSourceWithUserToken(block_number++); +} + +void SetViewIDTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetViewIDTransform"); + token_info->setViewID(view_id); +} + +void SetViewBlockNumberTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetViewBlockNumberTransform"); + token_info->setViewBlockNumber(block_number++); +} + +void ResetTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in ResetTokenTransform"); + + token_info->reset(); +} + +} +} diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.h b/src/Processors/Transforms/DeduplicationTokenTransforms.h new file mode 100644 index 00000000000..d6aff9e1370 --- /dev/null +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.h @@ -0,0 +1,237 @@ +#pragma once + +#include +#include + +#include +#include "Common/Logger.h" + + +namespace DB +{ + class RestoreChunkInfosTransform : public ISimpleTransform + { + public: + RestoreChunkInfosTransform(Chunk::ChunkInfoCollection chunk_infos_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , chunk_infos(std::move(chunk_infos_)) + {} + + String getName() const override { return "RestoreChunkInfosTransform"; } + + void transform(Chunk & chunk) override; + + private: + Chunk::ChunkInfoCollection chunk_infos; + }; + + +namespace DeduplicationToken +{ + class TokenInfo : public ChunkInfoCloneable + { + public: + TokenInfo() = default; + TokenInfo(const TokenInfo & other) = default; + + String getToken() const; + String debugToken() const; + + bool empty() const { return parts.empty(); } + + bool isDefined() const { return stage == DEFINED; } + + void addChunkHash(String part); + void finishChunkHashes(); + + void setUserToken(const String & token); + void setSourceWithUserToken(size_t block_number); + + void setViewID(const String & id); + void setViewBlockNumber(size_t block_number); + + void reset(); + + private: + String getTokenImpl() const; + + void addTokenPart(String part); + size_t getTotalSize() const; + + /* Token has to be prepared in a particular order. + * BuildingStage ensures that token is expanded according the following order. + * Firstly token is expanded with information about the source. + * It could be done with two ways: add several hash sums from the source chunks or provide user defined deduplication token and its sequentional block number. + * + * transition // method + * UNDEFINED -> DEFINE_SOURCE_WITH_HASHES // addChunkHash + * DEFINE_SOURCE_WITH_HASHES -> DEFINE_SOURCE_WITH_HASHES // addChunkHash + * DEFINE_SOURCE_WITH_HASHES -> DEFINED // defineSourceWithChankHashes + * + * transition // method + * UNDEFINED -> DEFINE_SOURCE_USER_TOKEN // setUserToken + * DEFINE_SOURCE_USER_TOKEN -> DEFINED // defineSourceWithUserToken + * + * After token is defined, it could be extended with view id and view block number. Actually it has to be expanded with view details if there is one or several views. + * + * transition // method + * DEFINED -> DEFINE_VIEW // setViewID + * DEFINE_VIEW -> DEFINED // defineViewID + */ + + enum BuildingStage + { + UNDEFINED, + DEFINE_SOURCE_WITH_HASHES, + DEFINE_SOURCE_USER_TOKEN, + DEFINE_VIEW, + DEFINED, + }; + + BuildingStage stage = UNDEFINED; + std::vector parts; + }; + + +#ifdef ABORT_ON_LOGICAL_ERROR + /// use that class only with debug builds in CI for introspection + class CheckTokenTransform : public ISimpleTransform + { + public: + CheckTokenTransform(String debug_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , debug(std::move(debug_)) + { + } + + String getName() const override { return "DeduplicationToken::CheckTokenTransform"; } + + void transform(Chunk & chunk) override; + + private: + String debug; + LoggerPtr log = getLogger("CheckInsertDeduplicationTokenTransform"); + }; +#endif + + + class AddTokenInfoTransform : public ISimpleTransform + { + public: + explicit AddTokenInfoTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::AddTokenInfoTransform"; } + + void transform(Chunk & chunk) override + { + chunk.getChunkInfos().add(std::make_shared()); + } + }; + + + class DefineSourceWithChunkHashTransform : public ISimpleTransform + { + public: + explicit DefineSourceWithChunkHashTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::DefineSourceWithChunkHashesTransform"; } + + // Usually MergeTreeSink/ReplicatedMergeTreeSink calls addChunkHash for the deduplication token with hashes from the parts. + // But if there is some table with different engine, we still need to define the source of the data in deduplication token + // We use that transform to define the source as a hash of entire block in deduplication token + void transform(Chunk & chunk) override; + + static String getChunkHash(const Chunk & chunk); + }; + + class ResetTokenTransform : public ISimpleTransform + { + public: + explicit ResetTokenTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::ResetTokenTransform"; } + + void transform(Chunk & chunk) override; + }; + + + class SetUserTokenTransform : public ISimpleTransform + { + public: + SetUserTokenTransform(String user_token_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , user_token(std::move(user_token_)) + { + } + + String getName() const override { return "DeduplicationToken::SetUserTokenTransform"; } + + void transform(Chunk & chunk) override; + + private: + String user_token; + }; + + + class SetSourceBlockNumberTransform : public ISimpleTransform + { + public: + explicit SetSourceBlockNumberTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::SetSourceBlockNumberTransform"; } + + void transform(Chunk & chunk) override; + + private: + size_t block_number = 0; + }; + + + class SetViewIDTransform : public ISimpleTransform + { + public: + SetViewIDTransform(String view_id_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , view_id(std::move(view_id_)) + { + } + + String getName() const override { return "DeduplicationToken::SetViewIDTransform"; } + + void transform(Chunk & chunk) override; + + private: + String view_id; + }; + + + class SetViewBlockNumberTransform : public ISimpleTransform + { + public: + explicit SetViewBlockNumberTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::SetViewBlockNumberTransform"; } + + void transform(Chunk & chunk) override; + + private: + size_t block_number = 0; + }; + +} +} diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 2fbd2c21b8d..04fabc9a3c6 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -1,5 +1,7 @@ #include #include + + namespace DB { diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 3e2a9462e54..ca204bcb482 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -365,10 +365,9 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() return Status::Finished; } - if (!data.chunk.hasChunkInfo()) + task = data.chunk.getChunkInfos().get(); + if (!task) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); - - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else { @@ -479,7 +478,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (output.isFinished()) continue; Chunk chunk; - chunk.setChunkInfo(std::make_shared()); + chunk.getChunkInfos().add(std::make_shared()); output.push(std::move(chunk)); output.finish(); } @@ -496,7 +495,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() { Chunk chunk; auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); - chunk.setChunkInfo(task); + chunk.getChunkInfos().add(std::move(task)); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index a308af03662..5f6d9d6fff2 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -1,6 +1,7 @@ #pragma once #include - +#include +#include namespace DB { @@ -111,11 +112,12 @@ private: }; -class DelayedBlocksTask : public ChunkInfo +class DelayedBlocksTask : public ChunkInfoCloneable { public: DelayedBlocksTask() = default; + DelayedBlocksTask(const DelayedBlocksTask & other) = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index 1eaa5458d37..9ae80e21a68 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,6 +1,7 @@ #include #include + namespace DB { diff --git a/src/Processors/Transforms/MemoryBoundMerging.h b/src/Processors/Transforms/MemoryBoundMerging.h index 607087fb39c..d7bc320173b 100644 --- a/src/Processors/Transforms/MemoryBoundMerging.h +++ b/src/Processors/Transforms/MemoryBoundMerging.h @@ -150,11 +150,7 @@ private: if (!chunk.hasRows()) return; - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedForMemoryBoundMergingTransform."); - - const auto * agg_info = typeid_cast(info.get()); + const auto & agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception( ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedForMemoryBoundMergingTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index fc40c6894bb..ea9ebb0f96e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -30,10 +30,10 @@ void GroupingAggregatedTransform::pushData(Chunks chunks, Int32 bucket, bool is_ auto info = std::make_shared(); info->bucket_num = bucket; info->is_overflows = is_overflows; - info->chunks = std::make_unique(std::move(chunks)); + info->chunks = std::make_shared(std::move(chunks)); Chunk chunk; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); output.push(std::move(chunk)); } @@ -255,11 +255,10 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) if (!chunk.hasRows()) return; - const auto & info = chunk.getChunkInfo(); - if (!info) + if (chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in GroupingAggregatedTransform."); - if (const auto * agg_info = typeid_cast(info.get())) + if (auto agg_info = chunk.getChunkInfos().get()) { Int32 bucket = agg_info->bucket_num; bool is_overflows = agg_info->is_overflows; @@ -275,7 +274,7 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) last_bucket_number[input] = bucket; } } - else if (typeid_cast(info.get())) + else if (chunk.getChunkInfos().get()) { single_level_chunks.emplace_back(std::move(chunk)); } @@ -304,7 +303,11 @@ void GroupingAggregatedTransform::work() Int32 bucket = cur_block.info.bucket_num; auto chunk_info = std::make_shared(); chunk_info->bucket_num = bucket; - chunks_map[bucket].emplace_back(Chunk(cur_block.getColumns(), cur_block.rows(), std::move(chunk_info))); + + auto chunk = Chunk(cur_block.getColumns(), cur_block.rows()); + chunk.getChunkInfos().add(std::move(chunk_info)); + + chunks_map[bucket].emplace_back(std::move(chunk)); } } } @@ -319,9 +322,7 @@ MergingAggregatedBucketTransform::MergingAggregatedBucketTransform( void MergingAggregatedBucketTransform::transform(Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - const auto * chunks_to_merge = typeid_cast(info.get()); - + auto chunks_to_merge = chunk.getChunkInfos().get(); if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergingAggregatedSimpleTransform chunk must have ChunkInfo with type ChunksToMerge."); @@ -330,11 +331,10 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) BlocksList blocks_list; for (auto & cur_chunk : *chunks_to_merge->chunks) { - const auto & cur_info = cur_chunk.getChunkInfo(); - if (!cur_info) + if (cur_chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedBucketTransform."); - if (const auto * agg_info = typeid_cast(cur_info.get())) + if (auto agg_info = cur_chunk.getChunkInfos().get()) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = agg_info->is_overflows; @@ -342,7 +342,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) blocks_list.emplace_back(std::move(block)); } - else if (typeid_cast(cur_info.get())) + else if (cur_chunk.getChunkInfos().get()) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = false; @@ -361,7 +361,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) res_info->is_overflows = chunks_to_merge->is_overflows; res_info->bucket_num = chunks_to_merge->bucket_num; res_info->chunk_num = chunks_to_merge->chunk_num; - chunk.setChunkInfo(std::move(res_info)); + chunk.getChunkInfos().add(std::move(res_info)); auto block = params->aggregator.mergeBlocks(blocks_list, params->final, is_cancelled); @@ -405,11 +405,7 @@ bool SortingAggregatedTransform::tryPushChunk() void SortingAggregatedTransform::addChunk(Chunk chunk, size_t from_input) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedTransform."); - - const auto * agg_info = typeid_cast(info.get()); + auto agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 77ee3034ffc..3a3c1bd9c1e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -142,9 +143,9 @@ private: void addChunk(Chunk chunk, size_t from_input); }; -struct ChunksToMerge : public ChunkInfo +struct ChunksToMerge : public ChunkInfoCloneable { - std::unique_ptr chunks; + std::shared_ptr chunks; Int32 bucket_num = -1; bool is_overflows = false; UInt64 chunk_num = 0; // chunk number in order of generation, used during memory bound merging to restore chunks order diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index ad723da7527..446e60a0b81 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -32,11 +32,10 @@ void MergingAggregatedTransform::consume(Chunk chunk) total_input_rows += input_rows; ++total_input_blocks; - const auto & info = chunk.getChunkInfo(); - if (!info) + if (chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedTransform."); - if (const auto * agg_info = typeid_cast(info.get())) + if (auto agg_info = chunk.getChunkInfos().get()) { /** If the remote servers used a two-level aggregation method, * then blocks will contain information about the number of the bucket. @@ -49,7 +48,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block)); } - else if (typeid_cast(info.get())) + else if (chunk.getChunkInfos().get()) { auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); block.info.is_overflows = false; @@ -89,7 +88,8 @@ Chunk MergingAggregatedTransform::generate() UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); + + chunk.getChunkInfos().add(std::move(info)); return chunk; } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 0f433165f14..ee4dfa6a64e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -10,20 +10,20 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header, header), squashing(header, min_block_size_rows, min_block_size_bytes) + Block header_, size_t min_block_size_rows, size_t min_block_size_bytes) + : IInflatingTransform(header_, header_) + , squashing(header_, min_block_size_rows, min_block_size_bytes) { } void PlanSquashingTransform::consume(Chunk chunk) { - if (Chunk current_chunk = squashing.add(std::move(chunk)); current_chunk.hasChunkInfo()) - squashed_chunk.swap(current_chunk); + squashed_chunk = squashing.add(std::move(chunk)); } Chunk PlanSquashingTransform::generate() { - if (!squashed_chunk.hasChunkInfo()) + if (!squashed_chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); Chunk result_chunk; @@ -33,12 +33,11 @@ Chunk PlanSquashingTransform::generate() bool PlanSquashingTransform::canGenerate() { - return squashed_chunk.hasChunkInfo(); + return bool(squashed_chunk); } Chunk PlanSquashingTransform::getRemaining() { - Chunk current_chunk = squashing.flush(); - return current_chunk; + return squashing.flush(); } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 4ad2ec2d089..e6db245499e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -10,7 +10,7 @@ class PlanSquashingTransform : public IInflatingTransform { public: PlanSquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + Block header_, size_t min_block_size_rows, size_t min_block_size_bytes); String getName() const override { return "PlanSquashingTransform"; } @@ -23,7 +23,6 @@ protected: private: Squashing squashing; Chunk squashed_chunk; - Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/SelectByIndicesTransform.h b/src/Processors/Transforms/SelectByIndicesTransform.h index 480ab1a0f61..b44f5a3203e 100644 --- a/src/Processors/Transforms/SelectByIndicesTransform.h +++ b/src/Processors/Transforms/SelectByIndicesTransform.h @@ -26,7 +26,7 @@ public: void transform(Chunk & chunk) override { size_t num_rows = chunk.getNumRows(); - const auto * select_final_indices_info = typeid_cast(chunk.getChunkInfo().get()); + auto select_final_indices_info = chunk.getChunkInfos().extract(); if (!select_final_indices_info || !select_final_indices_info->select_final_indices) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk passed to SelectByIndicesTransform without indices column"); @@ -41,7 +41,6 @@ public: chunk.setColumns(std::move(columns), index_column->size()); } - chunk.setChunkInfo(nullptr); } }; diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 34b733cde5e..1fb4433240a 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -18,9 +18,7 @@ SquashingTransform::SquashingTransform( void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = squashing.add(std::move(chunk)); - if (planned_chunk.hasChunkInfo()) - cur_chunk = DB::Squashing::squash(std::move(planned_chunk)); + cur_chunk = Squashing::squash(squashing.add(std::move(chunk))); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -33,10 +31,7 @@ SquashingTransform::GenerateResult SquashingTransform::onGenerate() void SquashingTransform::onFinish() { - Chunk chunk = squashing.flush(); - if (chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(chunk)); - finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); + finish_chunk = Squashing::squash(squashing.flush()); } void SquashingTransform::work() @@ -49,6 +44,7 @@ void SquashingTransform::work() } ExceptionKeepingTransform::work(); + if (finish_chunk) { data.chunk = std::move(finish_chunk); @@ -67,18 +63,14 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - Chunk planned_chunk = squashing.add(std::move(chunk)); - if (planned_chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(planned_chunk)); + chunk = Squashing::squash(squashing.add(std::move(chunk))); } else { if (chunk.hasRows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk expected to be empty, otherwise it will be lost"); - chunk = squashing.flush(); - if (chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(chunk)); + chunk = Squashing::squash(squashing.flush()); } } diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index aa86879e62c..59fceccb538 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -150,11 +150,7 @@ void TotalsHavingTransform::transform(Chunk & chunk) /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. if (overflow_row) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in TotalsHavingTransform."); - - const auto * agg_info = typeid_cast(info.get()); + const auto & agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in TotalsHavingTransform."); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 25fbf13b0e7..312b333ab33 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -5,7 +5,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -16,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -24,9 +27,12 @@ #include #include #include +#include "base/defines.h" +#include #include #include +#include namespace ProfileEvents @@ -105,7 +111,7 @@ private: class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: - ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_); + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_, bool disable_deduplication_for_children_); String getName() const override { return "ExecutingInnerQueryFromView"; } @@ -116,6 +122,7 @@ protected: private: ViewsDataPtr views_data; ViewRuntimeData & view; + bool disable_deduplication_for_children; struct State { @@ -138,7 +145,7 @@ class PushingToLiveViewSink final : public SinkToStorage public: PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToLiveViewSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; private: StorageLiveView & live_view; @@ -152,7 +159,7 @@ class PushingToWindowViewSink final : public SinkToStorage public: PushingToWindowViewSink(const Block & header, StorageWindowView & window_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToWindowViewSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; private: StorageWindowView & window_view; @@ -216,45 +223,10 @@ std::optional generateViewChain( const auto & insert_settings = insert_context->getSettingsRef(); - // Do not deduplicate insertions into MV if the main insertion is Ok if (disable_deduplication_for_children) { insert_context->setSetting("insert_deduplicate", Field{false}); } - else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views && - !insert_settings.insert_deduplication_token.value.empty()) - { - /** Update deduplication token passed to dependent MV with current view id. So it is possible to properly handle - * deduplication in complex INSERT flows. - * - * Example: - * - * landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 - * | | - * └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ - * - * Here we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will - * be inserted into `ds_2_1`. - * - * We are forced to use view id instead of table id because there are some possible INSERT flows where no tables - * are involved. - * - * Example: - * - * landing -┬--> mv_1_1 --┬-> ds_1_1 - * | | - * └--> mv_1_2 --┘ - * - */ - auto insert_deduplication_token = insert_settings.insert_deduplication_token.value; - - if (view_id.hasUUID()) - insert_deduplication_token += "_" + toString(view_id.uuid); - else - insert_deduplication_token += "_" + view_id.getFullNameNotQuoted(); - - insert_context->setSetting("insert_deduplication_token", insert_deduplication_token); - } // Processing of blocks for MVs is done block by block, and there will // be no parallel reading after (plus it is not a costless operation) @@ -361,7 +333,13 @@ std::optional generateViewChain( insert_columns.emplace_back(column.name); } - InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); + InterpreterInsertQuery interpreter( + nullptr, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; @@ -378,6 +356,10 @@ std::optional generateViewChain( table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); } +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Before squashing", out.getInputHeader())); +#endif + auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); counting->setProgressCallback(insert_context->getProgressCallback()); @@ -420,11 +402,19 @@ std::optional generateViewChain( if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); +#endif + auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data); + storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); out.addSource(std::move(executing_inner_query)); + +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); +#endif } return out; @@ -465,11 +455,7 @@ Chain buildPushingToViewsChain( */ result_chain.addTableLock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); - /// If the "root" table deduplicates blocks, there are no need to make deduplication for children - /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks - bool disable_deduplication_for_children = false; - if (!context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); + bool disable_deduplication_for_children = !context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views; auto table_id = storage->getStorageID(); auto views = DatabaseCatalog::instance().getDependentViews(table_id); @@ -560,12 +546,25 @@ Chain buildPushingToViewsChain( auto sink = std::make_shared(live_view_header, *live_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } else if (auto * window_view = dynamic_cast(storage.get())) { auto sink = std::make_shared(window_view->getInputHeader(), *window_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); + } + else if (dynamic_cast(storage.get())) + { + auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); + metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); + sink->setRuntimeData(thread_status, elapsed_counter_ms); + result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } /// Do not push to destination table if the flag is set else if (!no_destination) @@ -573,8 +572,15 @@ Chain buildPushingToViewsChain( auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); sink->setRuntimeData(thread_status, elapsed_counter_ms); + + result_chain.addSource(std::make_shared(sink->getHeader())); + result_chain.addSource(std::move(sink)); } + else + { + result_chain.addSource(std::make_shared(storage_header)); + } if (result_chain.empty()) result_chain.addSink(std::make_shared(storage_header)); @@ -590,7 +596,7 @@ Chain buildPushingToViewsChain( return result_chain; } -static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data) +static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data, Chunk::ChunkInfoCollection && chunk_infos, bool disable_deduplication_for_children) { const auto & context = view.context; @@ -637,6 +643,19 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat pipeline.getHeader(), std::make_shared(std::move(converting)))); + pipeline.addTransform(std::make_shared(std::move(chunk_infos), pipeline.getHeader())); + + if (!disable_deduplication_for_children) + { + String materialize_view_id = view.table_id.hasUUID() ? toString(view.table_id.uuid) : view.table_id.getFullNameNotQuoted(); + pipeline.addTransform(std::make_shared(std::move(materialize_view_id), pipeline.getHeader())); + pipeline.addTransform(std::make_shared(pipeline.getHeader())); + } + else + { + pipeline.addTransform(std::make_shared(pipeline.getHeader())); + } + return QueryPipelineBuilder::getPipeline(std::move(pipeline)); } @@ -728,17 +747,19 @@ IProcessor::Status CopyingDataToViewsTransform::prepare() ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( const Block & header, ViewRuntimeData & view_, - std::shared_ptr views_data_) + std::shared_ptr views_data_, + bool disable_deduplication_for_children_) : ExceptionKeepingTransform(header, view_.sample_block) , views_data(std::move(views_data_)) , view(view_) + , disable_deduplication_for_children(disable_deduplication_for_children_) { } void ExecutingInnerQueryFromViewTransform::onConsume(Chunk chunk) { - auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); - state.emplace(process(block, view, *views_data)); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + state.emplace(process(std::move(block), view, *views_data, std::move(chunk.getChunkInfos()), disable_deduplication_for_children)); } @@ -770,10 +791,10 @@ PushingToLiveViewSink::PushingToLiveViewSink(const Block & header, StorageLiveVi { } -void PushingToLiveViewSink::consume(Chunk chunk) +void PushingToLiveViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); - live_view.writeBlock(getHeader().cloneWithColumns(chunk.detachColumns()), context); + live_view.writeBlock(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); @@ -793,11 +814,11 @@ PushingToWindowViewSink::PushingToWindowViewSink( { } -void PushingToWindowViewSink::consume(Chunk chunk) +void PushingToWindowViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( - window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); + window_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index f0b2ead687e..a9e5b1535c0 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -193,7 +193,7 @@ public: return concurrency_control; } - void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); } + void addResources(QueryPlanResourceHolder resources_) { resources.append(std::move(resources_)); } void setQueryIdHolder(std::shared_ptr query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); } void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); } diff --git a/src/QueryPipeline/QueryPlanResourceHolder.cpp b/src/QueryPipeline/QueryPlanResourceHolder.cpp index 2cd4dc42a83..bb2be2c8ffb 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.cpp +++ b/src/QueryPipeline/QueryPlanResourceHolder.cpp @@ -5,7 +5,7 @@ namespace DB { -QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept +QueryPlanResourceHolder & QueryPlanResourceHolder::append(QueryPlanResourceHolder && rhs) noexcept { table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); @@ -16,6 +16,12 @@ QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHo return *this; } +QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept +{ + append(std::move(rhs)); + return *this; +} + QueryPlanResourceHolder::QueryPlanResourceHolder() = default; QueryPlanResourceHolder::QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept = default; QueryPlanResourceHolder::~QueryPlanResourceHolder() = default; diff --git a/src/QueryPipeline/QueryPlanResourceHolder.h b/src/QueryPipeline/QueryPlanResourceHolder.h index ed9eb68b7ba..10f7f39ab09 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.h +++ b/src/QueryPipeline/QueryPlanResourceHolder.h @@ -20,8 +20,11 @@ struct QueryPlanResourceHolder QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept; ~QueryPlanResourceHolder(); + QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &) = delete; + /// Custom move assignment does not destroy data from lhs. It appends data from rhs to lhs. QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &&) noexcept; + QueryPlanResourceHolder & append(QueryPlanResourceHolder &&) noexcept; /// Some processors may implicitly use Context or temporary Storage created by Interpreter. /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ac1423f87c1..fccea9e258e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -888,12 +888,11 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { - squashing.header = state.block_for_insert; - auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); - if (planned_chunk.hasChunkInfo()) + squashing.setHeader(state.block_for_insert.cloneEmpty()); + auto result_chunk = Squashing::squash(squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()})); + if (result_chunk) { - Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -902,12 +901,13 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto planned_chunk = squashing.flush(); - Chunk result_chunk; - if (planned_chunk.hasChunkInfo()) - result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + Chunk result_chunk = Squashing::squash(squashing.flush()); + if (!result_chunk) + { + return insert_queue.pushQueryWithBlock(state.parsed_query, squashing.getHeader(), query_context); + } - auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index e556bda2561..8791668cd89 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -134,7 +134,7 @@ DistributedSink::DistributedSink( } -void DistributedSink::consume(Chunk chunk) +void DistributedSink::consume(Chunk & chunk) { if (is_first_chunk) { @@ -142,7 +142,7 @@ void DistributedSink::consume(Chunk chunk) is_first_chunk = false; } - auto ordinary_block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto ordinary_block = getHeader().cloneWithColumns(chunk.getColumns()); if (insert_sync) writeSync(ordinary_block); @@ -420,7 +420,13 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// to resolve tables (in InterpreterInsertQuery::getTable()) auto copy_query_ast = query_ast->clone(); - InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); + InterpreterInsertQuery interp( + copy_query_ast, + job.local_context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interp.execute(); job.pipeline = std::move(block_io.pipeline); @@ -715,7 +721,13 @@ void DistributedSink::writeToLocal(const Cluster::ShardInfo & shard_info, const try { - InterpreterInsertQuery interp(query_ast, context, allow_materialized); + InterpreterInsertQuery interp( + query_ast, + context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interp.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index a4c95633595..5b7396f2c6f 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -49,7 +49,7 @@ public: const Names & columns_to_send_); String getName() const override { return "DistributedSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index abd4b4ce23b..0f9bd8b6ff9 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -740,7 +740,14 @@ bool StorageFileLog::streamToViews() auto new_context = Context::createCopy(getContext()); - InterpreterInsertQuery interpreter(insert, new_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + new_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); + auto block_io = interpreter.execute(); /// Each stream responsible for closing it's files and store meta diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index f5c5d093ce1..809401bb279 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1099,7 +1099,13 @@ bool StorageKafka::streamToViews() // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + kafka_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index 792133ced64..9803fa0a160 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -71,9 +71,9 @@ public: new_hash.reset(); } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); block.updateHash(*new_hash); new_blocks->push_back(std::move(block)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 57a1ea302f9..82759e8a851 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,6 +21,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -330,7 +331,7 @@ Pipe StorageLiveView::watch( return reader; } -void StorageLiveView::writeBlock(const Block & block, ContextPtr local_context) +void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) { auto output = std::make_shared(*this); @@ -407,6 +408,21 @@ void StorageLiveView::writeBlock(const Block & block, ContextPtr local_context) builder = interpreter.buildQueryPipeline(); } + builder.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(chunk_infos.clone(), cur_header); + }); + + String live_view_id = live_view.getStorageID().hasUUID() ? toString(live_view.getStorageID().uuid) : live_view.getStorageID().getFullNameNotQuoted(); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(live_view_id, stream_header); + }); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header); + }); + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 91daac32c7b..12d8e898347 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -118,7 +118,7 @@ public: return 0; } - void writeBlock(const Block & block, ContextPtr context); + void writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); void refresh(); diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index bc8cb0ce69a..ff5214a5e51 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -377,7 +377,13 @@ void RefreshTask::executeRefreshUnlocked(std::shared_ptr(task->getInfo().data_part->info.level)); + return ChunkAndProgress{ - .chunk = Chunk(ordered_columns, res.row_count, add_part_level ? std::make_shared(task->getInfo().data_part->info.level) : nullptr), + .chunk = std::move(chunk), .num_read_rows = res.num_read_rows, .num_read_bytes = res.num_read_bytes, .is_finished = false}; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 02f8d6f4f6a..4f90f7131da 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -264,7 +264,10 @@ try ++it; } - return Chunk(std::move(res_columns), rows_read, add_part_level ? std::make_shared(data_part->info.level) : nullptr); + auto result = Chunk(std::move(res_columns), rows_read); + if (add_part_level) + result.getChunkInfos().add(std::make_shared(data_part->info.level)); + return result; } } else diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 05751e0fa6f..d8cfce1ca99 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -1,14 +1,27 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include + +#include namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; } +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace DB { @@ -58,12 +71,12 @@ void MergeTreeSink::onCancel() { } -void MergeTreeSink::consume(Chunk chunk) +void MergeTreeSink::consume(Chunk & chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(nullptr, context, false); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -76,6 +89,18 @@ void MergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in MergeTreeSink for table: {}", + storage.getStorageID().getNameForLogs()); + + const bool need_to_define_dedup_token = !token_info->isDefined(); + + String block_dedup_token; + if (token_info->isDefined()) + block_dedup_token = token_info->getToken(); + for (auto & current_block : part_blocks) { ProfileEvents::Counters part_counters; @@ -100,22 +125,16 @@ void MergeTreeSink::consume(Chunk chunk) if (!temp_part.part) continue; + if (need_to_define_dedup_token) + { + chassert(temp_part.part); + const auto hash_value = temp_part.part->getPartBlockIDHash(); + token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); + } + if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite()) support_parallel_write = true; - String block_dedup_token; - if (storage.getDeduplicationLog()) - { - const String & dedup_token = settings.insert_deduplication_token; - if (!dedup_token.empty()) - { - /// multiple blocks can be inserted within the same insert query - /// an ordinal number is added to dedup token to generate a distinctive block id for each block - block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); - ++chunk_dedup_seqnum; - } - } - size_t max_insert_delayed_streams_for_parallel_write; if (settings.max_insert_delayed_streams_for_parallel_write.changed) @@ -127,6 +146,7 @@ void MergeTreeSink::consume(Chunk chunk) /// In case of too much columns/parts in block, flush explicitly. streams += temp_part.streams.size(); + if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(); @@ -143,11 +163,16 @@ void MergeTreeSink::consume(Chunk chunk) { .temp_part = std::move(temp_part), .elapsed_ns = elapsed_ns, - .block_dedup_token = std::move(block_dedup_token), + .block_dedup_token = block_dedup_token, .part_counters = std::move(part_counters), }); } + if (need_to_define_dedup_token) + { + token_info->finishChunkHashes(); + } + finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); @@ -160,6 +185,8 @@ void MergeTreeSink::finishDelayedChunk() if (!delayed_chunk) return; + const Settings & settings = context->getSettingsRef(); + for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -178,7 +205,8 @@ void MergeTreeSink::finishDelayedChunk() storage.fillNewPartName(part, lock); auto * deduplication_log = storage.getDeduplicationLog(); - if (deduplication_log) + + if (settings.insert_deduplicate && deduplication_log) { const String block_id = part->getZeroLevelPartBlockID(partition.block_dedup_token); auto res = deduplication_log->addPart(block_id, part->info); diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index cf6715a3415..90976020d52 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -25,7 +25,7 @@ public: ~MergeTreeSink() override; String getName() const override { return "MergeTreeSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onStart() override; void onFinish() override; void onCancel() override; @@ -36,7 +36,6 @@ private: size_t max_parts_per_block; ContextPtr context; StorageSnapshotPtr storage_snapshot; - UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token UInt64 num_blocks_processed = 0; /// We can delay processing for previous chunk and start writing a new one. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a552ee89aee..3dbcb5e5bda 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1297,6 +1297,7 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; + Block projection_header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1314,14 +1315,12 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].header = block_to_squash; - Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); + projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - if (planned_chunk.hasChunkInfo()) + Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + if (squashed_chunk) { - Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - - auto result = block_to_squash.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); @@ -1342,12 +1341,10 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; auto & projection_squash_plan = projection_squashes[i]; - auto planned_chunk = projection_squash_plan.flush(); - if (planned_chunk.hasChunkInfo()) + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) { - Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - - auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4b4f4c33e7d..bbae054fbed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1,21 +1,25 @@ -#include -#include -#include -#include -#include #include "Common/Exception.h" #include #include #include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include +#include +#include +#include + #include +#include + namespace ProfileEvents { @@ -253,12 +257,12 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const } template -void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) +void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); const auto & settings = context->getSettingsRef(); @@ -284,13 +288,25 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { - const auto & chunk_info = chunk.getChunkInfo(); - if (const auto * async_insert_info_ptr = typeid_cast(chunk_info.get())) + const auto async_insert_info_ptr = chunk.getChunkInfos().get(); + if (async_insert_info_ptr) async_insert_info = std::make_shared(async_insert_info_ptr->offsets, async_insert_info_ptr->tokens); else throw Exception(ErrorCodes::LOGICAL_ERROR, "No chunk info for async inserts"); } + String block_dedup_token; + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in ReplicatedMergeTreeSink for table: {}", + storage.getStorageID().getNameForLogs()); + + const bool need_to_define_dedup_token = !token_info->isDefined(); + + if (token_info->isDefined()) + block_dedup_token = token_info->getToken(); + auto part_blocks = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), max_parts_per_block, metadata_snapshot, context, async_insert_info); using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; @@ -342,23 +358,10 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) } else { - if (deduplicate) { - String block_dedup_token; - /// We add the hash from the data and partition identifier to deduplication ID. /// That is, do not insert the same data to the same partition twice. - - const String & dedup_token = settings.insert_deduplication_token; - if (!dedup_token.empty()) - { - /// multiple blocks can be inserted within the same insert query - /// an ordinal number is added to dedup token to generate a distinctive block id for each block - block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); - ++chunk_dedup_seqnum; - } - block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); LOG_DEBUG(log, "Wrote block with ID '{}', {} rows{}", block_id, current_block.block.rows(), quorumLogMessage(replicas_num)); } @@ -366,6 +369,13 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); } + + if (need_to_define_dedup_token) + { + chassert(temp_part.part); + const auto hash_value = temp_part.part->getPartBlockIDHash(); + token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); + } } profile_events_scope.reset(); @@ -411,17 +421,15 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) )); } + if (need_to_define_dedup_token) + { + token_info->finishChunkHashes(); + } + finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); - /// If deduplicated data should not be inserted into MV, we need to set proper - /// value for `last_block_is_duplicate`, which is possible only after the part is committed. - /// Othervide we can delay commit. - /// TODO: we can also delay commit if there is no MVs. - if (!settings.deduplicate_blocks_in_dependent_materialized_views) - finishDelayedChunk(zookeeper); - ++num_blocks_processed; } @@ -431,8 +439,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF if (!delayed_chunk) return; - last_block_is_duplicate = false; - for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -445,8 +451,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF { bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num).second; - last_block_is_duplicate = last_block_is_duplicate || deduplicated; - /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); @@ -535,7 +539,7 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl ProfileEventsScope profile_events_scope; String original_part_dir = part->getDataPartStorage().getPartDirectory(); - auto try_rollback_part_rename = [this, &part, &original_part_dir]() + auto try_rollback_part_rename = [this, &part, &original_part_dir] () { if (original_part_dir == part->getDataPartStorage().getPartDirectory()) return; @@ -1151,8 +1155,16 @@ void ReplicatedMergeTreeSinkImpl::onStart() template void ReplicatedMergeTreeSinkImpl::onFinish() { - auto zookeeper = storage.getZooKeeper(); - finishDelayedChunk(std::make_shared(zookeeper)); + const auto & settings = context->getSettingsRef(); + + ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( + settings.insert_keeper_fault_injection_probability, + settings.insert_keeper_fault_injection_seed, + storage.getZooKeeper(), + "ReplicatedMergeTreeSink::onFinish", + log); + + finishDelayedChunk(zookeeper); } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 39623c20584..7d025361717 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -51,7 +51,7 @@ public: ~ReplicatedMergeTreeSinkImpl() override; void onStart() override; - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; String getName() const override { return "ReplicatedMergeTreeSink"; } @@ -59,16 +59,6 @@ public: /// For ATTACHing existing data on filesystem. bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part); - /// For proper deduplication in MaterializedViews - bool lastBlockIsDuplicate() const override - { - /// If MV is responsible for deduplication, block is not considered duplicating. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - return false; - - return last_block_is_duplicate; - } - struct DelayedChunk; private: std::vector detectConflictsInAsyncBlockIDs(const std::vector & ids); @@ -126,7 +116,6 @@ private: bool allow_attach_while_readonly = false; bool quorum_parallel = false; const bool deduplicate = true; - bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; LoggerPtr log; diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 4fb81d69070..36899011e33 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -40,7 +40,7 @@ void MessageQueueSink::onFinish() producer->finish(); } -void MessageQueueSink::consume(Chunk chunk) +void MessageQueueSink::consume(Chunk & chunk) { const auto & columns = chunk.getColumns(); if (columns.empty()) diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index b3c1e61734f..4a9248c6c4d 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -35,7 +35,7 @@ public: String getName() const override { return storage_name + "Sink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onStart() override; void onFinish() override; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 0b88a9e8929..8f0e2d76473 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -644,7 +644,13 @@ bool StorageNATS::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + nats_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index f2f6eac333c..d2bdd0af302 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -39,12 +39,12 @@ StorageObjectStorageSink::StorageObjectStorageSink( configuration->format, *write_buf, sample_block, context, format_settings_); } -void StorageObjectStorageSink::consume(Chunk chunk) +void StorageObjectStorageSink::consume(Chunk & chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void StorageObjectStorageSink::onCancel() diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index e0081193686..6ab531bb21a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "StorageObjectStorageSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onCancel() override; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 4388864434e..14b828e7268 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -454,7 +454,13 @@ bool StorageObjectStorageQueue::streamToViews() while (!shutdown_called && !file_iterator->isFinished()) { - InterpreterInsertQuery interpreter(insert, queue_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + queue_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto read_from_format_info = prepareReadingFromFormat( block_io.pipeline.getHeader().getNames(), diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 09b009b26d8..ee2570756ed 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -51,7 +51,7 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) return it->second; } -void PartitionedSink::consume(Chunk chunk) +void PartitionedSink::consume(Chunk & chunk) { const auto & columns = chunk.getColumns(); @@ -104,7 +104,7 @@ void PartitionedSink::consume(Chunk chunk) for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { auto sink = getSinkForPartitionKey(partition_key); - sink->consume(std::move(partition_index_to_chunk[partition_index])); + sink->consume(partition_index_to_chunk[partition_index]); } } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 68edeb6fd73..fcd67556dc9 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartitionedSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onException(std::exception_ptr exception) override; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index ba3cc6f58d0..44479bd01e2 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -697,7 +697,13 @@ void MaterializedPostgreSQLConsumer::syncTables() insert->table_id = storage->getStorageID(); insert->columns = std::make_shared(buffer->columns_ast); - InterpreterInsertQuery interpreter(insert, insert_context, true); + InterpreterInsertQuery interpreter( + insert, + insert_context, + /* allow_materialized */ true, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto io = interpreter.execute(); auto input = std::make_shared( result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2bb1e2dde0d..f632e553a0d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -437,7 +437,13 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection auto insert_context = materialized_storage->getNestedTableContext(); - InterpreterInsertQuery interpreter(insert, insert_context); + InterpreterInsertQuery interpreter( + insert, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index e4b19992151..f3d2aff68c8 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1129,7 +1129,13 @@ bool StorageRabbitMQ::tryStreamToViews() } // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, rabbitmq_context, /* allow_materialized_ */ false, /* no_squash_ */ true, /* no_destination_ */ true); + InterpreterInsertQuery interpreter( + insert, + rabbitmq_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 90792c59d38..4b5188ca9f2 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -218,7 +218,7 @@ std::pair EmbeddedRocksDBBulkSink::seriali return {std::move(serialized_key_column), std::move(serialized_value_column)}; } -void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) +void EmbeddedRocksDBBulkSink::consume(Chunk & chunk_) { std::vector chunks_to_write = squash(std::move(chunk_)); @@ -247,7 +247,10 @@ void EmbeddedRocksDBBulkSink::onFinish() { /// If there is any data left, write it. if (!chunks.empty()) - consume({}); + { + Chunk empty; + consume(empty); + } } String EmbeddedRocksDBBulkSink::getTemporarySSTFilePath() diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index 1f548e7813d..64190c8c86f 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -32,7 +32,7 @@ public: ~EmbeddedRocksDBBulkSink() override; - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index c451cfd1bf5..1f7f6939f40 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -29,7 +29,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( serializations = getHeader().getSerializations(); } -void EmbeddedRocksDBSink::consume(Chunk chunk) +void EmbeddedRocksDBSink::consume(Chunk & chunk) { auto rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.h b/src/Storages/RocksDB/EmbeddedRocksDBSink.h index 011322df829..2e1e0c7b429 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.h @@ -17,7 +17,7 @@ public: StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; String getName() const override { return "EmbeddedRocksDBSink"; } private: diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index b9d3e071b6c..3473166a080 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -313,7 +313,8 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt Block block; while (executor.pull(block)) { - sink->consume(Chunk{block.getColumns(), block.rows()}); + auto chunk = Chunk(block.getColumns(), block.rows()); + sink->consume(chunk); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a3f6b6afc5d..b064fba223a 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -607,7 +607,7 @@ public: String getName() const override { return "BufferSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { size_t rows = chunk.getNumRows(); if (!rows) @@ -1020,7 +1020,13 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto insert_context = Context::createCopy(getContext()); insert_context->makeQueryContext(); - InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; + InterpreterInsertQuery interpreter( + insert, + insert_context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 849fa5dbe0b..67586985ce8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1050,7 +1050,13 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu const auto & shard_info = shards_info[shard_index]; if (shard_info.isLocal()) { - InterpreterInsertQuery interpreter(new_query, query_context); + InterpreterInsertQuery interpreter( + new_query, + query_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); pipeline.addCompletedPipeline(interpreter.execute().pipeline); } else diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7f39ff615f0..3fb397c7b81 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1778,12 +1778,12 @@ public: String getName() const override { return "StorageFileSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { std::lock_guard cancel_lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void onCancel() override diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index 20f99070000..c80e799a92b 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -119,10 +119,10 @@ public: std::string getName() const override { return "StorageKeeperMapSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -1248,7 +1248,10 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca Block block; while (executor.pull(block)) - sink->consume(Chunk{block.getColumns(), block.rows()}); + { + auto chunk = Chunk(block.getColumns(), block.rows()); + sink->consume(chunk); + } sink->finalize(strict); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index de0324d7998..463694c63aa 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -21,7 +22,6 @@ #include #include -#include "StorageLogSettings.h" #include #include #include @@ -341,7 +341,7 @@ public: } } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: @@ -398,9 +398,9 @@ private: }; -void LogSink::consume(Chunk chunk) +void LogSink::consume(Chunk & chunk) { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); metadata_snapshot->check(block, true); for (auto & stream : streams | boost::adaptors::map_values) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f69c4adb552..b1bd7053c2e 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -63,7 +63,7 @@ public: String getName() const override { return "MemorySink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); storage_snapshot->metadata->check(block, true); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 62a2a048642..e0818fafae9 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include @@ -107,12 +106,12 @@ public: String getName() const override { return "StorageMongoDBSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { Poco::MongoDB::Database db(db_name); Poco::MongoDB::Document::Vector documents; - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); size_t num_rows = block.rows(); size_t num_cols = block.columns(); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index da391909dff..2a8a7bd2ee7 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -151,9 +151,9 @@ public: String getName() const override { return "StorageMySQLSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); auto blocks = splitBlocks(block, max_batch_rows); mysqlxx::Transaction trans(entry); try diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index a8713c61e4d..cdfeab62b58 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -227,9 +227,9 @@ public: String getName() const override { return "PostgreSQLSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); if (!inserter) { if (on_conflict.empty()) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 83bb3c606c9..1a275320f43 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -147,7 +147,7 @@ class RedisSink : public SinkToStorage public: RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; String getName() const override { return "RedisSink"; } private: @@ -169,10 +169,10 @@ RedisSink::RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadat } } -void RedisSink::consume(Chunk chunk) +void RedisSink::consume(Chunk & chunk) { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -567,7 +567,8 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ Block block; while (executor.pull(block)) { - sink->consume(Chunk{block.getColumns(), block.rows()}); + Chunk chunk(block.getColumns(), block.rows()); + sink->consume(chunk); } } diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 179e4cee199..85417a2f2a4 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -141,7 +141,7 @@ public: String getName() const override { return "SQLiteSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString sqlbuf; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 5b7f9fc0ac2..0d094c15880 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -44,7 +44,7 @@ public: const String & backup_file_name_, bool persistent_); String getName() const override { return "SetOrJoinSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: @@ -82,9 +82,9 @@ SetOrJoinSink::SetOrJoinSink( { } -void SetOrJoinSink::consume(Chunk chunk) +void SetOrJoinSink::consume(Chunk & chunk) { - Block block = getHeader().cloneWithColumns(chunk.detachColumns()); + Block block = getHeader().cloneWithColumns(chunk.getColumns()); table.insertBlock(block, getContext()); if (persistent) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 8df87d6290f..9b6d9f041e1 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -226,9 +226,9 @@ public: } } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - block_out->write(getHeader().cloneWithColumns(chunk.detachColumns())); + block_out->write(getHeader().cloneWithColumns(chunk.getColumns())); } void onFinish() override diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 895da028fc2..90e05c44e31 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -565,12 +565,12 @@ StorageURLSink::StorageURLSink( } -void StorageURLSink::consume(Chunk chunk) +void StorageURLSink::consume(Chunk & chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void StorageURLSink::onCancel() diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fa7cc6eeeef..1804079e75f 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -251,7 +251,7 @@ public: const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onCancel() override; void onException(std::exception_ptr exception) override; void onFinish() override; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index cb46cd19517..c9c606de049 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -119,7 +119,7 @@ public: ZooKeeperSink(const Block & header, ContextPtr context) : SinkToStorage(header), zookeeper(context->getZooKeeper()) { } String getName() const override { return "ZooKeeperSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); size_t rows = block.rows(); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 77e6ee9cb24..e36247103c7 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -304,7 +305,7 @@ namespace public: explicit AddingAggregatedChunkInfoTransform(Block header) : ISimpleTransform(header, header, false) { } - void transform(Chunk & chunk) override { chunk.setChunkInfo(std::make_shared()); } + void transform(Chunk & chunk) override { chunk.getChunkInfos().add(std::make_shared()); } String getName() const override { return "AddingAggregatedChunkInfoTransform"; } }; @@ -689,7 +690,13 @@ inline void StorageWindowView::fire(UInt32 watermark) StoragePtr target_table = getTargetTable(); auto insert = std::make_shared(); insert->table_id = target_table->getStorageID(); - InterpreterInsertQuery interpreter(insert, getContext()); + InterpreterInsertQuery interpreter( + insert, + getContext(), + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto pipe = Pipe(std::make_shared(blocks, header)); @@ -1413,7 +1420,7 @@ void StorageWindowView::eventTimeParser(const ASTCreateQuery & query) } void StorageWindowView::writeIntoWindowView( - StorageWindowView & window_view, const Block & block, ContextPtr local_context) + StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) { window_view.throwIfWindowViewIsDisabled(local_context); while (window_view.modifying_query) @@ -1428,7 +1435,7 @@ void StorageWindowView::writeIntoWindowView( window_view.max_watermark = window_view.getWindowUpperBound(first_record_timestamp); } - Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); + Pipe pipe(std::make_shared(block)); UInt32 lateness_bound = 0; UInt32 t_max_watermark = 0; @@ -1473,10 +1480,10 @@ void StorageWindowView::writeIntoWindowView( auto syntax_result = TreeRewriter(local_context).analyze(query, columns); auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, local_context).getActionsDAG(false); - pipe.addSimpleTransform([&](const Block & header) + pipe.addSimpleTransform([&](const Block & header_) { return std::make_shared( - header, std::make_shared(filter_expression), + header_, std::make_shared(filter_expression), filter_function->getColumnName(), true); }); } @@ -1531,6 +1538,30 @@ void StorageWindowView::writeIntoWindowView( QueryProcessingStage::WithMergeableState); builder = select_block.buildQueryPipeline(); + + builder.addSimpleTransform([&](const Block & stream_header) + { + // Can't move chunk_infos here, that function could be called several times + return std::make_shared(chunk_infos.clone(), stream_header); + }); + + String window_view_id = window_view.getStorageID().hasUUID() ? toString(window_view.getStorageID().uuid) : window_view.getStorageID().getFullNameNotQuoted(); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(window_view_id, stream_header); + }); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header); + }); + +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Afrer tmp table before squashing", stream_header); + }); +#endif + builder.addSimpleTransform([&](const Block & current_header) { return std::make_shared( @@ -1570,6 +1601,13 @@ void StorageWindowView::writeIntoWindowView( lateness_upper_bound); }); +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Afrer WatermarkTransform", stream_header); + }); +#endif + auto inner_table = window_view.getInnerTable(); auto lock = inner_table->lockForShare( local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); @@ -1586,9 +1624,16 @@ void StorageWindowView::writeIntoWindowView( auto convert_actions = std::make_shared( convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, convert_actions); }); + builder.addSimpleTransform([&](const Block & header_) { return std::make_shared(header_, convert_actions); }); } +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Before out", stream_header); + }); +#endif + builder.addChain(Chain(std::move(output))); builder.setSinks([&](const Block & cur_header, Pipe::StreamType) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index f79867df424..14ac65091d3 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -166,7 +166,7 @@ public: BlockIO populate(); - static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, ContextPtr context); + static void writeIntoWindowView(StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); ASTPtr getMergeableQuery() const { return mergeable_query->clone(); } diff --git a/tests/integration/test_force_deduplication/test.py b/tests/integration/test_force_deduplication/test.py index 87b2c45bbc5..14c11bc8500 100644 --- a/tests/integration/test_force_deduplication/test.py +++ b/tests/integration/test_force_deduplication/test.py @@ -29,6 +29,8 @@ def get_counts(): def test_basic(start_cluster): + old_src, old_a, old_b, old_c = 0, 0, 0, 0 + node.query( """ CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple(); @@ -39,6 +41,15 @@ def test_basic(start_cluster): INSERT INTO test values(999); """ ) + + src, a, b, c = get_counts() + assert src == old_src + 1 + assert a == old_a + 2 + assert b == old_b + 2 + assert c == old_c + 2 + old_src, old_a, old_b, old_c = src, a, b, c + + # that issert fails on test_mv_b due to partitions by A with pytest.raises(QueryRuntimeException): node.query( """ @@ -46,22 +57,23 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(10); """ ) + src, a, b, c = get_counts() + assert src == old_src + 10 + assert a == old_a + 10 + assert b == old_b + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c - old_src, old_a, old_b, old_c = get_counts() - # number of rows in test_mv_a and test_mv_c depends on order of inserts into views - assert old_src == 11 - assert old_a in (1, 11) - assert old_b == 1 - assert old_c in (1, 11) - + # deduplication only for src table node.query("INSERT INTO test SELECT number FROM numbers(10)") src, a, b, c = get_counts() - # no changes because of deduplication in source table assert src == old_src - assert a == old_a - assert b == old_b - assert c == old_c + assert a == old_a + 10 + assert b == old_b + 10 + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for MV tables does not work, because previous inserts have not written their deduplications tokens to the log due to `deduplicate_blocks_in_dependent_materialized_views = 0`. node.query( """ SET deduplicate_blocks_in_dependent_materialized_views = 1; @@ -69,11 +81,27 @@ def test_basic(start_cluster): """ ) src, a, b, c = get_counts() - assert src == 11 - assert a == old_a + 10 # first insert could be succesfull with disabled dedup - assert b == 11 + assert src == old_src + assert a == old_a + 10 + assert b == old_b + 10 assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for all the tables + node.query( + """ + SET deduplicate_blocks_in_dependent_materialized_views = 1; + INSERT INTO test SELECT number FROM numbers(10); + """ + ) + src, a, b, c = get_counts() + assert src == old_src + assert a == old_a + assert b == old_b + assert c == old_c + old_src, old_a, old_b, old_c = src, a, b, c + + # that issert fails on test_mv_b due to partitions by A, it is an uniq data which is not deduplicated with pytest.raises(QueryRuntimeException): node.query( """ @@ -82,16 +110,23 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(100,10); """ ) + src, a, b, c = get_counts() + assert src == old_src + 10 + assert a == old_a + 10 + assert b == old_b + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for all tables, except test_mv_b. For test_mv_b it is an uniq data which is not deduplicated due to exception at previous insert node.query( """ SET deduplicate_blocks_in_dependent_materialized_views = 1; INSERT INTO test SELECT number FROM numbers(100,10); """ ) - src, a, b, c = get_counts() - assert src == 21 - assert a == old_a + 20 - assert b == 21 - assert c == old_c + 20 + assert src == old_src + assert a == old_a + assert b == old_b + 10 + assert c == old_c + old_src, old_a, old_b, old_c = src, a, b, c diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference index adf6abb7298..9c9281dc7e4 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference @@ -1,7 +1,7 @@ 2 3 -2 +3 3 1 diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index d3c4da86b41..51e6a513608 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -29,7 +29,7 @@ INSERT INTO without_deduplication VALUES (43); SELECT count() FROM with_deduplication; SELECT count() FROM without_deduplication; --- Implicit insert isn't deduplicated +-- Implicit insert isn't deduplicated, because deduplicate_blocks_in_dependent_materialized_views = 0 by default SELECT ''; SELECT countMerge(cnt) FROM with_deduplication_mv; SELECT countMerge(cnt) FROM without_deduplication_mv; diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index 1fb219108da..8f7d19028b0 100755 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -36,8 +36,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE c" echo ${CLICKHOUSE_CLIENT} --query "CREATE TABLE root (d UInt64) ENGINE = Null" ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW d (d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/d', '1') ORDER BY d AS SELECT * FROM root" -${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; -${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "SELECT * FROM d"; ${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE d" diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index a9801e3b910..dadf2f35e6e 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -137,7 +137,7 @@ select arrayUniq(thread_ids) from system.query_log where Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '0' and Settings['max_insert_threads'] = '16'; -5 +18 select count() from testX; 60 select count() from testXA; @@ -185,7 +185,7 @@ select arrayUniq(thread_ids) from system.query_log where Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and Settings['max_insert_threads'] = '16'; -5 +18 select count() from testX; 80 select count() from testXA; diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.sql b/tests/queries/0_stateless/01927_query_views_log_current_database.sql index ba42795333c..6287156daaf 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.sql +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.sql @@ -16,6 +16,7 @@ CREATE MATERIALIZED VIEW matview_b_to_c TO table_c AS SELECT SUM(a + sleepEachRo CREATE MATERIALIZED VIEW matview_join_d_e TO table_f AS SELECT table_d.a as a, table_e.count + sleepEachRow(0.000003) as count FROM table_d LEFT JOIN table_e ON table_d.a = table_e.a; -- ENABLE LOGS +SET parallel_view_processing=0; SET log_query_views=1; SET log_queries_min_type='QUERY_FINISH'; SET log_queries=1; diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference index e0cc8f0ce63..2d9f236ada9 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference @@ -1,8 +1,8 @@ -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent -18 18 9 18 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent -18 9 9 9 -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent -18 18 9 18 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 +18 36 27 36 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data +18 18 18 18 +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 +18 36 27 36 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data 18 18 18 18 diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql index fdd75b91b1f..465c8d6136c 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql @@ -1,6 +1,6 @@ -- Tags: long -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; drop table if exists test sync; drop table if exists test_mv_a sync; @@ -35,7 +35,7 @@ select (select sum(c) from test_mv_c where test='case1'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data'; set deduplicate_blocks_in_dependent_materialized_views=1; @@ -53,7 +53,7 @@ select (select sum(c) from test_mv_c where test='case2'); -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; set deduplicate_blocks_in_dependent_materialized_views=0; @@ -70,7 +70,7 @@ select (select sum(c) from test_mv_b where test='case3'), (select sum(c) from test_mv_c where test='case3'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data'; set deduplicate_blocks_in_dependent_materialized_views=1; diff --git a/tests/queries/0_stateless/02125_query_views_log.sql b/tests/queries/0_stateless/02125_query_views_log.sql index d2d19b76a1f..ba50902ebea 100644 --- a/tests/queries/0_stateless/02125_query_views_log.sql +++ b/tests/queries/0_stateless/02125_query_views_log.sql @@ -8,7 +8,7 @@ create table dst (key Int) engine=Null(); create materialized view mv1 to dst as select * from src; create materialized view mv2 to dst as select * from src; -insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=1; +insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=0; system flush logs; -- { echo } diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference index 335b55f05c8..07deb7c2565 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference @@ -10,13 +10,14 @@ 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 42 +2022-09-01 12:00:00 84 +2023-09-01 12:00:00 42 -- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1 -- Landing 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 42 +2022-09-01 12:00:00 84 2023-09-01 12:00:00 42 -- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 -- Landing (Agg/Replacing)MergeTree diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql index f206f0d7775..a2378fd8f67 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql @@ -54,8 +54,9 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view - 1st insert works for landing and mv tables - 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table - - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded + - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded, now that block is inserted because deduplicate_blocks_in_dependent_materialized_views=0 + Now it is fixed. */ SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 1000; @@ -97,7 +98,7 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view This is what happens now: - 1st insert works for landing and mv tables - - 2nd insert gets first block 20220901 deduplicated and second one inserted for landing and mv tables + - 2nd insert gets first block 20220901 deduplicated for landing and both rows are inserted for mv tables */ SET deduplicate_blocks_in_dependent_materialized_views = 1, max_insert_delayed_streams_for_parallel_write = 1000; diff --git a/tests/queries/0_stateless/03008_deduplication.python b/tests/queries/0_stateless/03008_deduplication.python new file mode 100644 index 00000000000..dd1058518c9 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication.python @@ -0,0 +1,657 @@ +#!/usr/bin/env python3 + +import os +import sys +import argparse +import string + + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + + +def __format(template, **params): + field_names = [v[1] for v in string.Formatter().parse(template) if v[1] is not None] + kv_args = {} + for field in field_names: + if field in params: + kv_args[field] = params[field] + else: + kv_args[field] = "" + + return template.format(**kv_args) + + +def instance_create_statement( + table_name, + table_columns, + table_keys, + table_engine, + with_deduplication, + no_merges=True, +): + template = """ + CREATE TABLE {table_name} + {table_columns} + ENGINE = {table_engine} + ORDER BY {table_keys} + {table_settings}; + {table_no_merges} + """ + + params = dict() + params["table_name"] = table_name + params["table_columns"] = table_columns + params["table_keys"] = table_keys + params["table_no_merges"] = f"SYSTEM STOP MERGES {table_name};" if no_merges else "" + params["table_engine"] = ( + "MergeTree()" + if table_engine == "MergeTree" + else f"ReplicatedMergeTree('/clickhouse/tables/{{database}}/{table_name}', '1')" + ) + + deduplication_window_setting_name = ( + "non_replicated_deduplication_window" + if table_engine == "MergeTree" + else "replicated_deduplication_window" + ) + deduplication_window_setting_value = 1000 if with_deduplication else 0 + + settings = list() + settings += [ + f"{deduplication_window_setting_name}={deduplication_window_setting_value}" + ] + params["table_settings"] = "SETTINGS " + ",".join(settings) + + return __format(template, **params) + + +def instance_insert_statement( + table_name, count, insert_method, insert_unique_blocks, use_insert_token +): + insert_settings = ( + "" if not use_insert_token else "SETTINGS insert_deduplication_token='UDT'" + ) + + if insert_method == "InsertSelect": + template = """ + INSERT INTO {table_name} + SELECT {insert_columns} + FROM numbers({count}) {insert_settings}; + """ + return __format( + template, + table_name=table_name, + count=count, + insert_columns="'src_4', 4" + if not insert_unique_blocks + else "'src_' || toString(number), number", + insert_settings=insert_settings, + ) + + else: + template = """ + INSERT INTO {table_name} + {insert_settings} VALUES {insert_values}; + """ + + values = [] + for i in range(count): + values += ( + [f"('src_{i}', {i})"] if insert_unique_blocks else ["('src_4', 4)"] + ) + insert_values = ", ".join(values) + + return __format( + template, + table_name=table_name, + insert_settings=insert_settings, + insert_values=insert_values, + ) + + +def get_drop_tables_statements(tables): + return "".join( + [f"DROP TABLE IF EXISTS {table_name};\n" for table_name in tables[::-1]] + ) + + +def get_logs_statement(args): + if args.get_logs: + return "SET send_logs_level='test';" + return "" + + +def str2bool(v): + if isinstance(v, bool): + return v + if v.lower() in ("yes", "true", "t", "y", "1"): + return True + elif v.lower() in ("no", "false", "f", "n", "0"): + return False + else: + raise argparse.ArgumentTypeError("Boolean value expected.") + + +class ArgsFactory: + def __init__(self, parser): + self.__parser = parser + + def add_opt_engine(self): + self.__parser.add_argument( + "--table-engine", + choices=["ReplicatedMergeTree", "MergeTree"], + default="MergeTree", + ) + + def add_opt_user_token(self): + self.__parser.add_argument( + "--use-insert-token", type=str2bool, nargs="?", const=True, default=False + ) + + def add_opt_single_thread(self): + self.__parser.add_argument( + "--single-thread", type=str2bool, nargs="?", const=True, default=True + ) + + def add_opt_dedup_src(self): + self.__parser.add_argument( + "--deduplicate-src-table", + type=str2bool, + nargs="?", + const=True, + default=True, + ) + + def add_opt_dedup_dst(self): + self.__parser.add_argument( + "--deduplicate-dst-table", + type=str2bool, + nargs="?", + const=True, + default=True, + ) + + def add_opt_get_logs(self): + self.__parser.add_argument( + "--get-logs", type=str2bool, nargs="?", const=True, default=False + ) + + def add_opt_uniq_blocks(self): + self.__parser.add_argument( + "--insert-unique-blocks", type=str2bool, nargs="?", const=True, default=True + ) + + def add_opt_insert_method(self): + self.__parser.add_argument( + "--insert-method", + choices=["InsertSelect", "InsertValues"], + default="InsertSelect", + ) + + def add_all(self): + self.add_opt_engine() + self.add_opt_user_token() + self.add_opt_single_thread() + self.add_opt_dedup_src() + self.add_opt_dedup_dst() + self.add_opt_get_logs() + self.add_opt_insert_method() + self.add_opt_uniq_blocks() + + +def test_insert_several_blocks(parser): + ArgsFactory(parser).add_all() + + def calle(args): + create_table_a_b_statement = instance_create_statement( + table_name="table_a_b", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_when_b_even_statement = instance_create_statement( + table_name="table_when_b_even", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + create_mv_statement = """ + CREATE MATERIALIZED VIEW mv_b_even + TO table_when_b_even + AS + SELECT a, b + FROM table_a_b + WHERE b % 2 = 0; + """ + + drop_tables_statements = get_drop_tables_statements( + ["table_a_b", "table_when_b_even", "mv_b_even"] + ) + + insert_statement = instance_insert_statement( + "table_a_b", + 10, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + print_details_statements = f""" + SELECT 'table_a_b'; + SELECT 'count', count() FROM table_a_b; + {"" if not args.get_logs else "SELECT _part, count() FROM table_a_b GROUP BY _part ORDER BY _part;"} + + SELECT 'table_when_b_even'; + SELECT 'count', count() FROM table_when_b_even; + {"" if not args.get_logs else "SELECT _part, count() FROM table_when_b_even GROUP BY _part ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 10 ) + FROM table_a_b; + SELECT throwIf( count() != 5 ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {5 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = """ + SELECT throwIf( count() != 10 ) + FROM table_a_b; + SELECT throwIf( count() != 10 ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + {create_table_a_b_statement} + + {create_table_when_b_even_statement} + + {create_mv_statement} + + -- first insert + {insert_statement} + + {print_details_statements} + + {assert_first_insert_statements} + + -- second insert, it is retry + {insert_statement} + + {print_details_statements} + + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def test_mv_generates_several_blocks(parser): + ArgsFactory(parser).add_all() + + def calle(args): + tables = [ + "table_for_join_with", + "table_a_b", + "table_when_b_even_and_joined", + "mv_b_even", + ] + drop_tables_statements = get_drop_tables_statements(tables) + + details_print_for_table_for_join_with = "" + if args.get_logs: + details_print_for_table_for_join_with = """ + SELECT 'table_for_join_with'; + SELECT a_join, b, _part FROM table_for_join_with ORDER BY _part, a_join, b; + """ + + create_table_a_b_statement = instance_create_statement( + table_name="table_a_b", + table_columns="(a_src String, b UInt64)", + table_keys="(a_src, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_when_b_even_and_joined_statement = instance_create_statement( + table_name="table_when_b_even_and_joined", + table_columns="(a_src String, a_join String, b UInt64)", + table_keys="(a_src, a_join, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + insert_statement = instance_insert_statement( + "table_a_b", + 5, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + details_print_statements = f""" + SELECT 'table_a_b'; + SELECT 'count', count() FROM table_a_b; + + SELECT 'table_when_b_even_and_joined'; + SELECT 'count', count() FROM table_when_b_even_and_joined; + {"" if not args.get_logs else "SELECT _part, a_src, a_join, b FROM table_when_b_even_and_joined ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 5 ) + FROM table_a_b; + + SELECT throwIf( count() != 9 ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {9 if args.deduplicate_dst_table else 18} ) + FROM table_when_b_even_and_joined; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 5} ) + FROM table_a_b; + + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even_and_joined; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 5} ) + FROM table_a_b; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even_and_joined; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + CREATE TABLE table_for_join_with + (a_join String, b UInt64) + ENGINE = MergeTree() + ORDER BY (a_join, b); + INSERT INTO table_for_join_with + SELECT 'joined_' || toString(number), number + FROM numbers(1); + {details_print_for_table_for_join_with} + + {create_table_a_b_statement} + SYSTEM STOP MERGES table_a_b; + + {create_table_when_b_even_and_joined_statement} + SYSTEM STOP MERGES table_when_b_even_and_joined; + + CREATE MATERIALIZED VIEW mv_b_even + TO table_when_b_even_and_joined + AS + SELECT a_src, a_join, table_for_join_with.b as b + FROM table_a_b + FULL OUTER JOIN table_for_join_with + ON table_a_b.b = table_for_join_with.b AND table_a_b.b % 2 = 0 + ORDER BY a_src, a_join, b; + + -- first insert + {insert_statement} + + {details_print_statements} + + -- first assertion + {assert_first_insert_statements} + + -- second insert + {insert_statement} + + {details_print_statements} + + -- second assertion + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def test_several_mv_into_one_table(parser): + ArgsFactory(parser).add_all() + + def calle(args): + tables = ["table_src", "table_dst", "mv_b_even", "mv_b_even_even"] + drop_tables_statements = get_drop_tables_statements(tables) + + create_table_src_statement = instance_create_statement( + table_name="table_src", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_dst_statement = instance_create_statement( + table_name="table_dst", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + insert_statement = instance_insert_statement( + "table_src", + 8, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + details_print_statements = f""" + SELECT 'table_src count', count() FROM table_src; + + SELECT 'table_dst count', count() FROM table_dst; + {"" if not args.get_logs else "SELECT _part, count() FROM table_dst GROUP BY _part ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 8 ) + FROM table_src; + + SELECT throwIf( count() != 6 ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {6 if args.deduplicate_dst_table else 12} ) + FROM table_dst; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 8} ) + FROM table_src; + + SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 16} ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 32} ) + FROM table_dst; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 8} ) + FROM table_src; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 16} ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 32} ) + FROM table_dst; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + {create_table_src_statement} + + {create_table_dst_statement} + + CREATE MATERIALIZED VIEW mv_b_even + TO table_dst + AS + SELECT a, b + FROM table_src + WHERE b % 2 = 0; + + CREATE MATERIALIZED VIEW mv_b_even_even + TO table_dst + AS + SELECT a, b + FROM table_src + WHERE b % 4 = 0; + + -- first insert + {insert_statement} + + {details_print_statements} + + {assert_first_insert_statements} + + -- second insert, retry + {insert_statement} + + {details_print_statements} + + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def parse_args(): + parser = argparse.ArgumentParser() + subparsers = parser.add_subparsers(dest="test") + test_insert_several_blocks( + subparsers.add_parser("insert_several_blocks_into_table") + ) + test_mv_generates_several_blocks( + subparsers.add_parser("mv_generates_several_blocks") + ) + test_several_mv_into_one_table(subparsers.add_parser("several_mv_into_one_table")) + args = parser.parse_args() + if args.test is None: + parser.print_help() + return args + + +def main(): + args = parse_args() + if args.test is not None: + args.func(args) + + +if __name__ == "__main__": + main() diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference new file mode 100644 index 00000000000..4893274c1cd --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference @@ -0,0 +1,41 @@ +Different materialized view insert into one underlayed table equal data. +first attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +second attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +Different insert operations generate the same data after transformation in underlied table of materialized view. +first attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +second attempt +from dst 1 A all_1_1_0 +from dst 2 A all_2_2_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +Indentical blocks in insertion with `insert_deduplication_token` +first attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +second attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +third attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +Indentical blocks in insertion +from dst 0 A all_1_1_0 +Indentical blocks after materialised view`s transformation +first attempt +from dst 1 B all_1_1_0 +from dst 2 B all_2_2_0 +from mv_dst 0 B all_1_1_0 +from mv_dst 0 B all_2_2_0 +second attempt +from dst 1 B all_1_1_0 +from dst 2 B all_2_2_0 +from mv_dst 0 B all_1_1_0 +from mv_dst 0 B all_2_2_0 diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql new file mode 100644 index 00000000000..7927a6b1edf --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql @@ -0,0 +1,331 @@ +-- ######### +select 'Different materialized view insert into one underlayed table equal data.'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; +DROP TABLE IF EXISTS mv_first; +DROP TABLE IF EXISTS mv_second; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE TABLE mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_first +TO mv_dst +AS SELECT + 0 AS key, + value AS value +FROM dst; + +CREATE MATERIALIZED VIEW mv_second +TO mv_dst +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_second; +DROP TABLE mv_first; +DROP TABLE mv_dst; +DROP TABLE dst; + + +-- ######### +select 'Different insert operations generate the same data after transformation in underlied table of materialized view.'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000 +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst VALUES (2, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_dst; +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks in insertion with `insert_deduplication_token`'; + +DROP TABLE IF EXISTS dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +select 'first attempt'; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +select 'third attempt'; + +INSERT INTO dst SELECT + 1 AS key, + 'b' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks in insertion'; + +DROP TABLE IF EXISTS dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks after materialised view`s transformation'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000 +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst SELECT + number + 1 AS key, + IF(key = 0, 'A', 'B') AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst SELECT + number + 1 AS key, + IF(key = 0, 'A', 'B') AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_dst; +DROP TABLE dst; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference new file mode 100644 index 00000000000..c82a6eaa213 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference @@ -0,0 +1,35 @@ +no user deduplication token +partitioned_table is deduplicated bacause deduplication works in scope of one partiotion: +1 A +1 D +2 B +2 C +mv_table is not deduplicated because the inserted blocks was different: +1 A +1 A +1 D +2 B +2 B +2 C +with user deduplication token +partitioned_table is not deduplicated because different tokens: +1 A +1 A +1 D +2 B +2 B +2 C +mv_table is not deduplicated because different tokens: +1 A +1 A +1 D +2 B +2 B +2 C +with incorrect ussage of user deduplication token +partitioned_table is deduplicated because equal tokens: +1 A +2 B +mv_table is deduplicated because equal tokens: +1 A +2 B diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql new file mode 100644 index 00000000000..2eb931f7f73 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql @@ -0,0 +1,83 @@ +DROP TABLE IF EXISTS partitioned_table; +DROP TABLE IF EXISTS mv_table; + + +SET deduplicate_blocks_in_dependent_materialized_views = 1; + + +SELECT 'no user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is deduplicated bacause deduplication works in scope of one partiotion:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is not deduplicated because the inserted blocks was different:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; + + +SELECT 'with user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_1' VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_2' VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_3' VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is not deduplicated because different tokens:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is not deduplicated because different tokens:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; + + +SELECT 'with incorrect ussage of user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is deduplicated because equal tokens:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is deduplicated because equal tokens:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference new file mode 100644 index 00000000000..bf900aa84d2 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh new file mode 100755 index 00000000000..49eb52b47fd --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference new file mode 100644 index 00000000000..c815324b455 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh new file mode 100755 index 00000000000..53af06d4a6f --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference new file mode 100644 index 00000000000..6e76ec46aa8 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh new file mode 100755 index 00000000000..7d4f5240cd1 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 20: engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference new file mode 100644 index 00000000000..a25e8713c61 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh new file mode 100755 index 00000000000..109d1674f3a --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 20: engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference new file mode 100644 index 00000000000..b6a3e0175a7 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference @@ -0,0 +1,706 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh new file mode 100755 index 00000000000..fe3d610a758 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference new file mode 100644 index 00000000000..1921103f49e --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference @@ -0,0 +1,706 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh new file mode 100755 index 00000000000..9adee6d53d4 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03035_max_insert_threads_support.sh b/tests/queries/0_stateless/03035_max_insert_threads_support.sh index 1e6bfb414d8..cedb651a430 100755 --- a/tests/queries/0_stateless/03035_max_insert_threads_support.sh +++ b/tests/queries/0_stateless/03035_max_insert_threads_support.sh @@ -8,7 +8,7 @@ DATA_FILE="data_$CLICKHOUSE_TEST_UNIQUE_NAME.csv" $CLICKHOUSE_CLIENT --max_insert_threads=4 --query=" EXPLAIN PIPELINE INSERT INTO FUNCTION file('$DATA_FILE') SELECT * FROM numbers_mt(1000000) ORDER BY number DESC -" | grep -o MaterializingTransform | wc -l +" | grep -o StorageFileSink | wc -l DATA_FILE_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path from file('$DATA_FILE', 'One')") rm $DATA_FILE_PATH From d2d72794a14459e60be1c3a88d39440af1f7a8f9 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 15:06:50 +0000 Subject: [PATCH 042/178] proper destruction order of AsyncLoader::Pool fields --- src/Common/AsyncLoader.cpp | 4 ++-- src/Common/AsyncLoader.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index cfb273b9058..6264eb03106 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -49,6 +49,7 @@ void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopw AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init) : name(init.name) , priority(init.priority) + , max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores()) , thread_pool(std::make_unique( init.metric_threads, init.metric_active_threads, @@ -56,17 +57,16 @@ AsyncLoader::Pool::Pool(const AsyncLoader::PoolInitializer & init) /* max_threads = */ std::numeric_limits::max(), // Unlimited number of threads, we do worker management ourselves /* max_free_threads = */ 0, // We do not require free threads /* queue_size = */0)) // Unlimited queue to avoid blocking during worker spawning - , max_threads(init.max_threads > 0 ? init.max_threads : getNumberOfPhysicalCPUCores()) {} AsyncLoader::Pool::Pool(Pool&& o) noexcept : name(o.name) , priority(o.priority) - , thread_pool(std::move(o.thread_pool)) , ready_queue(std::move(o.ready_queue)) , max_threads(o.max_threads) , workers(o.workers) , suspended_workers(o.suspended_workers.load()) // All these constructors are needed because std::atomic is neither copy-constructible, nor move-constructible. We never move pools after init, so it is safe. + , thread_pool(std::move(o.thread_pool)) {} void cancelOnDependencyFailure(const LoadJobPtr & self, const LoadJobPtr & dependency, std::exception_ptr & cancel) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 42707a4ee91..05b809aceae 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -365,11 +365,11 @@ private: { const String name; const Priority priority; - std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. std::map ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno` size_t max_threads; // Max number of workers to be spawn size_t workers = 0; // Number of currently executing workers std::atomic suspended_workers{0}; // Number of workers that are blocked by `wait()` call on a job executing in the same pool (for deadlock resolution) + std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. explicit Pool(const PoolInitializer & init); Pool(Pool&& o) noexcept; From 376472c8ceffe82a466067c9fc0039517726da62 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 5 Jul 2024 19:12:23 +0200 Subject: [PATCH 043/178] add test when chunk with 0 columns has to be produced in squashing --- src/Core/Settings.h | 2 +- src/Interpreters/Squashing.cpp | 36 ++++++++++++------- src/Interpreters/Squashing.h | 10 +++--- .../DeduplicationTokenTransforms.cpp | 4 ++- .../0_stateless/01275_parallel_mv.sql.j2 | 6 ++-- .../03008_deduplication_wrong_mv.reference | 15 ++++++++ .../03008_deduplication_wrong_mv.sql | 21 +++++++++++ 7 files changed, 71 insertions(+), 23 deletions(-) create mode 100644 tests/queries/0_stateless/03008_deduplication_wrong_mv.reference create mode 100644 tests/queries/0_stateless/03008_deduplication_wrong_mv.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 81d0aa0c51d..adf437100b1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -952,7 +952,7 @@ class IColumn; #define OBSOLETE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ - MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 1) \ + MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 0) \ MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \ MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \ MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \ diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 25434d1103e..a076494a4bc 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,5 +1,7 @@ #include #include +#include "Common/Logger.h" +#include "Common/logger_useful.h" #include #include @@ -16,6 +18,7 @@ Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_bloc , min_block_size_bytes(min_block_size_bytes_) , header(header_) { + LOG_TEST(getLogger("Squashing"), "header columns {}", header.columns()); } Chunk Squashing::flush() @@ -23,7 +26,7 @@ Chunk Squashing::flush() if (!accumulated) return {}; - auto result = convertToChunk(accumulated.extract()); + auto result = convertToChunk(extract()); chassert(result); return result; } @@ -43,6 +46,8 @@ Chunk Squashing::squash(Chunk && input_chunk) Chunk Squashing::add(Chunk && input_chunk) { + LOG_TEST(getLogger("Squashing"), "add columns {} rows {}", input_chunk.getNumColumns(), input_chunk.getNumRows()); + if (!input_chunk) return {}; @@ -53,11 +58,11 @@ Chunk Squashing::add(Chunk && input_chunk) if (!accumulated) { accumulated.add(std::move(input_chunk)); - return convertToChunk(accumulated.extract()); + return convertToChunk(extract()); } /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Chunk res_chunk = convertToChunk(accumulated.extract()); + Chunk res_chunk = convertToChunk(extract()); accumulated.add(std::move(input_chunk)); return res_chunk; } @@ -66,7 +71,7 @@ Chunk Squashing::add(Chunk && input_chunk) if (isEnoughSize()) { /// Return accumulated data and place new block to accumulated data. - Chunk res_chunk = convertToChunk(accumulated.extract()); + Chunk res_chunk = convertToChunk(extract()); accumulated.add(std::move(input_chunk)); return res_chunk; } @@ -76,21 +81,25 @@ Chunk Squashing::add(Chunk && input_chunk) /// If accumulated data is big enough, we send it if (isEnoughSize()) - return convertToChunk(accumulated.extract()); + return convertToChunk(extract()); return {}; } -Chunk Squashing::convertToChunk(std::vector && chunks) const +Chunk Squashing::convertToChunk(CurrentData && data) const { - if (chunks.empty()) + LOG_TEST(getLogger("Squashing"), "convertToChunk {}", data.chunks.size()); + + if (data.chunks.empty()) return {}; auto info = std::make_shared(); - info->chunks = std::move(chunks); + info->chunks = std::move(data.chunks); // It is imortant that chunk is not empty, it has to have columns even if they are empty - auto aggr_chunk = Chunk(header.getColumns(), 0); + // Sometimes there are could be no columns in header but not empty rows in chunks + // That happens when we intend to add defaults for the missing columns after + auto aggr_chunk = Chunk(header.getColumns(), header.columns() ? 0 : data.getRows()); aggr_chunk.getChunkInfos().add(std::move(info)); chassert(aggr_chunk); return aggr_chunk; @@ -149,17 +158,18 @@ bool Squashing::isEnoughSize(const Chunk & chunk) const return isEnoughSize(chunk.getNumRows(), chunk.bytes()); } -void Squashing::CurrentSize::add(Chunk && chunk) +void Squashing::CurrentData::add(Chunk && chunk) { rows += chunk.getNumRows(); bytes += chunk.bytes(); chunks.push_back(std::move(chunk)); } -std::vector Squashing::CurrentSize::extract() +Squashing::CurrentData Squashing::extract() { - auto result = std::move(chunks); - *this = {}; + auto result = std::move(accumulated); + accumulated = {}; return result; } + } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 64a9768a71f..71ed4c4185a 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -49,25 +49,23 @@ public: const Block & getHeader() const { return header; } private: - class CurrentSize + struct CurrentData { std::vector chunks = {}; size_t rows = 0; size_t bytes = 0; - public: explicit operator bool () const { return !chunks.empty(); } size_t getRows() const { return rows; } size_t getBytes() const { return bytes; } void add(Chunk && chunk); - std::vector extract(); }; const size_t min_block_size_rows; const size_t min_block_size_bytes; Block header; - CurrentSize accumulated; + CurrentData accumulated; static Chunk squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos); @@ -75,7 +73,9 @@ private: bool isEnoughSize(size_t rows, size_t bytes) const; bool isEnoughSize(const Chunk & chunk) const; - Chunk convertToChunk(std::vector && chunks) const; + CurrentData extract(); + + Chunk convertToChunk(CurrentData && data) const; }; } diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp index 6786f76cbef..e6f7e44e026 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -148,9 +148,11 @@ void CheckTokenTransform::transform(Chunk & chunk) auto token_info = chunk.getChunkInfos().get(); if (!token_info) + { throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk has to have DedupTokenInfo as ChunkInfo, {}", debug); + } - LOG_DEBUG(log, "debug: {}, token: {}", debug, token_info->debugToken()); + LOG_TEST(log, "debug: {}, token: {}, columns {} rows {}", debug, token_info->debugToken(), chunk.getNumColumns(), chunk.getNumRows()); } #endif diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 index 9d74474c1a4..bc663bd0e63 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 +++ b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 @@ -12,9 +12,9 @@ drop table if exists testXC; create table testX (A Int64) engine=MergeTree order by tuple(); -create materialized view testXA engine=MergeTree order by tuple() as select sleep(0.1) from testX; -create materialized view testXB engine=MergeTree order by tuple() as select sleep(0.2), throwIf(A=1) from testX; -create materialized view testXC engine=MergeTree order by tuple() as select sleep(0.1) from testX; +create materialized view testXA engine=MergeTree order by tuple() as select sleepEachRow(0.2) from testX; +create materialized view testXB engine=MergeTree order by tuple() as select sleepEachRow(0.4), throwIf(A=1) from testX; +create materialized view testXC engine=MergeTree order by tuple() as select sleepEachRow(0.2) from testX; -- { echoOn } {% for parallel_view_processing in [0, 1] %} diff --git a/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference b/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference new file mode 100644 index 00000000000..89b4f324d5e --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference @@ -0,0 +1,15 @@ +-- { echo ON } +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv1 TO dst AS SELECT * FROM src; +INSERT INTO src VALUES (0); +SELECT * from dst; +0 +TRUNCATE TABLE dst; +--DROP TABLE src SYNC; +--CREATE TABLE src (y String) ENGINE = MergeTree order by tuple(); +ALTER TABLE src ADD COLUMN y UInt8; +ALTER TABLE src DROP COLUMN x; +INSERT INTO src VALUES (0); +SELECT * from dst; +0 diff --git a/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql b/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql new file mode 100644 index 00000000000..dd74c38ab17 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS mv; +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; + +-- { echo ON } +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv1 TO dst AS SELECT * FROM src; + +INSERT INTO src VALUES (0); +SELECT * from dst; + +TRUNCATE TABLE dst; + +--DROP TABLE src SYNC; +--CREATE TABLE src (y String) ENGINE = MergeTree order by tuple(); +ALTER TABLE src ADD COLUMN y UInt8; +ALTER TABLE src DROP COLUMN x; + +INSERT INTO src VALUES (0); +SELECT * from dst; From d821dfe9033357ae6038f3f7b16d87fd91e05da9 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 6 Jul 2024 15:03:13 +0000 Subject: [PATCH 044/178] Fix tests --- src/Interpreters/getColumnFromBlock.cpp | 2 +- tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/getColumnFromBlock.cpp b/src/Interpreters/getColumnFromBlock.cpp index 2e70a58b5a1..89166bb2b3e 100644 --- a/src/Interpreters/getColumnFromBlock.cpp +++ b/src/Interpreters/getColumnFromBlock.cpp @@ -40,7 +40,7 @@ ColumnPtr tryGetSubcolumnFromBlock(const Block & block, const DataTypePtr & requ auto subcolumn_name = requested_subcolumn.getSubcolumnName(); /// If requested subcolumn is dynamic, we should first perform cast and then /// extract the subcolumn, because the data of dynamic subcolumn can change after cast. - if (elem->type->hasDynamicSubcolumns() && !elem->type->equals(*requested_column_type)) + if ((elem->type->hasDynamicSubcolumns() || requested_column_type->hasDynamicSubcolumns()) && !elem->type->equals(*requested_column_type)) { auto casted_column = castColumn({elem->column, elem->type, ""}, requested_column_type); auto elem_column = requested_column_type->tryGetSubcolumn(subcolumn_name, casted_column); diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh index 65517061b99..35d4f3a02cf 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 optimize_functions_to_subcolumns=0" function test() From fc3ae09a11c45346f18ec39094adc448dcb9ec45 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 6 Jul 2024 21:24:56 +0200 Subject: [PATCH 045/178] Fix typo in the test --- tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh index 35d4f3a02cf..a1cb35e4609 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 optimize_functions_to_subcolumns=0" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --optimize_functions_to_subcolumns=0" function test() From 30bb03d6c327d15c7161ca18b35b9edf04310de4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Jul 2024 11:13:44 +0200 Subject: [PATCH 046/178] Update test --- tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh index a1cb35e4609..65517061b99 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --optimize_functions_to_subcolumns=0" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" function test() From 7ab4af85e5d5bea114f1c74dd5cbe4b5a6176772 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jul 2024 12:36:02 +0200 Subject: [PATCH 047/178] Delete flaky case from 02956_rocksdb_bulk_sink --- tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh | 7 ------- 1 file changed, 7 deletions(-) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh index f7111d0afe2..4e6e123bba2 100755 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -25,13 +25,6 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 F ${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" -# Testing insert with multiple sinks and fixed block size -${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" -# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks -${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '', optimize_trivial_insert_select = 1;" -${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 2 because default bulk sink size is ~1M rows / SST file -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" - # Testing insert with duplicated keys ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1, optimize_trivial_insert_select = 1;" From 1e924768f95250acbfeed67841e40f7ec2fcf9c5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 8 Jul 2024 16:14:51 +0200 Subject: [PATCH 048/178] fix 02956_rocksdb_bulk_sink.reference --- tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index 74c71827e6e..2b887148ffb 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -3,7 +3,6 @@ 1000 1 1000 -2 1000000 1000 0 999001 From 0f38756f4e02061ddb4c144c2d80e574ccb97cbd Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 8 Jul 2024 16:33:23 +0200 Subject: [PATCH 049/178] Disable optimize_functions_to_subcolumns in 03036_dynamic_read_subcolumns --- tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh index 65517061b99..a1cb35e4609 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --optimize_functions_to_subcolumns=0" function test() From 22154aa079132184183659ebee80abcc855be403 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Jul 2024 14:36:31 +0000 Subject: [PATCH 050/178] Disable optimize_functions_to_subcolumns in 03202_dynamic_null_map_subcolumn --- tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh index aa06e48376c..d357419f03a 100755 --- a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 optimize_functions_to_subcolumns=0" function test() From 8035582169ae0666a3e024d9329d98e4c3a9b7a9 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 14:57:40 +0200 Subject: [PATCH 051/178] adjust test 01275_parallel_mv --- src/Interpreters/InterpreterInsertQuery.cpp | 24 +++++++------------ .../0_stateless/01275_parallel_mv.sql.j2 | 19 +++++++++------ 2 files changed, 21 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 2cbfc55d008..636db546a79 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -600,16 +601,15 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & /// Otherwise ResizeProcessor them down to 1 stream. size_t presink_streams_size = std::max(settings.max_insert_threads, pipeline.getNumStreams()); + if (settings.max_insert_threads.changed) + presink_streams_size = std::max(1, settings.max_insert_threads); size_t sink_streams_size = table->supportsParallelInsert() ? std::max(1, settings.max_insert_threads) : 1; - if (!settings.parallel_view_processing) + size_t views_involved = table->isView() || DatabaseCatalog::instance().getDependentViews(table->getStorageID()).size() > 0; + if (!settings.parallel_view_processing && views_involved) { - auto table_id = table->getStorageID(); - auto views = DatabaseCatalog::instance().getDependentViews(table_id); - - if (table->isView() || !views.empty()) - sink_streams_size = 1; + sink_streams_size = 1; } auto [presink_chains, sink_chains] = buildPreAndSinkChains( @@ -639,20 +639,12 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & pipeline.addResources(chain.detachResources()); pipeline.addChains(std::move(sink_chains)); - if (!settings.parallel_view_processing) + if (!settings.parallel_view_processing && views_involved) { /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. if (pipeline.getNumThreads() > num_select_threads) pipeline.setMaxThreads(num_select_threads); } - else if (pipeline.getNumThreads() < settings.max_threads) - { - /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, - /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. - /// - /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. - pipeline.setMaxThreads(settings.max_threads); - } pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr { @@ -794,6 +786,8 @@ BlockIO InterpreterInsertQuery::execute() if (const auto * mv = dynamic_cast(table.get())) res.pipeline.addStorageHolder(mv->getTargetTable()); + LOG_TEST(getLogger("InterpreterInsertQuery"), "Pipeline could use up to {} thread", res.pipeline.getNumThreads()); + return res; } diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 index bc663bd0e63..4e45c68b5ad 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 +++ b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 @@ -3,7 +3,12 @@ -- no-parallel: it checks the number of threads, which can be lowered in presence of other queries -- avoid settings randomization by clickhouse-test -set max_threads = 0; +set max_threads = 10; + + +-- more blocks to process +set max_block_size = 10; +set min_insert_block_size_rows = 10; drop table if exists testX; drop table if exists testXA; @@ -12,23 +17,23 @@ drop table if exists testXC; create table testX (A Int64) engine=MergeTree order by tuple(); -create materialized view testXA engine=MergeTree order by tuple() as select sleepEachRow(0.2) from testX; -create materialized view testXB engine=MergeTree order by tuple() as select sleepEachRow(0.4), throwIf(A=1) from testX; -create materialized view testXC engine=MergeTree order by tuple() as select sleepEachRow(0.2) from testX; +create materialized view testXA engine=MergeTree order by tuple() as select sleep(0.1) from testX; +create materialized view testXB engine=MergeTree order by tuple() as select sleep(0.2), throwIf(A=1) from testX; +create materialized view testXC engine=MergeTree order by tuple() as select sleep(0.1) from testX; -- { echoOn } {% for parallel_view_processing in [0, 1] %} {% for optimize_trivial_insert_select in [0, 1] %} -{% for max_insert_threads in [0, 16] %} +{% for max_insert_threads in [0, 5] %} select 'optimize_trivial_insert_select={{ optimize_trivial_insert_select }}', 'max_insert_threads={{ max_insert_threads }}'; -insert into testX select number from numbers(10) settings +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing={{ parallel_view_processing }}, optimize_trivial_insert_select={{ optimize_trivial_insert_select }}, max_insert_threads={{ max_insert_threads }}; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and From 7cf38826afa53deccee9aeb904d98bf98ae78d20 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 8 Jul 2024 17:48:49 +0200 Subject: [PATCH 052/178] Fix detection of number of CPUs in containers In the case when the 'parent' cgroup was used (i.e. name of cgroup was empty, which is common for containers) ClickHouse was ignoring the CPU limits set for the container. --- base/base/cgroupsv2.cpp | 17 ++++++++++------- base/base/cgroupsv2.h | 6 +++--- base/base/getMemoryAmount.cpp | 6 ++++-- src/Common/CgroupsMemoryUsageObserver.cpp | 6 ++++-- src/Common/getNumberOfPhysicalCPUCores.cpp | 10 +++++----- 5 files changed, 26 insertions(+), 19 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index f20b9daf22e..466ebbc3ffb 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -33,8 +33,9 @@ bool cgroupsV2MemoryControllerEnabled() /// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available /// for the current + child cgroups. The set of available controllers can be restricted from level to level using file /// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file. - std::string cgroup = cgroupV2OfProcess(); - auto cgroup_dir = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup); + auto cgroup_dir = currentCGroupV2Path(); + if (cgroup_dir.empty()) + return false; std::ifstream controllers_file(cgroup_dir / "cgroup.controllers"); if (!controllers_file.is_open()) return false; @@ -46,7 +47,7 @@ bool cgroupsV2MemoryControllerEnabled() #endif } -std::string cgroupV2OfProcess() +std::filesystem::path currentCGroupV2Path() { #if defined(OS_LINUX) chassert(cgroupsV2Enabled()); @@ -54,17 +55,19 @@ std::string cgroupV2OfProcess() /// A simpler way to get the membership is: std::ifstream cgroup_name_file("/proc/self/cgroup"); if (!cgroup_name_file.is_open()) - return ""; + return {}; /// With cgroups v2, there will be a *single* line with prefix "0::/" /// (see https://docs.kernel.org/admin-guide/cgroup-v2.html) std::string cgroup; std::getline(cgroup_name_file, cgroup); static const std::string v2_prefix = "0::/"; if (!cgroup.starts_with(v2_prefix)) - return ""; + return {}; + + // the 'root' cgroup can have empty path, which is valid cgroup = cgroup.substr(v2_prefix.length()); - return cgroup; + return default_cgroups_mount / cgroup; #else - return ""; + return {}; #endif } diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 70219d87cd1..2c58682ce31 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -16,7 +16,7 @@ bool cgroupsV2Enabled(); /// Assumes that cgroupsV2Enabled() is enabled. bool cgroupsV2MemoryControllerEnabled(); -/// Which cgroup does the process belong to? -/// Returns an empty string if the cgroup cannot be determined. +/// Detects which cgroup the process belong and returns the path to it in sysfs (for cgroups v2). +/// Returns an empty path if the cgroup cannot be determined. /// Assumes that cgroupsV2Enabled() is enabled. -std::string cgroupV2OfProcess(); +std::filesystem::path currentCGroupV2Path(); diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index f47cba9833d..9bd5ad75445 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -23,8 +23,10 @@ std::optional getCgroupsV2MemoryLimit() if (!cgroupsV2MemoryControllerEnabled()) return {}; - std::string cgroup = cgroupV2OfProcess(); - auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup); + auto current_cgroup = currentCGroupV2Path(); + + if (current_cgroup.empty()) + return {}; /// Open the bottom-most nested memory limit setting file. If there is no such file at the current /// level, try again at the parent level as memory settings are inherited. diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index d36c7fd08aa..e034319b21f 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -129,8 +129,10 @@ std::optional getCgroupsV2Path() if (!cgroupsV2MemoryControllerEnabled()) return {}; - String cgroup = cgroupV2OfProcess(); - auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup); + auto current_cgroup = currentCGroupV2Path(); + + if (current_cgroup.empty()) + return {}; /// Return the bottom-most nested current memory file. If there is no such file at the current /// level, try again at the parent level as memory settings are inherited. diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 7e18a93e6ed..b16c635f23e 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -37,12 +37,12 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) /// cgroupsv2 if (cgroupsV2Enabled()) { - /// First, we identify the cgroup the process belongs - std::string cgroup = cgroupV2OfProcess(); - if (cgroup.empty()) + /// First, we identify the path of the cgroup the process belongs + auto cgroup_path = currentCGroupV2Path(); + if (cgroup_path.empty()) return default_cpu_count; - auto current_cgroup = cgroup.empty() ? default_cgroups_mount : (default_cgroups_mount / cgroup); + auto current_cgroup = cgroup_path; // Looking for cpu.max in directories from the current cgroup to the top level // It does not stop on the first time since the child could have a greater value than parent @@ -62,7 +62,7 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) } current_cgroup = current_cgroup.parent_path(); } - current_cgroup = default_cgroups_mount / cgroup; + current_cgroup = cgroup_path; // Looking for cpuset.cpus.effective in directories from the current cgroup to the top level while (current_cgroup != default_cgroups_mount.parent_path()) { From b4e9e410e30a02799f446e635f4c9d0f0da76913 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 15:00:23 +0200 Subject: [PATCH 053/178] fix peak_threads_usage --- src/Interpreters/ThreadStatusExt.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 9ca521a4ab3..6ec6a64b13d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -233,7 +233,8 @@ void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) { /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; - thread_group->linkThread(thread_id); + if (!internal_thread) + thread_group->linkThread(thread_id); performance_counters.setParent(&thread_group->performance_counters); memory_tracker.setParent(&thread_group->memory_tracker); @@ -269,7 +270,8 @@ void ThreadStatus::detachFromGroup() /// Extract MemoryTracker out from query and user context memory_tracker.setParent(&total_memory_tracker); - thread_group->unlinkThread(); + if (!internal_thread) + thread_group->unlinkThread(); thread_group.reset(); From 898260c45cfb03d715f3e5a38c4298c399a9cfd0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 8 Jul 2024 21:16:05 +0200 Subject: [PATCH 054/178] fix style --- src/Processors/Transforms/SquashingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 605c6afcd00..490a57d4e23 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -8,8 +8,7 @@ namespace DB namespace ErrorCodes { -extern const int LOGICAL_ERROR; -extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } SquashingTransform::SquashingTransform( From 859c63298ef044d92c585921579f90dd6e56deda Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Jul 2024 10:51:40 +0000 Subject: [PATCH 055/178] upd 02956_rocksdb_bulk_sink --- .../queries/0_stateless/02956_rocksdb_bulk_sink.reference | 1 + tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference index 2b887148ffb..1f140df1d6b 100644 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.reference @@ -3,6 +3,7 @@ 1000 1 1000 +1 1000000 1000 0 999001 diff --git a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh index 4e6e123bba2..95c136584f0 100755 --- a/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh +++ b/tests/queries/0_stateless/02956_rocksdb_bulk_sink.sh @@ -25,6 +25,13 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 F ${CLICKHOUSE_CLIENT} --query "SELECT sum(value) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be 1 ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" +# Testing insert with multiple sinks and fixed block size +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" +# Must set both max_threads and max_insert_threads to 2 to make sure there is only two sinks +${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number, number+1 FROM numbers_mt(1000000) SETTINGS max_threads = 2, max_insert_threads = 2, max_block_size = 10000, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, insert_deduplication_token = '', optimize_trivial_insert_select = 1;" +${CLICKHOUSE_CLIENT} --query "SELECT sum(value) IN (1, 2) FROM system.rocksdb WHERE database = currentDatabase() AND table = 'rocksdb_worm' AND name = 'no.file.opens';" # should be not more than 2 because default bulk sink size is ~1M rows / SST file. +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM rocksdb_worm;" + # Testing insert with duplicated keys ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE rocksdb_worm;" ${CLICKHOUSE_CLIENT} --query "INSERT INTO rocksdb_worm SELECT number % 1000, number+1 FROM numbers_mt(1000000) SETTINGS max_block_size = 100000, max_insert_threads = 1, optimize_trivial_insert_select = 1;" From 751ee04e75a75629ba9b939d02070d2444afcab7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 9 Jul 2024 13:10:04 +0200 Subject: [PATCH 056/178] fix tidy build, canonize 01275_parallel_mv.reference --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../0_stateless/01275_parallel_mv.reference | 124 +++++++++--------- 2 files changed, 63 insertions(+), 63 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 636db546a79..6fd53070f41 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -606,7 +606,7 @@ QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & size_t sink_streams_size = table->supportsParallelInsert() ? std::max(1, settings.max_insert_threads) : 1; - size_t views_involved = table->isView() || DatabaseCatalog::instance().getDependentViews(table->getStorageID()).size() > 0; + size_t views_involved = table->isView() || !DatabaseCatalog::instance().getDependentViews(table->getStorageID()).empty(); if (!settings.parallel_view_processing && views_involved) { sink_streams_size = 1; diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index dadf2f35e6e..221d0b5ce8d 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -4,13 +4,13 @@ select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; optimize_trivial_insert_select=0 max_insert_threads=0 -insert into testX select number from numbers(10) settings +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -19,46 +19,46 @@ select arrayUniq(thread_ids) from system.query_log where Settings['max_insert_threads'] = '0'; 2 select count() from testX; -10 +200 select count() from testXA; -10 +200 select count() from testXB; 0 select count() from testXC; -10 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=16'; -optimize_trivial_insert_select=0 max_insert_threads=16 -insert into testX select number from numbers(10) settings +200 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5'; +optimize_trivial_insert_select=0 max_insert_threads=5 +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0' and Settings['optimize_trivial_insert_select'] = '0' and - Settings['max_insert_threads'] = '16'; + Settings['max_insert_threads'] = '5'; 2 select count() from testX; -20 +400 select count() from testXA; -20 +400 select count() from testXB; 0 select count() from testXC; -20 +400 select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; optimize_trivial_insert_select=1 max_insert_threads=0 -insert into testX select number from numbers(10) settings +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -67,46 +67,46 @@ select arrayUniq(thread_ids) from system.query_log where Settings['max_insert_threads'] = '0'; 2 select count() from testX; -30 +600 select count() from testXA; -30 +600 select count() from testXB; 0 select count() from testXC; -30 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=16'; -optimize_trivial_insert_select=1 max_insert_threads=16 -insert into testX select number from numbers(10) settings +600 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5'; +optimize_trivial_insert_select=1 max_insert_threads=5 +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '0' and Settings['optimize_trivial_insert_select'] = '1' and - Settings['max_insert_threads'] = '16'; + Settings['max_insert_threads'] = '5'; 2 select count() from testX; -40 +800 select count() from testXA; -40 +800 select count() from testXB; 0 select count() from testXC; -40 +800 select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; optimize_trivial_insert_select=0 max_insert_threads=0 -insert into testX select number from numbers(10) settings +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and @@ -115,82 +115,82 @@ select arrayUniq(thread_ids) from system.query_log where Settings['max_insert_threads'] = '0'; 5 select count() from testX; -50 +1000 select count() from testXA; -50 +1000 select count() from testXB; 0 select count() from testXC; -50 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=16'; -optimize_trivial_insert_select=0 max_insert_threads=16 -insert into testX select number from numbers(10) settings +1000 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5'; +optimize_trivial_insert_select=0 max_insert_threads=5 +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '0' and - Settings['max_insert_threads'] = '16'; -18 + Settings['max_insert_threads'] = '5'; +12 select count() from testX; -60 +1190 select count() from testXA; -60 +1130 select count() from testXB; -0 -select count() from testXC; 60 +select count() from testXC; +1130 select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; optimize_trivial_insert_select=1 max_insert_threads=0 -insert into testX select number from numbers(10) settings +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and Settings['max_insert_threads'] = '0'; -5 +2 select count() from testX; -70 +1390 select count() from testXA; -70 +1330 select count() from testXB; -0 +60 select count() from testXC; -70 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=16'; -optimize_trivial_insert_select=1 max_insert_threads=16 -insert into testX select number from numbers(10) settings +1330 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5'; +optimize_trivial_insert_select=1 max_insert_threads=5 +insert into testX select number from numbers(200) settings log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } system flush logs; -select arrayUniq(thread_ids) from system.query_log where +select peak_threads_usage from system.query_log where current_database = currentDatabase() and type != 'QueryStart' and query like '%insert into testX %' and Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and - Settings['max_insert_threads'] = '16'; -18 + Settings['max_insert_threads'] = '5'; +7 select count() from testX; -80 +1590 select count() from testXA; -80 +1480 select count() from testXB; -0 +160 select count() from testXC; -80 +1490 From 5daa28d5c5726478cbd635ce0ca6b8ce77f5fdce Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 9 Jul 2024 13:27:58 +0200 Subject: [PATCH 057/178] mark heavy tests with no-asan tag, they are slow with asan and asuze --- .../03008_deduplication_insert_several_blocks_nonreplicated.sh | 2 +- .../03008_deduplication_insert_several_blocks_replicated.sh | 2 +- ...8_deduplication_mv_generates_several_blocks_nonreplicated.sh | 2 +- ...3008_deduplication_mv_generates_several_blocks_replicated.sh | 2 +- ...008_deduplication_several_mv_into_one_table_nonreplicated.sh | 2 +- .../03008_deduplication_several_mv_into_one_table_replicated.sh | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh index 49eb52b47fd..0791c7566f9 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh index 53af06d4a6f..661dfa5f930 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh index 7d4f5240cd1..10083506af4 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh index 109d1674f3a..0d5158d18cd 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh index fe3d610a758..a9a135d6839 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh index 9adee6d53d4..49d556e70f9 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel +# Tags: long, no-fasttest, no-parallel, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From b60151c1d8e8e2d0fe5ea0946bdbdbfae9abecc1 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 10 Jul 2024 10:13:49 +0000 Subject: [PATCH 058/178] Remove optimize_functions_to_subcolumns setting from test --- tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh index d357419f03a..aa06e48376c 100755 --- a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 optimize_functions_to_subcolumns=0" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" function test() From 678c472236f8479455cdfcd697ed2a822f68a6b4 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Wed, 10 Jul 2024 14:42:13 +0300 Subject: [PATCH 059/178] Close log files in watchdog --- src/Daemon/BaseDaemon.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 48f76769a09..f82f02c9d9d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1305,6 +1305,10 @@ void BaseDaemon::setupWatchdog() int status = 0; do { + // Close log files to prevent keeping descriptors of unlinked rotated files. + // On next log write files will be reopened. + closeLogs(logger()); + if (-1 != waitpid(pid, &status, WUNTRACED | WCONTINUED) || errno == ECHILD) { if (WIFSTOPPED(status)) From 1846705497018b01b48b7ffaea353537edfb5b67 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Wed, 10 Jul 2024 17:57:09 +0800 Subject: [PATCH 060/178] fix the bug that LogicalExpressionOptimizerPass lost logical type of constant --- src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 698602ca5bc..bd8b6f9faa1 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -11,6 +11,7 @@ #include #include +#include namespace DB { @@ -615,6 +616,7 @@ private: bool is_any_nullable = false; Tuple args; args.reserve(equals_functions.size()); + DataTypes tuple_element_types; /// first we create tuple from RHS of equals functions for (const auto & equals : equals_functions) { @@ -627,16 +629,18 @@ private: if (const auto * rhs_literal = equals_arguments[1]->as()) { args.push_back(rhs_literal->getValue()); + tuple_element_types.push_back(rhs_literal->getResultType()); } else { const auto * lhs_literal = equals_arguments[0]->as(); assert(lhs_literal); args.push_back(lhs_literal->getValue()); + tuple_element_types.push_back(lhs_literal->getResultType()); } } - auto rhs_node = std::make_shared(std::move(args)); + auto rhs_node = std::make_shared(std::move(args), std::make_shared(std::move(tuple_element_types))); auto in_function = std::make_shared("in"); From 1ac2933d7d079ea31767da6304fa22712d38d6f4 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Wed, 10 Jul 2024 22:12:15 +0800 Subject: [PATCH 061/178] add test --- ...03203_optimize_disjunctions_chain_to_in.reference | 2 ++ .../03203_optimize_disjunctions_chain_to_in.sql | 12 ++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference create mode 100644 tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql diff --git a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference new file mode 100644 index 00000000000..353c8f98b03 --- /dev/null +++ b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference @@ -0,0 +1,2 @@ +2020-01-01 +2020-01-02 diff --git a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql new file mode 100644 index 00000000000..e48138ab990 --- /dev/null +++ b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql @@ -0,0 +1,12 @@ +SET allow_experimental_analyzer=1; +CREATE OR REPLACE TABLE foo (i Date) ENGINE MergeTree ORDER BY i; +INSERT INTO foo VALUES ('2020-01-01'); +INSERT INTO foo VALUES ('2020-01-02'); + +SET optimize_min_equality_disjunction_chain_length = 3; +SELECT * +FROM foo +WHERE (foo.i = parseDateTimeBestEffort('2020-01-01')) + OR (foo.i = parseDateTimeBestEffort('2020-01-02')) + OR (foo.i = parseDateTimeBestEffort('2020-01-03')) +ORDER BY foo.i ASC From faa0cb93fb524c14155e3ab409a099654cec8b91 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Wed, 10 Jul 2024 23:37:27 +0800 Subject: [PATCH 062/178] Empty commit From 1fb0bdf51093d3482950f1cfd9d1ad0bb727a69e Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Thu, 11 Jul 2024 01:00:49 +0800 Subject: [PATCH 063/178] fix test failed --- .../0_stateless/03203_optimize_disjunctions_chain_to_in.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql index e48138ab990..f9ba28bcd60 100644 --- a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql +++ b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql @@ -1,5 +1,5 @@ SET allow_experimental_analyzer=1; -CREATE OR REPLACE TABLE foo (i Date) ENGINE MergeTree ORDER BY i; +CREATE TABLE foo (i Date) ENGINE MergeTree ORDER BY i; INSERT INTO foo VALUES ('2020-01-01'); INSERT INTO foo VALUES ('2020-01-02'); From 2d1f45bb985debfb20037cb5f7dba6fb4f6903d8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 11 Jul 2024 08:32:00 +0000 Subject: [PATCH 064/178] Remove redundant code --- src/Client/IConnections.h | 2 -- src/Client/MultiplexedConnections.cpp | 11 +++++------ src/Interpreters/ClientInfo.cpp | 4 ++-- src/Interpreters/ClientInfo.h | 2 +- src/Interpreters/Context.cpp | 7 ------- src/Interpreters/Context.h | 1 - src/Processors/QueryPlan/ReadFromRemote.cpp | 13 ++++++------- 7 files changed, 14 insertions(+), 26 deletions(-) diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index ebc71511834..09211de53b0 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -54,8 +54,6 @@ public: struct ReplicaInfo { - bool collaborate_with_initiator{false}; - size_t all_replicas_count{0}; size_t number_of_current_replica{0}; }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index bcef286ecbc..866338bee8d 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -141,13 +141,12 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } + } - if (replica_info) - { - client_info.collaborate_with_initiator = true; - client_info.count_participating_replicas = replica_info->all_replicas_count; - client_info.number_of_current_replica = replica_info->number_of_current_replica; - } + if (replica_info) + { + client_info.collaborate_with_initiator = true; + client_info.number_of_current_replica = replica_info->number_of_current_replica; } /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index ce1efb61cc0..39fdef23baa 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -95,7 +95,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS) { writeVarUInt(static_cast(collaborate_with_initiator), out); - writeVarUInt(count_participating_replicas, out); + writeVarUInt(obsolete_count_participating_replicas, out); writeVarUInt(number_of_current_replica, out); } } @@ -185,7 +185,7 @@ void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision) UInt64 value; readVarUInt(value, in); collaborate_with_initiator = static_cast(value); - readVarUInt(count_participating_replicas, in); + readVarUInt(obsolete_count_participating_replicas, in); readVarUInt(number_of_current_replica, in); } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 3054667e264..ca32b4c5cfa 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -127,7 +127,7 @@ public: /// For parallel processing on replicas bool collaborate_with_initiator{false}; - UInt64 count_participating_replicas{0}; + UInt64 obsolete_count_participating_replicas{0}; UInt64 number_of_current_replica{0}; enum class BackgroundOperationType : uint8_t diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f2626696492..0d60f7dc5f4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4941,13 +4941,6 @@ void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 cli client_info.connection_tcp_protocol_version = client_tcp_protocol_version; } -void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica) -{ - client_info.collaborate_with_initiator = collaborate_with_initiator; - client_info.count_participating_replicas = all_replicas_count; - client_info.number_of_current_replica = number_of_current_replica; -} - void Context::increaseDistributedDepth() { ++client_info.distributed_depth; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8c5492bcbc8..5cd1153ab44 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -696,7 +696,6 @@ public: void setInitialQueryStartTime(std::chrono::time_point initial_query_start_time); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); - void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica); void increaseDistributedDepth(); const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; } OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 84c2515e8ca..8ebd95f27e5 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -411,8 +411,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); const auto & shard = cluster->getShardsInfo().at(0); - size_t all_replicas_count = current_settings.max_parallel_replicas; - if (all_replicas_count > shard.getAllNodeCount()) + size_t max_replicas_to_use = current_settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) { LOG_INFO( getLogger("ReadFromParallelRemoteReplicasStep"), @@ -420,14 +420,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, shard.getAllNodeCount()); - all_replicas_count = shard.getAllNodeCount(); + max_replicas_to_use = shard.getAllNodeCount(); } std::vector shuffled_pool; - if (all_replicas_count < shard.getAllNodeCount()) + if (max_replicas_to_use < shard.getAllNodeCount()) { shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(all_replicas_count); + shuffled_pool.resize(max_replicas_to_use); } else { @@ -437,11 +437,10 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - for (size_t i=0; i < all_replicas_count; ++i) + for (size_t i=0; i < max_replicas_to_use; ++i) { IConnections::ReplicaInfo replica_info { - .all_replicas_count = all_replicas_count, /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, }; From e8d831ce905d9ea6acbdb4e20142191c98431638 Mon Sep 17 00:00:00 2001 From: joelynch Date: Mon, 17 Jun 2024 19:06:23 +0200 Subject: [PATCH 065/178] Fix more dictGet ACL bypasses. --- src/Storages/StorageDictionary.cpp | 2 ++ .../02916_dictionary_access.reference | 2 ++ .../0_stateless/02916_dictionary_access.sh | 16 +++++++++++++++- 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 447fd87cdc9..a64acaebff6 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -160,6 +161,7 @@ Pipe StorageDictionary::read( const size_t max_block_size, const size_t threads) { + local_context->checkAccess(AccessType::dictGet, getStorageID()); auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); return dictionary->read(column_names, max_block_size, threads); diff --git a/tests/queries/0_stateless/02916_dictionary_access.reference b/tests/queries/0_stateless/02916_dictionary_access.reference index d7d02e37689..ab44e74249e 100644 --- a/tests/queries/0_stateless/02916_dictionary_access.reference +++ b/tests/queries/0_stateless/02916_dictionary_access.reference @@ -1,3 +1,5 @@ 0 ACCESS_DENIED ACCESS_DENIED +ACCESS_DENIED +ACCESS_DENIED diff --git a/tests/queries/0_stateless/02916_dictionary_access.sh b/tests/queries/0_stateless/02916_dictionary_access.sh index 08ee517ab3b..5f80de120f4 100755 --- a/tests/queries/0_stateless/02916_dictionary_access.sh +++ b/tests/queries/0_stateless/02916_dictionary_access.sh @@ -6,6 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" +dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" ${CLICKHOUSE_CLIENT} -nm --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} @@ -18,9 +19,13 @@ ${CLICKHOUSE_CLIENT} -nm --query " LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000); CREATE USER IF NOT EXISTS ${username} NOT IDENTIFIED; - GRANT CREATE TEMPORARY TABLE ON *.* to ${username}; + GRANT SELECT, CREATE TEMPORARY TABLE ON *.* to ${username}; SELECT * FROM dictionary(${dictname}); + SELECT * FROM ${dictname}; SELECT dictGet(${dictname}, 'value', 1); + CREATE TABLE ${dicttablename} (id UInt64, value UInt64) + ENGINE = Dictionary(${CLICKHOUSE_DATABASE}.${dictname}); + SELECT * FROM ${dicttablename}; " $CLICKHOUSE_CLIENT -nm --user="${username}" --query " @@ -31,7 +36,16 @@ $CLICKHOUSE_CLIENT -nm --user="${username}" --query " SELECT dictGet(${dictname}, 'value', 1); " 2>&1 | grep -o ACCESS_DENIED | uniq +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dictname}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dicttablename}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + ${CLICKHOUSE_CLIENT} -nm --query " + DROP TABLE IF EXISTS ${dicttablename} SYNC; DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; " From cd9fd1880f672fe0c8b072bf63a73e4d85ecfda4 Mon Sep 17 00:00:00 2001 From: joelynch Date: Wed, 19 Jun 2024 16:44:47 +0200 Subject: [PATCH 066/178] Check for dict access on underlying dict rather than table This is better for the table function because otherwise the database is set to "_table_function" when checking access which is not the database where the actual dict is. --- src/Storages/StorageDictionary.cpp | 2 +- src/TableFunctions/TableFunctionDictionary.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index a64acaebff6..72f873d1a0e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -161,9 +161,9 @@ Pipe StorageDictionary::read( const size_t max_block_size, const size_t threads) { - local_context->checkAccess(AccessType::dictGet, getStorageID()); auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); + local_context->checkAccess(AccessType::dictGet, dictionary->getDatabaseOrNoDatabaseTag(), dictionary->getDictionaryID().getTableName()); return dictionary->read(column_names, max_block_size, threads); } diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 867fbf5b11e..d58c01385e0 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -80,7 +80,6 @@ ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr c StoragePtr TableFunctionDictionary::executeImpl( const ASTPtr &, ContextPtr context, const std::string & table_name, ColumnsDescription, bool is_insert_query) const { - context->checkAccess(AccessType::dictGet, getDatabaseName(), table_name); StorageID dict_id(getDatabaseName(), table_name); auto dictionary_table_structure = getActualTableStructure(context, is_insert_query); From 29dac3c97ad69736f876073ab6023d9e6c7c564a Mon Sep 17 00:00:00 2001 From: joelynch Date: Tue, 2 Jul 2024 20:47:02 +0200 Subject: [PATCH 067/178] move tests to new file --- .../02916_dictionary_access.reference | 2 - .../0_stateless/02916_dictionary_access.sh | 16 +------- .../03199_dictionary_table_access.reference | 2 + .../03199_dictionary_table_access.sh | 41 +++++++++++++++++++ 4 files changed, 44 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/03199_dictionary_table_access.reference create mode 100755 tests/queries/0_stateless/03199_dictionary_table_access.sh diff --git a/tests/queries/0_stateless/02916_dictionary_access.reference b/tests/queries/0_stateless/02916_dictionary_access.reference index ab44e74249e..d7d02e37689 100644 --- a/tests/queries/0_stateless/02916_dictionary_access.reference +++ b/tests/queries/0_stateless/02916_dictionary_access.reference @@ -1,5 +1,3 @@ 0 ACCESS_DENIED ACCESS_DENIED -ACCESS_DENIED -ACCESS_DENIED diff --git a/tests/queries/0_stateless/02916_dictionary_access.sh b/tests/queries/0_stateless/02916_dictionary_access.sh index 5f80de120f4..08ee517ab3b 100755 --- a/tests/queries/0_stateless/02916_dictionary_access.sh +++ b/tests/queries/0_stateless/02916_dictionary_access.sh @@ -6,7 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" -dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" ${CLICKHOUSE_CLIENT} -nm --query " CREATE DICTIONARY IF NOT EXISTS ${dictname} @@ -19,13 +18,9 @@ ${CLICKHOUSE_CLIENT} -nm --query " LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000); CREATE USER IF NOT EXISTS ${username} NOT IDENTIFIED; - GRANT SELECT, CREATE TEMPORARY TABLE ON *.* to ${username}; + GRANT CREATE TEMPORARY TABLE ON *.* to ${username}; SELECT * FROM dictionary(${dictname}); - SELECT * FROM ${dictname}; SELECT dictGet(${dictname}, 'value', 1); - CREATE TABLE ${dicttablename} (id UInt64, value UInt64) - ENGINE = Dictionary(${CLICKHOUSE_DATABASE}.${dictname}); - SELECT * FROM ${dicttablename}; " $CLICKHOUSE_CLIENT -nm --user="${username}" --query " @@ -36,16 +31,7 @@ $CLICKHOUSE_CLIENT -nm --user="${username}" --query " SELECT dictGet(${dictname}, 'value', 1); " 2>&1 | grep -o ACCESS_DENIED | uniq -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " - SELECT * FROM ${dictname}; -" 2>&1 | grep -o ACCESS_DENIED | uniq - -$CLICKHOUSE_CLIENT -nm --user="${username}" --query " - SELECT * FROM ${dicttablename}; -" 2>&1 | grep -o ACCESS_DENIED | uniq - ${CLICKHOUSE_CLIENT} -nm --query " - DROP TABLE IF EXISTS ${dicttablename} SYNC; DROP DICTIONARY IF EXISTS ${dictname}; DROP USER IF EXISTS ${username}; " diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.reference b/tests/queries/0_stateless/03199_dictionary_table_access.reference new file mode 100644 index 00000000000..4a703b3be84 --- /dev/null +++ b/tests/queries/0_stateless/03199_dictionary_table_access.reference @@ -0,0 +1,2 @@ +ACCESS_DENIED +ACCESS_DENIED diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.sh b/tests/queries/0_stateless/03199_dictionary_table_access.sh new file mode 100755 index 00000000000..952b466b5da --- /dev/null +++ b/tests/queries/0_stateless/03199_dictionary_table_access.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" +dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" +dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" + +${CLICKHOUSE_CLIENT} -nm --query " + CREATE DICTIONARY IF NOT EXISTS ${dictname} + ( + id UInt64, + value UInt64 + ) + PRIMARY KEY id + SOURCE(NULL()) + LAYOUT(FLAT()) + LIFETIME(MIN 0 MAX 1000); + CREATE USER IF NOT EXISTS ${username} NOT IDENTIFIED; + GRANT SELECT, CREATE TEMPORARY TABLE ON *.* to ${username}; + SELECT * FROM ${dictname}; + CREATE TABLE ${dicttablename} (id UInt64, value UInt64) + ENGINE = Dictionary(${CLICKHOUSE_DATABASE}.${dictname}); + SELECT * FROM ${dicttablename}; +" + +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dictname}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dicttablename}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + +${CLICKHOUSE_CLIENT} -nm --query " + DROP TABLE IF EXISTS ${dicttablename} SYNC; + DROP DICTIONARY IF EXISTS ${dictname}; + DROP USER IF EXISTS ${username}; +" From 2f391cfa6c18bea786fbe21c692b7c7e33b6d34b Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Thu, 11 Jul 2024 17:50:46 +0800 Subject: [PATCH 068/178] restart ci From f85be2b452e9d62f766cee3d170dacd1906e04b7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:17:59 +0000 Subject: [PATCH 069/178] Add --memory-usage option to client in non interactive mode --- src/Client/ClientBase.cpp | 22 ++++++++++++++++++++-- src/Common/ProgressIndication.h | 10 +++++----- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d472ba99b9..b5de348d583 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2069,9 +2069,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.writeFinalProgress(); output_stream << std::endl << std::endl; } - else if (getClientConfiguration().getBool("print-time-to-stderr", false)) + else { - error_stream << progress_indication.elapsedSeconds() << "\n"; + const auto & config = getClientConfiguration(); + if (config.getBool("print-time-to-stderr", false)) + error_stream << progress_indication.elapsedSeconds() << "\n"; + + const auto & print_memory_mode = config.getString("print-memory-to-stderr", ""); + auto peak_memeory_usage = std::max(progress_indication.getMemoryUsage().peak, 0); + if (print_memory_mode == "default") + error_stream << peak_memeory_usage << "\n"; + else if (print_memory_mode == "readable") + error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n"; } if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false)) @@ -3035,6 +3044,7 @@ void ClientBase::init(int argc, char ** argv) ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") ("echo", "in batch mode, print query before execution") @@ -3120,6 +3130,14 @@ void ClientBase::init(int argc, char ** argv) /// Output execution time to stderr in batch mode. if (options.count("time")) getClientConfiguration().setBool("print-time-to-stderr", true); + if (options.count("memory-usage")) + { + const auto & memory_usage_mode = options["memory-usage"].as(); + if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); + getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); + } + if (options.count("query")) queries = options["query"].as>(); if (options.count("query_id")) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index ae39fb49bcc..c7f61a7fba9 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -71,11 +71,6 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; } - void updateThreadEventData(HostToTimesMap & new_hosts_data); - -private: - double getCPUUsage(); - struct MemoryUsage { UInt64 total = 0; @@ -85,6 +80,11 @@ private: MemoryUsage getMemoryUsage() const; + void updateThreadEventData(HostToTimesMap & new_hosts_data); + +private: + double getCPUUsage(); + UInt64 getElapsedNanoseconds() const; /// This flag controls whether to show the progress bar. We start showing it after From b9cd1bddd7c8612e3b43172ea8670a7e904fe237 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:20:37 +0000 Subject: [PATCH 070/178] upd doc --- docs/en/interfaces/cli.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index e18ff6f1a3f..63b7353d092 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. +- `--memory-usage` – If specified, print the query memory usage to ‘stderr’ in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). @@ -339,7 +340,7 @@ clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000 Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`. ``` bash -clickhouse-client clickhouse://192.168.1.15,192.168.1.25 +clickhouse-client clickhouse://192.168.1.15,192.168.1.25 ``` ### Configuration Files {#configuration_files} @@ -367,7 +368,7 @@ Example of a config file: ``` Or the same config in a YAML format: - + ```yaml user: username password: 'password' From 05c3692e1dfcacffc9de62bac9a53a0e28f4bea7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:22:03 +0000 Subject: [PATCH 071/178] add 03203_client_benchmark_options --- .../03203_client_benchmark_options.reference | 6 ++++++ .../0_stateless/03203_client_benchmark_options.sh | 13 +++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03203_client_benchmark_options.reference create mode 100755 tests/queries/0_stateless/03203_client_benchmark_options.sh diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.reference b/tests/queries/0_stateless/03203_client_benchmark_options.reference new file mode 100644 index 00000000000..fd2996b1c78 --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.reference @@ -0,0 +1,6 @@ +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh new file mode 100755 index 00000000000..a9b9d69822b --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" # expected no output +${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9].*B$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "BAD_ARGUMENTS" && echo "Ok" || echo "Fail" From 82e13d785a460592e68662f0d94d4c0ce6d79057 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Thu, 11 Jul 2024 22:23:09 +0800 Subject: [PATCH 072/178] restart ci From ef4f34545e46a4e0298bd4bca28597ae642f7b50 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 11 Jul 2024 18:00:32 +0200 Subject: [PATCH 073/178] make 01275_parallel_mv.sql.j2 stable --- .../Transforms/buildPushingToViewsChain.cpp | 4 +- .../0_stateless/01275_parallel_mv.reference | 101 +++++++++--------- .../0_stateless/01275_parallel_mv.sql.j2 | 16 ++- tests/result | 12 +++ 4 files changed, 78 insertions(+), 55 deletions(-) create mode 100644 tests/result diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 312b333ab33..3df726aa0e8 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -794,7 +794,7 @@ PushingToLiveViewSink::PushingToLiveViewSink(const Block & header, StorageLiveVi void PushingToLiveViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); - live_view.writeBlock(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); + live_view.writeBlock(live_view, getHeader().cloneWithColumns(chunk.getColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); @@ -818,7 +818,7 @@ void PushingToWindowViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( - window_view, getHeader().cloneWithColumns(chunk.detachColumns()), std::move(chunk.getChunkInfos()), context); + window_view, getHeader().cloneWithColumns(chunk.getColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index 221d0b5ce8d..cf1f7054f6c 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -2,13 +2,18 @@ -select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; -optimize_trivial_insert_select=0 max_insert_threads=0 + + + +select 'optimize_trivial_insert_select=0', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=0 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_1', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -22,17 +27,17 @@ select count() from testX; 200 select count() from testXA; 200 -select count() from testXB; -0 select count() from testXC; 200 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=5'; -optimize_trivial_insert_select=0 max_insert_threads=5 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=5 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_2', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, - max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -46,17 +51,17 @@ select count() from testX; 400 select count() from testXA; 400 -select count() from testXB; -0 select count() from testXC; 400 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; -optimize_trivial_insert_select=1 max_insert_threads=0 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=0 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_3', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -70,17 +75,17 @@ select count() from testX; 600 select count() from testXA; 600 -select count() from testXB; -0 select count() from testXC; 600 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=5'; -optimize_trivial_insert_select=1 max_insert_threads=5 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=5 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_4', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, - max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -94,17 +99,17 @@ select count() from testX; 800 select count() from testXA; 800 -select count() from testXB; -0 select count() from testXC; 800 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; -optimize_trivial_insert_select=0 max_insert_threads=0 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=0 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_5', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -118,17 +123,17 @@ select count() from testX; 1000 select count() from testXA; 1000 -select count() from testXB; -0 select count() from testXC; 1000 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=5'; -optimize_trivial_insert_select=0 max_insert_threads=5 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=5 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_6', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, - max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -139,20 +144,20 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '5'; 12 select count() from testX; -1190 +1200 select count() from testXA; -1130 -select count() from testXB; -60 +1200 select count() from testXC; -1130 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; -optimize_trivial_insert_select=1 max_insert_threads=0 +1200 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=0 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_7', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -163,20 +168,20 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '0'; 2 select count() from testX; -1390 +1400 select count() from testXA; -1330 -select count() from testXB; -60 +1400 select count() from testXC; -1330 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=5'; -optimize_trivial_insert_select=1 max_insert_threads=5 +1400 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=5 iteration_num= insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_8', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, - max_insert_threads=5; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -187,10 +192,8 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '5'; 7 select count() from testX; -1590 +1600 select count() from testXA; -1480 -select count() from testXB; -160 +1600 select count() from testXC; -1490 +1600 diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 index 4e45c68b5ad..6805b8902ec 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 +++ b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 @@ -5,11 +5,12 @@ -- avoid settings randomization by clickhouse-test set max_threads = 10; - -- more blocks to process set max_block_size = 10; set min_insert_block_size_rows = 10; +set materialized_views_ignore_errors = 1; + drop table if exists testX; drop table if exists testXA; drop table if exists testXB; @@ -21,17 +22,25 @@ create materialized view testXA engine=MergeTree order by tuple() as select slee create materialized view testXB engine=MergeTree order by tuple() as select sleep(0.2), throwIf(A=1) from testX; create materialized view testXC engine=MergeTree order by tuple() as select sleep(0.1) from testX; + +{% set count = namespace(value=0) %} + -- { echoOn } {% for parallel_view_processing in [0, 1] %} {% for optimize_trivial_insert_select in [0, 1] %} {% for max_insert_threads in [0, 5] %} -select 'optimize_trivial_insert_select={{ optimize_trivial_insert_select }}', 'max_insert_threads={{ max_insert_threads }}'; + +{% set count.value = count.value + 1 %} + +select 'optimize_trivial_insert_select={{ optimize_trivial_insert_select }}', 'max_insert_threads={{ max_insert_threads }}', 'iteration_num={{ iteration_num }}'; insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_{{ count.value }}', log_queries=1, parallel_view_processing={{ parallel_view_processing }}, optimize_trivial_insert_select={{ optimize_trivial_insert_select }}, - max_insert_threads={{ max_insert_threads }}; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads={{ max_insert_threads }}; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -43,7 +52,6 @@ select peak_threads_usage from system.query_log where select count() from testX; select count() from testXA; -select count() from testXB; select count() from testXC; {% endfor %} {% endfor %} diff --git a/tests/result b/tests/result new file mode 100644 index 00000000000..b76f44f1e6a --- /dev/null +++ b/tests/result @@ -0,0 +1,12 @@ +Using queries from 'queries' directory +Connecting to ClickHouse server...... OK +Connected to server 24.7.1.1 @ 246f421f2402799fd11b22a608b4d0d497cb8438 chesema-processor-onCancel + +Running 1 stateless tests (MainProcess). + +00993_system_parts_race_condition_drop_zookeeper: [ OK ] + +1 tests passed. 0 tests skipped. 124.59 s elapsed (MainProcess). + +0 tests passed. 0 tests skipped. 0.00 s elapsed (MainProcess). +All tests have finished. From 861dcbbffbef382c542e6e9485ec21a8e5fa5ffd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jul 2024 16:04:00 +0000 Subject: [PATCH 074/178] Do not check parent scope for group_by_use_nulls outside of subquery. --- .../Resolve/IdentifierResolveScope.cpp | 3 +- src/Analyzer/Resolve/IdentifierResolveScope.h | 4 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 30 +++++++------ ...2535_analyzer_group_by_use_nulls.reference | 41 +++++++++++++++++ .../02535_analyzer_group_by_use_nulls.sql | 45 +++++++++++++++++++ 5 files changed, 108 insertions(+), 15 deletions(-) diff --git a/src/Analyzer/Resolve/IdentifierResolveScope.cpp b/src/Analyzer/Resolve/IdentifierResolveScope.cpp index ae363b57047..32b3107ac16 100644 --- a/src/Analyzer/Resolve/IdentifierResolveScope.cpp +++ b/src/Analyzer/Resolve/IdentifierResolveScope.cpp @@ -11,9 +11,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_) +IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_, bool is_query_) : scope_node(std::move(scope_node_)) , parent_scope(parent_scope_) + , is_query(is_query_) { if (parent_scope) { diff --git a/src/Analyzer/Resolve/IdentifierResolveScope.h b/src/Analyzer/Resolve/IdentifierResolveScope.h index ab2e27cc14d..917e032321d 100644 --- a/src/Analyzer/Resolve/IdentifierResolveScope.h +++ b/src/Analyzer/Resolve/IdentifierResolveScope.h @@ -128,7 +128,7 @@ constexpr auto PROJECTION_NAME_PLACEHOLDER = "__projection_name_placeholder"; struct IdentifierResolveScope { /// Construct identifier resolve scope using scope node, and parent scope - IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_); + IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_, bool is_query_); QueryTreeNodePtr scope_node; @@ -188,6 +188,8 @@ struct IdentifierResolveScope /// Join retutns NULLs instead of default values bool join_use_nulls = false; + bool is_query; + /// JOINs count size_t joins_count = 0; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 576c4943ccb..63249ad2aeb 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -117,7 +117,7 @@ QueryAnalyzer::~QueryAnalyzer() = default; void QueryAnalyzer::resolve(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, ContextPtr context) { - IdentifierResolveScope scope(node, nullptr /*parent_scope*/); + IdentifierResolveScope scope(node, nullptr /*parent_scope*/, true /*is_query*/); if (!scope.context) scope.context = context; @@ -509,7 +509,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden /// exception with this settings enabled(https://github.com/ClickHouse/ClickHouse/issues/52494). subquery_context->setSetting("use_structure_from_insertion_table_in_table_functions", false); - auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/); + auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_query*/); options.only_analyze = only_analyze; auto interpreter = std::make_unique(node->toAST(), subquery_context, subquery_context->getViewSource(), options); @@ -2163,7 +2163,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::LAMBDA) { auto lambda_expression_to_resolve = expression_node->clone(); - IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); + IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/, false /*is_query*/); node_projection_names = resolveLambda(expression_node, lambda_expression_to_resolve, {node}, lambda_scope); auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); node = lambda_expression_to_resolve_typed.getExpression(); @@ -3036,7 +3036,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto lambda_expression_clone = lambda_expression_untyped->clone(); - IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); + IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/, false /*is_query*/); ProjectionNames lambda_projection_names = resolveLambda(lambda_expression_untyped, lambda_expression_clone, function_arguments, lambda_scope); auto & resolved_lambda = lambda_expression_clone->as(); @@ -3291,7 +3291,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi lambda_arguments.push_back(std::make_shared(std::move(column_name_and_type), lambda_to_resolve)); } - IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); + IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/, false /*is_query*/); lambda_projection_names = resolveLambda(lambda_argument, lambda_to_resolve, lambda_arguments, lambda_scope); if (auto * lambda_list_node_result = lambda_to_resolve_typed.getExpression()->as()) @@ -3512,7 +3512,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( auto node_type = node->getNodeType(); if (!allow_table_expression && (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION)) { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/, false /*is_query*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; evaluateScalarSubqueryIfNeeded(node, subquery_scope); @@ -3619,7 +3619,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( else union_node->setIsCTE(false); - IdentifierResolveScope subquery_scope(resolved_identifier_node, &scope /*parent_scope*/); + IdentifierResolveScope subquery_scope(resolved_identifier_node, &scope /*parent_scope*/, true /*is_query*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; /// CTE is being resolved, it's required to forbid to resolve to it again @@ -3752,7 +3752,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( [[fallthrough]]; case QueryTreeNodeType::UNION: { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/, true /*is_query*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; std::string projection_name = "_subquery_" + std::to_string(subquery_counter); @@ -3826,6 +3826,10 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( node->convertToNullable(); break; } + + /// Check parent scopes until find current query scope. + if (scope_ptr->is_query) + break; } } @@ -4112,7 +4116,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo bool is_column_constant = interpolate_node_typed.getExpression()->getNodeType() == QueryTreeNodeType::CONSTANT; auto & interpolation_to_resolve = interpolate_node_typed.getInterpolateExpression(); - IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/); + IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/, false /*is_query*/); auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node_typed.getExpression()); if (is_column_constant) @@ -4410,7 +4414,7 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table */ alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; - IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); + IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/, false /*is_query*/); alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); alias_column_resolve_scope.context = scope.context; @@ -5003,7 +5007,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS left_subquery->getProjection().getNodes().push_back(projection_node->clone()); left_subquery->getJoinTree() = left_table_expression; - IdentifierResolveScope left_subquery_scope(left_subquery, nullptr /*parent_scope*/); + IdentifierResolveScope left_subquery_scope(left_subquery, nullptr /*parent_scope*/, true /*is_query*/); resolveQuery(left_subquery, left_subquery_scope); const auto & resolved_nodes = left_subquery->getProjection().getNodes(); @@ -5612,7 +5616,7 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier auto & non_recursive_query_mutable_context = non_recursive_query_is_query_node ? non_recursive_query->as().getMutableContext() : non_recursive_query->as().getMutableContext(); - IdentifierResolveScope non_recursive_subquery_scope(non_recursive_query, &scope /*parent_scope*/); + IdentifierResolveScope non_recursive_subquery_scope(non_recursive_query, &scope /*parent_scope*/, true /*is_query*/); non_recursive_subquery_scope.subquery_depth = scope.subquery_depth + 1; if (non_recursive_query_is_query_node) @@ -5643,7 +5647,7 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier { auto & query_node = queries_nodes[i]; - IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/); + IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/, true /*is_subquery*/); if (recursive_cte_table_node) subquery_scope.expression_argument_name_to_node[union_node_typed.getCTEName()] = recursive_cte_table_node; diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference index 63610604ddd..858fbe98838 100644 --- a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference @@ -264,3 +264,44 @@ SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; 9 \N 9 \N 0 20 \N 1 25 +CREATE TABLE test +ENGINE = ReplacingMergeTree +PRIMARY KEY id +AS SELECT number AS id FROM numbers(100); +SELECT id +FROM test +GROUP BY id + WITH CUBE +HAVING id IN ( + SELECT id + FROM test +) +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true; +SELECT id +FROM test +FINAL +GROUP BY id + WITH CUBE +HAVING id IN ( + SELECT DISTINCT id + FROM test + FINAL +) +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true; +SELECT id +FROM test +FINAL +GROUP BY + GROUPING SETS ((id)) +ORDER BY + id IN ( + SELECT DISTINCT id + FROM test + FINAL + LIMIT 4 + ) ASC +LIMIT 256 BY id +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls=true; diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql index a4d4f2f8bc9..4ae5df9629a 100644 --- a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql @@ -83,3 +83,48 @@ GROUP BY ) ORDER BY 1, tuple(val) SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; + +CREATE TABLE test +ENGINE = ReplacingMergeTree +PRIMARY KEY id +AS SELECT number AS id FROM numbers(100); + +SELECT id +FROM test +GROUP BY id + WITH CUBE +HAVING id IN ( + SELECT id + FROM test +) +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true; + +SELECT id +FROM test +FINAL +GROUP BY id + WITH CUBE +HAVING id IN ( + SELECT DISTINCT id + FROM test + FINAL +) +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls = true; + +SELECT id +FROM test +FINAL +GROUP BY + GROUPING SETS ((id)) +ORDER BY + id IN ( + SELECT DISTINCT id + FROM test + FINAL + LIMIT 4 + ) ASC +LIMIT 256 BY id +FORMAT `NUll` +SETTINGS allow_experimental_analyzer = 1, group_by_use_nulls=true; From fa2270fd59c0de91d45ac50f2064d31f568935e4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 11 Jul 2024 22:35:39 +0000 Subject: [PATCH 075/178] Fix VALID UNTIL clause resetting after restart --- .../Access/InterpreterCreateUserQuery.cpp | 17 ++++++++++++++- .../integration/test_user_valid_until/test.py | 21 ++++++++++++++++++- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 32c51b745c7..a09de7e688a 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -259,7 +259,22 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat if (query.auth_data) auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach); - updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true); + std::optional valid_until; + if (query.valid_until) + { + const String valid_until_str = checkAndGetLiteralArgument(query.valid_until, "valid_until"); + time_t time = 0; + + if (valid_until_str != "infinity") + { + ReadBufferFromString in(valid_until_str); + readDateTimeText(time, in); + } + + valid_until = time; + } + + updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true); } void registerInterpreterCreateUserQuery(InterpreterFactory & factory) diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index d6d5bf8b18e..39ca5997067 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -5,7 +5,7 @@ from time import sleep from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") +node = cluster.add_instance("node", stay_alive=True) @pytest.fixture(scope="module") @@ -84,3 +84,22 @@ def test_details(started_cluster): node.query("SHOW CREATE USER user_details_time_only") == f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n" ) + + +def test_restart(started_cluster): + node.query("CREATE USER user_restart VALID UNTIL '06/11/2010 08:03:20 Z+3'") + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + node.restart_clickhouse() + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + error = "Authentication failed" + assert error in node.query_and_get_error("SELECT 1", user="user_restart") From 25157fd17d27429cf27d4cd1c9e6e87be37d6798 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 Jul 2024 11:01:42 +0200 Subject: [PATCH 076/178] Disable optimize_functions_to_subcolumns in test 03202_dynamic_null_map_subcolumn --- tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh index aa06e48376c..968c9e5271f 100755 --- a/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh +++ b/tests/queries/0_stateless/03202_dynamic_null_map_subcolumn.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --optimize_functions_to_subcolumns=0" function test() From 7781067ba18ed4817770a804f4407c670dea5550 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 16:53:38 +0000 Subject: [PATCH 077/178] Fix pushdown for join with external table Fix possible incorrect result for queries joining and filtering table external engine (like PostgreSQL), due to too agressive filter pushdown. Since now, conditions from where section won't be send to external database in case of outer join with external table. --- .../transformQueryForExternalDatabase.cpp | 2 +- ...nsformQueryForExternalDatabaseAnalyzer.cpp | 29 ++++++++-- ...ransformQueryForExternalDatabaseAnalyzer.h | 2 +- .../test_storage_postgresql/test.py | 54 +++++++++++++++++++ 4 files changed, 82 insertions(+), 5 deletions(-) diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index fc85bde11d9..7aac138296d 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -419,7 +419,7 @@ String transformQueryForExternalDatabase( throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "No column names for query '{}' to external table '{}.{}'", query_info.query_tree->formatASTForErrorMessage(), database, table); - auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree); + auto clone_query = getASTForExternalDatabaseFromQueryTree(query_info.query_tree, query_info.table_expression); return transformQueryForExternalDatabaseImpl( clone_query, diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index 5e0bfdd5f2a..dc1749b3196 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -10,7 +11,7 @@ #include #include #include - +#include #include @@ -20,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; } namespace @@ -55,7 +57,7 @@ public: } -ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree) +ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression) { auto new_tree = query_tree->clone(); @@ -63,6 +65,21 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre visitor.visit(new_tree); const auto * query_node = new_tree->as(); + const auto & join_tree = query_node->getJoinTree(); + bool allow_where = true; + if (const auto * join_node = join_tree->as()) + { + if (join_node->getStrictness() != JoinStrictness::All) + allow_where = false; + + if (join_node->getKind() == JoinKind::Left) + allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression); + else if (join_node->getKind() == JoinKind::Right) + allow_where = join_node->getRightTableExpression()->isEqual(*table_expression); + else + allow_where = (join_node->getKind() == JoinKind::Inner); + } + auto query_node_ast = query_node->toAST({ .add_cast_for_constants = false, .fully_qualified_identifiers = false }); const IAST * ast = query_node_ast.get(); @@ -76,7 +93,13 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre if (union_ast->list_of_selects->children.size() != 1) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "QueryNode AST is not a single ASTSelectQuery, got {}", union_ast->list_of_selects->children.size()); - return union_ast->list_of_selects->children.at(0); + ASTPtr select_query = union_ast->list_of_selects->children.at(0); + auto * select_query_typed = select_query->as(); + if (!select_query_typed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ASTSelectQuery, got {}", select_query ? select_query->formatForErrorMessage() : "nullptr"); + if (!allow_where) + select_query_typed->setExpression(ASTSelectQuery::Expression::WHERE, nullptr); + return select_query; } } diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.h b/src/Storages/transformQueryForExternalDatabaseAnalyzer.h index f8983619d1f..7d8bf99646b 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.h +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.h @@ -6,6 +6,6 @@ namespace DB { -ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree); +ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tree, const QueryTreeNodePtr & table_expression); } diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d9f3a9917ab..ffcff36c47e 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -834,6 +834,60 @@ def test_literal_escaping(started_cluster): cursor.execute(f"DROP TABLE escaping") +def test_filter_pushdown(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute("CREATE SCHEMA test_filter_pushdown") + cursor.execute( + "CREATE TABLE test_filter_pushdown.test_table (id integer, value integer)" + ) + cursor.execute( + "INSERT INTO test_filter_pushdown.test_table VALUES (1, 10), (1, 110), (2, 0), (3, 33), (4, 0)" + ) + + node1.query( + """ + CREATE TABLE test_filter_pushdown_pg_table (id UInt32, value UInt32) + ENGINE PostgreSQL('postgres1:5432', 'postgres', 'test_table', 'postgres', 'mysecretpassword', 'test_filter_pushdown'); + """ + ) + + node1.query( + """ + CREATE TABLE test_filter_pushdown_local_table (id UInt32, value UInt32) ENGINE Memory AS SELECT * FROM test_filter_pushdown_pg_table + """ + ) + + node1.query( + "CREATE TABLE ch_table (id UInt32, pg_id UInt32) ENGINE MergeTree ORDER BY id" + ) + node1.query("INSERT INTO ch_table VALUES (1, 1), (2, 2), (3, 1), (4, 2), (5, 999)") + + def compare_results(query, **kwargs): + result1 = node1.query( + query.format(pg_table="test_filter_pushdown_pg_table", **kwargs) + ) + result2 = node1.query( + query.format(pg_table="test_filter_pushdown_local_table", **kwargs) + ) + assert result1 == result2 + + for kind in ["INNER", "LEFT", "RIGHT", "FULL"]: + for value in [0, 10]: + compare_results( + "SELECT * FROM ch_table {kind} JOIN {pg_table} as p ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL", + kind=kind, + value=value, + ) + + compare_results( + "SELECT * FROM {pg_table} as p {kind} JOIN ch_table ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL", + kind=kind, + value=value, + ) + + cursor.execute("DROP SCHEMA test_filter_pushdown CASCADE") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 705134413ffd321a59c1c2c82af11a5edddc962c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 12 Jul 2024 12:00:39 +0200 Subject: [PATCH 078/178] Update docs/en/interfaces/cli.md --- docs/en/interfaces/cli.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 63b7353d092..849b3d4b486 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -185,7 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. -- `--memory-usage` – If specified, print the query memory usage to ‘stderr’ in non-interactive mode. +- `--memory-usage` – If specified, print memory usage to ‘stderr’ in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). From f7fc031d9d1d63a057e5597088f1b70bc81f38a6 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Fri, 12 Jul 2024 10:04:10 +0000 Subject: [PATCH 079/178] fix data race with permanently_detached_tables --- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/DatabaseOrdinary.h | 8 ++++++-- src/Interpreters/loadMetadata.cpp | 2 +- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 98ef4b26ae2..db812e89782 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -224,10 +224,10 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables if (fs::exists(full_path.string() + detached_suffix)) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); - permanently_detached_tables.push_back(table_name); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); std::lock_guard lock(mutex); + permanently_detached_tables.push_back(table_name); const auto detached_table_name = create_query->getTable(); diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index d2891147922..c2c5775e5ab 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -67,7 +67,11 @@ public: const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; - Strings getNamesOfPermanentlyDetachedTables() const override { return permanently_detached_tables; } + Strings getNamesOfPermanentlyDetachedTables() const override + { + std::lock_guard lock(mutex); + return permanently_detached_tables; + } protected: virtual void commitAlterTable( @@ -77,7 +81,7 @@ protected: const String & statement, ContextPtr query_context); - Strings permanently_detached_tables; + Strings permanently_detached_tables TSA_GUARDED_BY(mutex); std::unordered_map load_table TSA_GUARDED_BY(mutex); std::unordered_map startup_table TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 9c3922b8bda..03dd1714ead 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -384,7 +384,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons if (database->getEngineName() != "Ordinary") return; - Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables(); + const Strings permanently_detached_tables = database->getNamesOfPermanentlyDetachedTables(); if (!permanently_detached_tables.empty()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot automatically convert database {} from Ordinary to Atomic, " From 8c2e4900e301b7b54c8c266f2c2c8ef4a4ef5e2c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 10:48:34 +0000 Subject: [PATCH 080/178] Introduce isServerConstant() flag to functions. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 6 +- src/Functions/IFunction.h | 12 ++++ src/Functions/IFunctionAdaptors.h | 2 + src/Functions/getScalar.cpp | 4 ++ src/Functions/serverConstants.cpp | 56 +++++++++++-------- src/Planner/PlannerExpressionAnalysis.cpp | 6 +- .../02992_analyzer_group_by_const.sql | 43 ++++++++++++++ 7 files changed, 99 insertions(+), 30 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 576c4943ccb..818e75f9594 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3410,14 +3410,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi function_base = function->build(argument_columns); /// Do not constant fold get scalar functions - bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || - function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort"; + // bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || + // function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort"; /** If function is suitable for constant folding try to convert it to constant. * Example: SELECT plus(1, 1); * Result: SELECT 2; */ - if (function_base->isSuitableForConstantFolding() && !disable_constant_folding) + if (function_base->isSuitableForConstantFolding()) // && !disable_constant_folding) { auto result_type = function_base->getResultType(); auto executable_function = function_base->prepare(argument_columns); diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index a66456cabee..23aab420338 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -230,6 +230,17 @@ public: virtual bool isDeterministicInScopeOfQuery() const { return true; } + /** This is a special flags for functions which return constant value for the server, + * but the result could be different for different servers in distributed query. + * + * This fuctions can't support constant folding on the initiator, but can on the follower. + * We can't apply some optimizations as well (e.g. can't remove constant result from GROUP BY key). + * So, it is convenient to have a special flag for them. + * + * Examples are: "__getScalar" and every function from serverConstants.cpp + */ + virtual bool isServerConstant() const { return false; } + /** Lets you know if the function is monotonic in a range of values. * This is used to work with the index in a sorted chunk of data. * And allows to use the index not only when it is written, for example `date >= const`, but also, for example, `toMonth(date) >= 11`. @@ -488,6 +499,7 @@ public: virtual bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const { return false; } virtual bool isDeterministic() const { return true; } virtual bool isDeterministicInScopeOfQuery() const { return true; } + virtual bool isServerConstant() const { return false; } virtual bool isStateful() const { return false; } using ShortCircuitSettings = IFunctionBase::ShortCircuitSettings; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 04bd03a776e..c9929a083c1 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -86,6 +86,8 @@ public: bool isDeterministicInScopeOfQuery() const override { return function->isDeterministicInScopeOfQuery(); } + bool isServerConstant() const override { return function->isServerConstant(); } + bool isShortCircuit(ShortCircuitSettings & settings, size_t number_of_arguments) const override { return function->isShortCircuit(settings, number_of_arguments); } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & args) const override { return function->isSuitableForShortCircuitArgumentsExecution(args); } diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index 7196cbc0a36..5131dca962e 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -49,6 +49,8 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isServerConstant() const override { return true; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() != 1 || !isString(arguments[0].type) || !arguments[0].column || !isColumnConst(*arguments[0].column)) @@ -105,6 +107,8 @@ public: bool isDeterministic() const override { return false; } + bool isServerConstant() const override { return true; } + bool isSuitableForConstantFolding() const override { return !is_distributed; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index e7e423058f1..1e2c8c48bab 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -20,117 +20,125 @@ namespace DB namespace { + template + class FunctionServerConstantBase : public FunctionConstantBase + { + public: + using FunctionConstantBase::FunctionConstantBase; + bool isServerConstant() const override { return true; } + }; + #if defined(__ELF__) && !defined(OS_FREEBSD) /// buildId() - returns the compiler build id of the running binary. - class FunctionBuildId : public FunctionConstantBase + class FunctionBuildId : public FunctionServerConstantBase { public: static constexpr auto name = "buildId"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionBuildId(ContextPtr context) : FunctionConstantBase(SymbolIndex::instance().getBuildIDHex(), context->isDistributed()) {} + explicit FunctionBuildId(ContextPtr context) : FunctionServerConstantBase(SymbolIndex::instance().getBuildIDHex(), context->isDistributed()) {} }; #endif /// Get the host name. It is constant on single server, but is not constant in distributed queries. - class FunctionHostName : public FunctionConstantBase + class FunctionHostName : public FunctionServerConstantBase { public: static constexpr auto name = "hostName"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionHostName(ContextPtr context) : FunctionConstantBase(DNSResolver::instance().getHostName(), context->isDistributed()) {} + explicit FunctionHostName(ContextPtr context) : FunctionServerConstantBase(DNSResolver::instance().getHostName(), context->isDistributed()) {} }; - class FunctionServerUUID : public FunctionConstantBase + class FunctionServerUUID : public FunctionServerConstantBase { public: static constexpr auto name = "serverUUID"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerUUID(ContextPtr context) : FunctionConstantBase(ServerUUID::get(), context->isDistributed()) {} + explicit FunctionServerUUID(ContextPtr context) : FunctionServerConstantBase(ServerUUID::get(), context->isDistributed()) {} }; - class FunctionTCPPort : public FunctionConstantBase + class FunctionTCPPort : public FunctionServerConstantBase { public: static constexpr auto name = "tcpPort"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTCPPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {} + explicit FunctionTCPPort(ContextPtr context) : FunctionServerConstantBase(context->getTCPPort(), context->isDistributed()) {} }; /// Returns timezone for current session. - class FunctionTimezone : public FunctionConstantBase + class FunctionTimezone : public FunctionServerConstantBase { public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::instance().getTimeZone(), context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionServerConstantBase(DateLUT::instance().getTimeZone(), context->isDistributed()) {} }; /// Returns the server time zone (timezone in which server runs). - class FunctionServerTimezone : public FunctionConstantBase + class FunctionServerTimezone : public FunctionServerConstantBase { public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionServerConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed()) {} }; /// Returns server uptime in seconds. - class FunctionUptime : public FunctionConstantBase + class FunctionUptime : public FunctionServerConstantBase { public: static constexpr auto name = "uptime"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionUptime(ContextPtr context) : FunctionConstantBase(context->getUptimeSeconds(), context->isDistributed()) {} + explicit FunctionUptime(ContextPtr context) : FunctionServerConstantBase(context->getUptimeSeconds(), context->isDistributed()) {} }; /// version() - returns the current version as a string. - class FunctionVersion : public FunctionConstantBase + class FunctionVersion : public FunctionServerConstantBase { public: static constexpr auto name = "version"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionVersion(ContextPtr context) : FunctionConstantBase(VERSION_STRING, context->isDistributed()) {} + explicit FunctionVersion(ContextPtr context) : FunctionServerConstantBase(VERSION_STRING, context->isDistributed()) {} }; /// revision() - returns the current revision. - class FunctionRevision : public FunctionConstantBase + class FunctionRevision : public FunctionServerConstantBase { public: static constexpr auto name = "revision"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionRevision(ContextPtr context) : FunctionConstantBase(ClickHouseRevision::getVersionRevision(), context->isDistributed()) {} + explicit FunctionRevision(ContextPtr context) : FunctionServerConstantBase(ClickHouseRevision::getVersionRevision(), context->isDistributed()) {} }; - class FunctionZooKeeperSessionUptime : public FunctionConstantBase + class FunctionZooKeeperSessionUptime : public FunctionServerConstantBase { public: static constexpr auto name = "zookeeperSessionUptime"; explicit FunctionZooKeeperSessionUptime(ContextPtr context) - : FunctionConstantBase(context->getZooKeeperSessionUptime(), context->isDistributed()) + : FunctionServerConstantBase(context->getZooKeeperSessionUptime(), context->isDistributed()) { } static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; - class FunctionGetOSKernelVersion : public FunctionConstantBase + class FunctionGetOSKernelVersion : public FunctionServerConstantBase { public: static constexpr auto name = "getOSKernelVersion"; - explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionConstantBase(Poco::Environment::osName() + " " + Poco::Environment::osVersion(), context->isDistributed()) {} + explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionServerConstantBase(Poco::Environment::osName() + " " + Poco::Environment::osVersion(), context->isDistributed()) {} static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; - class FunctionDisplayName : public FunctionConstantBase + class FunctionDisplayName : public FunctionServerConstantBase { public: static constexpr auto name = "displayName"; - explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + explicit FunctionDisplayName(ContextPtr context) : FunctionServerConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } }; } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index ceb506d1bbb..f0de3e69113 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -85,14 +85,14 @@ bool canRemoveConstantFromGroupByKey(const ConstantNode & root) else if (function_node) { /// Do not allow removing constants like `hostName()` - if (!function_node->getFunctionOrThrow()->isDeterministic()) + if (function_node->getFunctionOrThrow()->isServerConstant()) return false; for (const auto & child : function_node->getArguments()) nodes.push(child.get()); } - else - return false; + // else + // return false; } return true; diff --git a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql index ede6e0deed9..89049515a37 100644 --- a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql +++ b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql @@ -30,3 +30,46 @@ SELECT min(dummy) FROM remote('127.0.0.{2,3}', system.one) GROUP BY y; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{1,2}', default, ttt) +GROUP BY c1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{3,2}', default, ttt) +GROUP BY c1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{1,2}', default, ttt) +GROUP BY c1 + 1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{3,2}', default, ttt) +GROUP BY c1 + 1 FORMAT Null; + +CREATE TABLE ttt (hr DateTime, ts DateTime) ENGINE=Memory +as select '2000-01-01' d, d; + +SELECT + count(), + tuple(nullIf(toDateTime(formatDateTime(hr, '%F %T', 'America/Los_Angeles'), 'America/Los_Angeles'), toDateTime(0))) as c1, + defaultValueOfArgumentType(toTimeZone(ts, 'America/Los_Angeles')) as c2, + formatDateTime(hr, '%F %T', 'America/Los_Angeles') as c3 +FROM remote('127.0.0.{1,2}', default, ttt) +GROUP BY c1, c2, c3 FORMAT Null; + +SELECT + count(), + tuple(nullIf(toDateTime(formatDateTime(hr, '%F %T', 'America/Los_Angeles'), 'America/Los_Angeles'), toDateTime(0))) as c1, + defaultValueOfArgumentType(toTimeZone(ts, 'America/Los_Angeles')) as c2, + formatDateTime(hr, '%F %T', 'America/Los_Angeles') as c3 +FROM remote('127.0.0.{3,2}', default, ttt) +GROUP BY c1, c2, c3 FORMAT Null; From dbfc9fc71fb32a8bc14d50ee6f15db224157fd22 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 Jul 2024 13:04:41 +0200 Subject: [PATCH 081/178] Update 03036_dynamic_read_subcolumns.sh --- tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh index a1cb35e4609..65517061b99 100755 --- a/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh +++ b/tests/queries/0_stateless/03036_dynamic_read_subcolumns.sh @@ -7,7 +7,7 @@ CLICKHOUSE_LOG_COMMENT= # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1 --optimize_functions_to_subcolumns=0" +CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --allow_experimental_dynamic_type=1" function test() From 1e9f47707fc7b47703c0e2957da09192bd044e81 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 12 Jul 2024 13:05:40 +0200 Subject: [PATCH 082/178] Fix optimizing array size to subcolumns in dynamic subcolumn --- src/Storages/ColumnsDescription.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index c07583cd39d..da749812167 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -701,15 +701,15 @@ std::optional ColumnsDescription::tryGetColumn(const GetColumns auto jt = subcolumns.get<0>().find(column_name); if (jt != subcolumns.get<0>().end()) return *jt; - } - /// Check for dynamic subcolumns. - auto [ordinary_column_name, dynamic_subcolumn_name] = Nested::splitName(column_name); - it = columns.get<1>().find(ordinary_column_name); - if (it != columns.get<1>().end() && it->type->hasDynamicSubcolumns()) - { - if (auto dynamic_subcolumn_type = it->type->tryGetSubcolumnType(dynamic_subcolumn_name)) - return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type); + /// Check for dynamic subcolumns. + auto [ordinary_column_name, dynamic_subcolumn_name] = Nested::splitName(column_name); + it = columns.get<1>().find(ordinary_column_name); + if (it != columns.get<1>().end() && it->type->hasDynamicSubcolumns()) + { + if (auto dynamic_subcolumn_type = it->type->tryGetSubcolumnType(dynamic_subcolumn_name)) + return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type); + } } return {}; From 64d2b01d4a1a56f7cab978142cb15cf71ff0b4a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 15:51:16 +0000 Subject: [PATCH 083/178] Fixing style --- src/Functions/IFunction.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 23aab420338..12931b51df2 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -233,7 +233,7 @@ public: /** This is a special flags for functions which return constant value for the server, * but the result could be different for different servers in distributed query. * - * This fuctions can't support constant folding on the initiator, but can on the follower. + * This functions can't support constant folding on the initiator, but can on the follower. * We can't apply some optimizations as well (e.g. can't remove constant result from GROUP BY key). * So, it is convenient to have a special flag for them. * From 00fa457a5d14a96b9f3e7ed3b937506f07476789 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jul 2024 17:06:07 +0000 Subject: [PATCH 084/178] Fixing test. --- src/Functions/getMacro.cpp | 2 + .../02992_analyzer_group_by_const.sql | 52 +++++++++---------- 2 files changed, 28 insertions(+), 26 deletions(-) diff --git a/src/Functions/getMacro.cpp b/src/Functions/getMacro.cpp index 8172fc8ba2e..b7f0b34d652 100644 --- a/src/Functions/getMacro.cpp +++ b/src/Functions/getMacro.cpp @@ -53,6 +53,8 @@ public: /// getMacro may return different values on different shards/replicas, so it's not constant for distributed query bool isSuitableForConstantFolding() const override { return !is_distributed; } + bool isServerConstant() const override { return true; } + size_t getNumberOfArguments() const override { return 1; diff --git a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql index 89049515a37..2a9e673d7bc 100644 --- a/tests/queries/0_stateless/02992_analyzer_group_by_const.sql +++ b/tests/queries/0_stateless/02992_analyzer_group_by_const.sql @@ -31,39 +31,39 @@ SELECT FROM remote('127.0.0.{2,3}', system.one) GROUP BY y; -SELECT - count(), - now() AS c1 -FROM remote('127.0.0.{1,2}', default, ttt) -GROUP BY c1 FORMAT Null; - -SELECT - count(), - now() AS c1 -FROM remote('127.0.0.{3,2}', default, ttt) -GROUP BY c1 FORMAT Null; - -SELECT - count(), - now() AS c1 -FROM remote('127.0.0.{1,2}', default, ttt) -GROUP BY c1 + 1 FORMAT Null; - -SELECT - count(), - now() AS c1 -FROM remote('127.0.0.{3,2}', default, ttt) -GROUP BY c1 + 1 FORMAT Null; - CREATE TABLE ttt (hr DateTime, ts DateTime) ENGINE=Memory as select '2000-01-01' d, d; +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{1,2}', currentDatabase(), ttt) +GROUP BY c1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{3,2}', currentDatabase(), ttt) +GROUP BY c1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{1,2}', currentDatabase(), ttt) +GROUP BY c1 + 1 FORMAT Null; + +SELECT + count(), + now() AS c1 +FROM remote('127.0.0.{3,2}', currentDatabase(), ttt) +GROUP BY c1 + 1 FORMAT Null; + SELECT count(), tuple(nullIf(toDateTime(formatDateTime(hr, '%F %T', 'America/Los_Angeles'), 'America/Los_Angeles'), toDateTime(0))) as c1, defaultValueOfArgumentType(toTimeZone(ts, 'America/Los_Angeles')) as c2, formatDateTime(hr, '%F %T', 'America/Los_Angeles') as c3 -FROM remote('127.0.0.{1,2}', default, ttt) +FROM remote('127.0.0.{1,2}', currentDatabase(), ttt) GROUP BY c1, c2, c3 FORMAT Null; SELECT @@ -71,5 +71,5 @@ SELECT tuple(nullIf(toDateTime(formatDateTime(hr, '%F %T', 'America/Los_Angeles'), 'America/Los_Angeles'), toDateTime(0))) as c1, defaultValueOfArgumentType(toTimeZone(ts, 'America/Los_Angeles')) as c2, formatDateTime(hr, '%F %T', 'America/Los_Angeles') as c3 -FROM remote('127.0.0.{3,2}', default, ttt) +FROM remote('127.0.0.{3,2}', currentDatabase(), ttt) GROUP BY c1, c2, c3 FORMAT Null; From 404840bf2c797d8a500d50ad1f2411b330622273 Mon Sep 17 00:00:00 2001 From: Zawa-II Date: Wed, 10 Jul 2024 21:29:09 -0700 Subject: [PATCH 085/178] [ClickHouse-66010] Add-machine_id-to-generateSnowflakeID Extend existing tests and retain expr argument in generateSnowflakeID Commit expected output file Add argument validation in generateSnowflakeID function Style Check Style Check by changing error type Style Check by removing error check Add new test cases for function including parameter Add new test cases for generateSnowflakeID function with machine_id parameter and update reference file Add new test cases for generateSnowflakeID function with machine_id parameter and update reference file to match expected output Fix generateSnowflakeID function to correctly handle optional machine_id parameter, update documentation and examples, and clean up tests. Fix typo in generateSnowflakeID function and correctly handle optional machine_id parameter. --- src/Functions/generateSnowflakeID.cpp | 31 ++++++++++++++----- .../03130_generateSnowflakeId.reference | 4 +++ .../0_stateless/03130_generateSnowflakeId.sql | 9 ++++-- 3 files changed, 34 insertions(+), 10 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index a171b6bf86e..6befdf7a86f 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include "base/types.h" @@ -165,24 +166,37 @@ public: { FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ - {"expr", nullptr, nullptr, "Arbitrary expression"} + {"expr", nullptr, nullptr, "Arbitrary expression"}, + {"machine_id", static_cast(&isNativeUInt), nullptr, "Optional machine ID in UInt*"} }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); + vec_to.resize(input_rows_count); + + uint64_t machine_id = 0; + if (arguments.size() == 2) + machine_id = arguments[1].column->getUInt(0); + + if (machine_id == 0) + machine_id = getMachineId(); + + // Ensure machine_id is within the valid range + machine_id &= (1ull << machine_id_bits_count) - 1; + + // Process expr argument here if necessary (currently a placeholder) if (input_rows_count != 0) { - vec_to.resize(input_rows_count); - Data data; SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range + snowflake_id.machine_id = machine_id; //Assign the provided machine_id for (UInt64 & to_row : vec_to) { @@ -208,10 +222,13 @@ public: REGISTER_FUNCTION(GenerateSnowflakeID) { FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; - FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])"; - FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression, [machine_id]])"; + FunctionDocumentation::Arguments arguments = { + {"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}, + {"machine_id", "A machine ID, the 10 least significant bits are used. Optional."} + }; FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; - FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"multiple", "SELECT generateSnowflakeID(1), generateSnowflakeID(2)", ""}}; + FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", ""}, {"with_machine_id_and_expression", "SELECT generateSnowflakeID('some_expression', 1)", ""}}; FunctionDocumentation::Categories categories = {"Snowflake ID"}; factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 39669d21bee..5cc19c73d88 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,5 +1,9 @@ 1 0 +1 0 1 +1 +1 +1 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index 0717c81aa0d..b28960cfcbf 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -1,12 +1,15 @@ -- Test SQL function 'generateSnowflakeID' SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero - -SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination -SELECT generateSnowflakeID(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs +SELECT generateSnowflakeID(1) != generateSnowflakeID(); -- Check different invocations yield different results + +SELECT generateSnowflakeID('expr', 1) = generateSnowflakeID('expr', 1); -- enabled common subexpression elimination +SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('expr', 2); -- different machine IDs should produce different results +SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('different_expr', 1); -- different expressions should bypass common subexpression elimination SELECT count(*) FROM From 0925830f2edbe85a7aac6d45608551c3efb43731 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 13 Jul 2024 14:01:03 -0700 Subject: [PATCH 086/178] Refactored generateSnowflakeID function implementation and added comprehensive SQL tests --- src/Functions/generateSnowflakeID.cpp | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 6befdf7a86f..6c5124872a0 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -181,22 +181,25 @@ public: vec_to.resize(input_rows_count); uint64_t machine_id = 0; - if (arguments.size() == 2) - machine_id = arguments[1].column->getUInt(0); + if (arguments.size() == 2 && input_rows_count > 0) + { + const auto & column = arguments[1].column; + if (column && !column->empty()) + machine_id = column->getUInt(0); + } if (machine_id == 0) machine_id = getMachineId(); - // Ensure machine_id is within the valid range + /// Ensure machine_id is within the valid range machine_id &= (1ull << machine_id_bits_count) - 1; - // Process expr argument here if necessary (currently a placeholder) + /// Process expr argument here if necessary (currently a placeholder) - if (input_rows_count != 0) + if (input_rows_count > 0) { Data data; - SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range - snowflake_id.machine_id = machine_id; //Assign the provided machine_id + SnowflakeId snowflake_id = data.reserveRange(input_rows_count, machine_id); for (UInt64 & to_row : vec_to) { From 08cce45258b133b2b44c8dd4c3a9c3fe70551e7e Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 13 Jul 2024 14:25:42 -0700 Subject: [PATCH 087/178] Fix handling of machine_id in generateSnowflakeID to read conditionally and assign efficiently --- src/Functions/generateSnowflakeID.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 6c5124872a0..e58d7cfd2d6 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -199,7 +199,8 @@ public: if (input_rows_count > 0) { Data data; - SnowflakeId snowflake_id = data.reserveRange(input_rows_count, machine_id); + SnowflakeId snowflake_id = data.reserveRange(input_rows_count); + snowflake_id.machine_id = machine_id; for (UInt64 & to_row : vec_to) { From 5826f1f073f971031a681637491d14934612b4c4 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 13 Jul 2024 15:39:48 -0700 Subject: [PATCH 088/178] [ClickHouse-66010] Add machine_id parameter to generateSnowflakeID function and update tests --- src/Functions/generateSnowflakeID.cpp | 28 +++++++++++---------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index e58d7cfd2d6..ca639d65cc5 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -8,7 +8,6 @@ #include #include "base/types.h" - namespace DB { @@ -38,7 +37,7 @@ constexpr auto machine_seq_num_bits_count = 12; /// bits masks for Snowflake ID components constexpr uint64_t machine_id_mask = ((1ull << machine_id_bits_count) - 1) << machine_seq_num_bits_count; -constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) - 1; +constexpr uint64_t machine_seq_num_mask = (1ull << machine seq_num_bits_count) - 1; /// max values constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; @@ -97,10 +96,10 @@ struct SnowflakeIdRange /// 1. calculate Snowflake ID by current timestamp (`now`) /// 2. `begin = max(available, now)` /// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow -SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count) +SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count, uint64_t machine_id) { /// 1. `now` - SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0}; + SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = machine_id, .machine_seq_num = 0}; /// 2. `begin` if (begin.timestamp <= available.timestamp) @@ -129,13 +128,13 @@ struct Data /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. static inline std::atomic lowest_available_snowflake_id = 0; - SnowflakeId reserveRange(size_t input_rows_count) + SnowflakeId reserveRange(size_t input_rows_count, uint64_t machine_id) { uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); SnowflakeIdRange range; do { - range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count); + range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count, machine_id); } while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); /// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try @@ -167,7 +166,7 @@ public: FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ {"expr", nullptr, nullptr, "Arbitrary expression"}, - {"machine_id", static_cast(&isNativeUInt), nullptr, "Optional machine ID in UInt*"} + {"machine_id", static_cast(&isNativeUInt), nullptr, "UInt*"} }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); @@ -183,24 +182,19 @@ public: uint64_t machine_id = 0; if (arguments.size() == 2 && input_rows_count > 0) { - const auto & column = arguments[1].column; - if (column && !column->empty()) - machine_id = column->getUInt(0); + const auto & col_machine_id = arguments[1].column; + machine_id = col_machine_id->getUInt(0); } - - if (machine_id == 0) + else machine_id = getMachineId(); - /// Ensure machine_id is within the valid range + /// Truncate machine id to 10 bits machine_id &= (1ull << machine_id_bits_count) - 1; - /// Process expr argument here if necessary (currently a placeholder) - if (input_rows_count > 0) { Data data; - SnowflakeId snowflake_id = data.reserveRange(input_rows_count); - snowflake_id.machine_id = machine_id; + SnowflakeId snowflake_id = data.reserveRange(input_rows_count, machine_id); for (UInt64 & to_row : vec_to) { From 6ed138b1a41b94da4daa456535da719af56c36f7 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 13 Jul 2024 16:06:31 -0700 Subject: [PATCH 089/178] Fix syntax error in generateSnowflakeID.cpp --- src/Functions/generateSnowflakeID.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index ca639d65cc5..3aaa0b50a75 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -37,7 +37,7 @@ constexpr auto machine_seq_num_bits_count = 12; /// bits masks for Snowflake ID components constexpr uint64_t machine_id_mask = ((1ull << machine_id_bits_count) - 1) << machine_seq_num_bits_count; -constexpr uint64_t machine_seq_num_mask = (1ull << machine seq_num_bits_count) - 1; +constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) - 1; /// max values constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; From ace4aa36bf0e3762711616a49d217929e2c808a0 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 09:41:24 -0700 Subject: [PATCH 090/178] Fixed generateSnowflakeID function and tests according Update uuid-functions.md Adjust comments in unit tests --- .../sql-reference/functions/uuid-functions.md | 14 ++++++++++++- src/Functions/generateSnowflakeID.cpp | 20 +++++++++---------- .../03130_generateSnowflakeId.reference | 5 ++--- .../0_stateless/03130_generateSnowflakeId.sql | 11 ++++++---- 4 files changed, 32 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index e990023efbc..f1abb90b455 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -567,13 +567,15 @@ While no standard or recommendation exists for the epoch of Snowflake IDs, imple **Syntax** ``` sql -generateSnowflakeID([expr]) +generateSnowflakeID([expr, [machine_id]]) ``` **Arguments** - `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional. +- `machine_id` — A machine ID, the 10 least significant bits are used. Optional. + **Returned value** A value of type UInt64. @@ -608,6 +610,16 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); └────────────────────────┴────────────────────────┘ ``` +Example with expression and machine ID +``` +SELECT generateSnowflakeID('expr', 1); + +┌─generateSnowflakeID('expr', 1)─┐ +│ 7201148511606784002 │ +└────────────────────────────────┘ +``` + + ## snowflakeToDateTime :::warning diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 3aaa0b50a75..b2f764d9d56 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -96,7 +96,8 @@ struct SnowflakeIdRange /// 1. calculate Snowflake ID by current timestamp (`now`) /// 2. `begin = max(available, now)` /// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow -SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count, uint64_t machine_id) +SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, uint64_t machine_id, size_t input_rows_count) + { /// 1. `now` SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = machine_id, .machine_seq_num = 0}; @@ -134,7 +135,7 @@ struct Data SnowflakeIdRange range; do { - range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count, machine_id); + range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), machine_id, input_rows_count); } while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); /// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try @@ -166,7 +167,9 @@ public: FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ {"expr", nullptr, nullptr, "Arbitrary expression"}, - {"machine_id", static_cast(&isNativeUInt), nullptr, "UInt*"} + {"machine_id", static_cast(&isNativeUInt), static_cast(&isColumnConst), "UInt*"} + {"machine_id", static_cast(&isNativeUInt), static_cast(&isColumnConst), "const UInt*"} + }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); @@ -179,17 +182,14 @@ public: typename ColumnVector::Container & vec_to = col_res->getData(); vec_to.resize(input_rows_count); - uint64_t machine_id = 0; + uint64_t machine_id = getMachineId(); if (arguments.size() == 2 && input_rows_count > 0) { const auto & col_machine_id = arguments[1].column; machine_id = col_machine_id->getUInt(0); + /// Truncate machine id to 10 bits + machine_id &= (1ull << machine_id_bits_count) - 1; } - else - machine_id = getMachineId(); - - /// Truncate machine id to 10 bits - machine_id &= (1ull << machine_id_bits_count) - 1; if (input_rows_count > 0) { @@ -226,7 +226,7 @@ REGISTER_FUNCTION(GenerateSnowflakeID) {"machine_id", "A machine ID, the 10 least significant bits are used. Optional."} }; FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; - FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", ""}, {"with_machine_id_and_expression", "SELECT generateSnowflakeID('some_expression', 1)", ""}}; + FunctionDocumentation::Examples examples = {{"no_arguments", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", "7201148511606784001"}, {"with_expression_and_machine_id", "SELECT generateSnowflakeID('some_expression', 1)", "7201148511606784002"}}; FunctionDocumentation::Categories categories = {"Snowflake ID"}; factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 5cc19c73d88..35592351ac6 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,9 +1,8 @@ 1 -0 1 0 1 1 +0 1 -1 -100 +100 \ No newline at end of file diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index b28960cfcbf..962faec46a8 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -2,14 +2,17 @@ SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ -SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination - SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs -SELECT generateSnowflakeID(1) != generateSnowflakeID(); -- Check different invocations yield different results +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination SELECT generateSnowflakeID('expr', 1) = generateSnowflakeID('expr', 1); -- enabled common subexpression elimination SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('expr', 2); -- different machine IDs should produce different results -SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('different_expr', 1); -- different expressions should bypass common subexpression elimination + +SELECT bitAnd(generateSnowflakeID(1023), 1023) = 1023; -- check if the last 10 bits match the machine ID + +SELECT generateSnowflakeID('invalid_machine_id'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT generateSnowflakeID(materialize(toUInt64(1))) = generateSnowflakeID(materialize(toUInt64(1))); -- -- test with non-const machine ID SELECT count(*) FROM From 27273be84f3d252562646384d8db1aab6206d4a0 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 09:51:13 -0700 Subject: [PATCH 091/178] Removed Duplicte args specifications --- src/Functions/generateSnowflakeID.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index b2f764d9d56..ada677a9355 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -167,9 +167,7 @@ public: FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ {"expr", nullptr, nullptr, "Arbitrary expression"}, - {"machine_id", static_cast(&isNativeUInt), static_cast(&isColumnConst), "UInt*"} {"machine_id", static_cast(&isNativeUInt), static_cast(&isColumnConst), "const UInt*"} - }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); From 5bce8086a86716f03ceb36f22412ed21852c610e Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 14 Jul 2024 19:02:20 +0200 Subject: [PATCH 092/178] CI: Create release workflow updates --- tests/ci/create_release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 7f4cf8c787a..cd01b646ebe 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -120,7 +120,7 @@ class ReleaseInfo: expected_version = copy(version) expected_version.bump() expected_tag_prefix = ( - f"v{expected_version.major}.{expected_version.minor}-" + f"v{expected_version.major}.{expected_version.minor}." ) expected_tag_suffix = "-new" else: From 92b2a7d33d153cafd7dc6eb17cb0d2501d886245 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 10:34:39 -0700 Subject: [PATCH 093/178] Updated generateSnowflakeID tests and reference outputs --- src/Functions/generateSnowflakeID.cpp | 6 ++++++ .../queries/0_stateless/03130_generateSnowflakeId.reference | 3 ++- tests/queries/0_stateless/03130_generateSnowflakeId.sql | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index ada677a9355..6ee61df7e3b 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -184,7 +184,13 @@ public: if (arguments.size() == 2 && input_rows_count > 0) { const auto & col_machine_id = arguments[1].column; + if (!isColumnConst(*col_machine_id)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The machine_id argument must be a constant UInt64"); + machine_id = col_machine_id->getUInt(0); + if (col_machine_id->getDataType() != TypeIndex::UInt64) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The machine_id argument must be of type UInt64"); + /// Truncate machine id to 10 bits machine_id &= (1ull << machine_id_bits_count) - 1; } diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 35592351ac6..dff613b29d3 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -5,4 +5,5 @@ 1 0 1 -100 \ No newline at end of file +1 +100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index 962faec46a8..be49f678dca 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -12,7 +12,7 @@ SELECT bitAnd(generateSnowflakeID(1023), 1023) = 1023; -- check if the last 10 b SELECT generateSnowflakeID('invalid_machine_id'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT generateSnowflakeID(materialize(toUInt64(1))) = generateSnowflakeID(materialize(toUInt64(1))); -- -- test with non-const machine ID +SELECT generateSnowflakeID(materialize(toUInt64(1))) = generateSnowflakeID(materialize(toUInt64(1))); -- Test with non-const machine ID SELECT count(*) FROM From 5ea81366a780557f11459573e24b774494145da7 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 10:55:57 -0700 Subject: [PATCH 094/178] Fix generateSnowflakeID function to handle invalid inputs gracefully and avoid output for invalid cases --- src/Functions/generateSnowflakeID.cpp | 15 +++++++++------ .../03130_generateSnowflakeId.reference | 1 - .../0_stateless/03130_generateSnowflakeId.sql | 4 ++-- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 6ee61df7e3b..2db40b10496 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -18,7 +18,7 @@ namespace https://en.wikipedia.org/wiki/Snowflake_ID 0 1 2 3 - 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 ├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ |0| timestamp | ├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ @@ -184,13 +184,16 @@ public: if (arguments.size() == 2 && input_rows_count > 0) { const auto & col_machine_id = arguments[1].column; - if (!isColumnConst(*col_machine_id)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The machine_id argument must be a constant UInt64"); + + // Check if the provided machine_id is a constant UInt64 + if (!isColumnConst(*col_machine_id) || !isNativeUInt(col_machine_id->getDataType())) + { + // If invalid machine_id, resize vector to 0 + vec_to.resize(0); + return col_res; + } machine_id = col_machine_id->getUInt(0); - if (col_machine_id->getDataType() != TypeIndex::UInt64) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The machine_id argument must be of type UInt64"); - /// Truncate machine id to 10 bits machine_id &= (1ull << machine_id_bits_count) - 1; } diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index dff613b29d3..b8a5829fed7 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -5,5 +5,4 @@ 1 0 1 -1 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index be49f678dca..c641cfd4bf1 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -10,9 +10,9 @@ SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('expr', 2); -- diff SELECT bitAnd(generateSnowflakeID(1023), 1023) = 1023; -- check if the last 10 bits match the machine ID -SELECT generateSnowflakeID('invalid_machine_id'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT generateSnowflakeID('invalid_machine_id'); -- no output for invalid type -SELECT generateSnowflakeID(materialize(toUInt64(1))) = generateSnowflakeID(materialize(toUInt64(1))); -- Test with non-const machine ID +SELECT generateSnowflakeID(materialize(toUInt64(1))); -- no output for non-const machine ID SELECT count(*) FROM From 1c868356b8a1ce6e698547f7c1d2b19b2d14c095 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 11:38:11 -0700 Subject: [PATCH 095/178] Fix generateSnowflakeID function to handle invalid inputs gracefully and update test queries --- src/Functions/generateSnowflakeID.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 2db40b10496..812c992203d 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -184,15 +184,12 @@ public: if (arguments.size() == 2 && input_rows_count > 0) { const auto & col_machine_id = arguments[1].column; - - // Check if the provided machine_id is a constant UInt64 - if (!isColumnConst(*col_machine_id) || !isNativeUInt(col_machine_id->getDataType())) + if (!isColumnConst(*col_machine_id)) { - // If invalid machine_id, resize vector to 0 - vec_to.resize(0); + // Return an empty column if machine ID is not constant + vec_to.clear(); return col_res; } - machine_id = col_machine_id->getUInt(0); /// Truncate machine id to 10 bits machine_id &= (1ull << machine_id_bits_count) - 1; From 2ddd0078dcf5f33004cfb558328303c29306a32a Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 14 Jul 2024 21:07:41 +0200 Subject: [PATCH 096/178] rename reusable docker workflow --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/create_release.yml | 2 +- .../workflows/{reusable_docker.yml => docker_test_images.yml} | 0 .github/workflows/master.yml | 2 +- .github/workflows/merge_queue.yml | 2 +- .github/workflows/nightly.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release_branches.yml | 2 +- tests/ci/create_release.py | 2 ++ 9 files changed, 9 insertions(+), 7 deletions(-) rename .github/workflows/{reusable_docker.yml => docker_test_images.yml} (100%) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 64c3d2f8342..60bd79560eb 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -62,7 +62,7 @@ jobs: BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() }} - uses: ./.github/workflows/reusable_docker.yml + uses: ./.github/workflows/docker_test_images.yml with: data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckX86: diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 972aff90195..d8d27531f28 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -94,7 +94,7 @@ jobs: echo "Generate Security" python3 ./utils/security-generator/generate_security.py > SECURITY.md git diff HEAD - - name: Generate ChangeLog + - name: Create ChangeLog PR if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} uses: peter-evans/create-pull-request@v6 with: diff --git a/.github/workflows/reusable_docker.yml b/.github/workflows/docker_test_images.yml similarity index 100% rename from .github/workflows/reusable_docker.yml rename to .github/workflows/docker_test_images.yml diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2a7e6f737ab..d27b1987532 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -58,7 +58,7 @@ jobs: # BuildDockers: # needs: [RunConfig] # if: ${{ !failure() && !cancelled() }} -# uses: ./.github/workflows/reusable_docker.yml +# uses: ./.github/workflows/docker_test_images.yml # with: # data: ${{ needs.RunConfig.outputs.data }} # StyleCheck: diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index 01685ee1f5a..c08c3fb30ac 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -51,7 +51,7 @@ jobs: BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} - uses: ./.github/workflows/reusable_docker.yml + uses: ./.github/workflows/docker_test_images.yml with: data: ${{ needs.RunConfig.outputs.data }} StyleCheck: diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 3e1c5576e7d..bffe5b4c1bf 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -40,7 +40,7 @@ jobs: } >> "$GITHUB_OUTPUT" BuildDockers: needs: [RunConfig] - uses: ./.github/workflows/reusable_docker.yml + uses: ./.github/workflows/docker_test_images.yml with: data: "${{ needs.RunConfig.outputs.data }}" set_latest: true diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index c9f4f858825..04ce4d29ce9 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -72,7 +72,7 @@ jobs: BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} - uses: ./.github/workflows/reusable_docker.yml + uses: ./.github/workflows/docker_test_images.yml with: data: ${{ needs.RunConfig.outputs.data }} StyleCheck: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 6bf846d7535..e1a97b91016 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -57,7 +57,7 @@ jobs: BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() }} - uses: ./.github/workflows/reusable_docker.yml + uses: ./.github/workflows/docker_test_images.yml with: data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckX86: diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index cd01b646ebe..f377e12d11c 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -618,6 +618,8 @@ sudo apt install --yes --no-install-recommends python3-dev python3-pip gh unzip sudo apt install --yes python3-boto3 sudo apt install --yes python3-github sudo apt install --yes python3-unidiff +sudo apt install --yes python3-tqdm # cloud changelog +sudo apt install --yes python3-thefuzz # cloud changelog sudo apt install --yes s3fs ### INSTALL AWS CLI From 619152eadb702b23a439517e0ab0e848d1ade466 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 14 Jul 2024 14:13:01 -0700 Subject: [PATCH 097/178] Retry with returning nullptr --- src/Functions/generateSnowflakeID.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 812c992203d..3b68e634708 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -185,11 +185,7 @@ public: { const auto & col_machine_id = arguments[1].column; if (!isColumnConst(*col_machine_id)) - { - // Return an empty column if machine ID is not constant - vec_to.clear(); - return col_res; - } + return nullptr; machine_id = col_machine_id->getUInt(0); /// Truncate machine id to 10 bits machine_id &= (1ull << machine_id_bits_count) - 1; From 24545caa3a7898e754d7b1f002c95132dc6a2d8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 15 Jul 2024 00:52:31 +0000 Subject: [PATCH 098/178] Move parsing into a separate function --- .../Access/InterpreterCreateUserQuery.cpp | 59 ++++++++++--------- 1 file changed, 30 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index a09de7e688a..855aa36b159 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -114,6 +114,34 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } + + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } } BlockIO InterpreterCreateUserQuery::execute() @@ -134,23 +162,7 @@ BlockIO InterpreterCreateUserQuery::execute() std::optional valid_until; if (query.valid_until) - { - const ASTPtr valid_until_literal = evaluateConstantExpressionAsLiteral(query.valid_until, getContext()); - const String valid_until_str = checkAndGetLiteralArgument(valid_until_literal, "valid_until"); - - time_t time = 0; - - if (valid_until_str != "infinity") - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - ReadBufferFromString in(valid_until_str); - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - - valid_until = time; - } + valid_until = getValidUntilFromAST(query.valid_until, getContext()); std::optional default_roles_from_query; if (query.default_roles) @@ -261,18 +273,7 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat std::optional valid_until; if (query.valid_until) - { - const String valid_until_str = checkAndGetLiteralArgument(query.valid_until, "valid_until"); - time_t time = 0; - - if (valid_until_str != "infinity") - { - ReadBufferFromString in(valid_until_str); - readDateTimeText(time, in); - } - - valid_until = time; - } + valid_until = getValidUntilFromAST(query.valid_until, {}); updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true); } From 20faa70389a2a7e3d4c30d3d647e96129e1eb8e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 03:30:03 +0200 Subject: [PATCH 099/178] Fix bad test `02530_dictionaries_update_field` --- .../02530_dictionaries_update_field.reference | 60 ------------------- .../02530_dictionaries_update_field.sh | 35 ++++++----- 2 files changed, 21 insertions(+), 74 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 4d5a7447a49..ada13443a10 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -1,102 +1,42 @@ flat --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third flat/custom --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third hashed --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third hashed/custom --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third complex_key_hashed --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third complex_key_hashed/custom --- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); -SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First -INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second -INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); -INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; -SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated 3 Third diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 9ced78a1196..a8e5804fba4 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -35,7 +35,7 @@ for layout in "${layouts[@]}"; do echo "$layout" fi - $CLICKHOUSE_CLIENT -nm -q " + $CLICKHOUSE_CLIENT --multiquery " TRUNCATE TABLE table_for_update_field_dictionary; CREATE DICTIONARY $dictionary_name @@ -49,24 +49,31 @@ for layout in "${layouts[@]}"; do LAYOUT($layout()) LIFETIME(1); - -- { echoOn } - INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now()); - SELECT key, value FROM $dictionary_name ORDER BY key ASC; + INSERT INTO table_for_update_field_dictionary VALUES (1, 'First', now());" - INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); - SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; + while true + do + $CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'First' && break; + sleep .1; + done - SELECT key, value FROM $dictionary_name ORDER BY key ASC; + $CLICKHOUSE_CLIENT --query "INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now());" - INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); - INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(20) SETTINGS function_sleep_max_microseconds_per_block = 20000000 FORMAT Null; + while true + do + $CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'Second' && break; + sleep .1; + done - SELECT key, value FROM $dictionary_name ORDER BY key ASC; - -- { echoOff } + $CLICKHOUSE_CLIENT --query "INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()), (3, 'Third', now())" - DROP DICTIONARY $dictionary_name; - " + while true + do + $CLICKHOUSE_CLIENT --query "SELECT key, value FROM $dictionary_name ORDER BY key ASC" | grep -A10 -B10 'SecondUpdated' && break; + sleep .1; + done + + $CLICKHOUSE_CLIENT --query "DROP DICTIONARY $dictionary_name" done done From 41327260bad798494fef0914b51d95fa85043057 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 03:47:35 +0200 Subject: [PATCH 100/178] Fix MSan report in GRPC --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index f5b7fdc2dff..1716359d2e2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df +Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db From e55fcb32654dfd9cf1f22cd2411a2c48eea7aced Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 01:48:55 +0000 Subject: [PATCH 101/178] add docs to client xml config --- programs/client/clickhouse-client.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index d0deb818c1e..c32b63413e9 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -1,5 +1,6 @@ + true @@ -72,6 +73,7 @@ Default: "hostname" will be used. --> default + 127.0.0.1 9000 From bc1b19591d1c78ba9b782722a27c1d29ce1d8d7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jul 2024 05:13:38 +0200 Subject: [PATCH 102/178] Disable suspend on Ctrl+Z --- src/Client/ClientBase.cpp | 5 +++-- src/Client/LineReader.h | 3 +-- src/Client/ReplxxLineReader.cpp | 3 +++ 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d472ba99b9..c6a070219a3 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -80,6 +80,7 @@ #include #include "config.h" + namespace fs = std::filesystem; using namespace std::literals; @@ -2565,12 +2566,12 @@ void ClientBase::runInteractive() word_break_characters, highlight_callback); #else + (void)word_break_characters; LineReader lr( history_file, getClientConfiguration().has("multiline"), query_extenders, - query_delimiters, - word_break_characters); + query_delimiters); #endif static const std::initializer_list> backslash_aliases = diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h index 0172bd7ec22..8c101401190 100644 --- a/src/Client/LineReader.h +++ b/src/Client/LineReader.h @@ -46,8 +46,7 @@ public: Patterns delimiters, std::istream & input_stream_ = std::cin, std::ostream & output_stream_ = std::cout, - int in_fd_ = STDIN_FILENO - ); + int in_fd_ = STDIN_FILENO); virtual ~LineReader() = default; diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index 46600168695..3b3508d1a58 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -362,6 +362,9 @@ ReplxxLineReader::ReplxxLineReader( rx.bind_key(Replxx::KEY::control('N'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_NEXT, code); }); rx.bind_key(Replxx::KEY::control('P'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::HISTORY_PREVIOUS, code); }); + /// We don't want the default, "suspend" behavior, it confuses people. + rx.bind_key_internal(replxx::Replxx::KEY::control('Z'), "insert_character"); + auto commit_action = [this](char32_t code) { /// If we allow multiline and there is already something in the input, start a newline. From 5b0bcf79f024514fbc2e2d7850d9eb1fdfd25e64 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 15 Jul 2024 06:39:33 +0000 Subject: [PATCH 103/178] Some fixups --- .../sql-reference/functions/uuid-functions.md | 7 ++--- src/Functions/generateSnowflakeID.cpp | 29 +++++++++---------- .../03130_generateSnowflakeId.reference | 9 ++++-- .../0_stateless/03130_generateSnowflakeId.sql | 25 +++++++++------- 4 files changed, 36 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index f1abb90b455..0bde207dcc9 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -573,8 +573,7 @@ generateSnowflakeID([expr, [machine_id]]) **Arguments** - `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional. - -- `machine_id` — A machine ID, the 10 least significant bits are used. Optional. +- `machine_id` — A machine ID, the lowest 10 bits are used. [Int64](../data-types/int-uint.md). Optional. **Returned value** @@ -610,7 +609,8 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); └────────────────────────┴────────────────────────┘ ``` -Example with expression and machine ID +**Example with expression and a machine ID** + ``` SELECT generateSnowflakeID('expr', 1); @@ -619,7 +619,6 @@ SELECT generateSnowflakeID('expr', 1); └────────────────────────────────┘ ``` - ## snowflakeToDateTime :::warning diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 3b68e634708..c95e3edf4ca 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -18,7 +18,7 @@ namespace https://en.wikipedia.org/wiki/Snowflake_ID 0 1 2 3 - 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 ├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ |0| timestamp | ├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ @@ -129,7 +129,7 @@ struct Data /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. static inline std::atomic lowest_available_snowflake_id = 0; - SnowflakeId reserveRange(size_t input_rows_count, uint64_t machine_id) + SnowflakeId reserveRange(uint64_t machine_id, size_t input_rows_count) { uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); SnowflakeIdRange range; @@ -178,23 +178,20 @@ public: { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - vec_to.resize(input_rows_count); - - uint64_t machine_id = getMachineId(); - if (arguments.size() == 2 && input_rows_count > 0) - { - const auto & col_machine_id = arguments[1].column; - if (!isColumnConst(*col_machine_id)) - return nullptr; - machine_id = col_machine_id->getUInt(0); - /// Truncate machine id to 10 bits - machine_id &= (1ull << machine_id_bits_count) - 1; - } if (input_rows_count > 0) { + vec_to.resize(input_rows_count); + + uint64_t machine_id = getMachineId(); + if (arguments.size() == 2) + { + machine_id = arguments[1].column->getUInt(0); + machine_id &= (1ull << machine_id_bits_count) - 1; + } + Data data; - SnowflakeId snowflake_id = data.reserveRange(input_rows_count, machine_id); + SnowflakeId snowflake_id = data.reserveRange(machine_id, input_rows_count); for (UInt64 & to_row : vec_to) { @@ -223,7 +220,7 @@ REGISTER_FUNCTION(GenerateSnowflakeID) FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression, [machine_id]])"; FunctionDocumentation::Arguments arguments = { {"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}, - {"machine_id", "A machine ID, the 10 least significant bits are used. Optional."} + {"machine_id", "A machine ID, the lowest 10 bits are used. Optional."} }; FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; FunctionDocumentation::Examples examples = {{"no_arguments", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", "7201148511606784001"}, {"with_expression_and_machine_id", "SELECT generateSnowflakeID('some_expression', 1)", "7201148511606784002"}}; diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index b8a5829fed7..fd264f00d36 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,8 +1,11 @@ +Negative tests +The first bit must be zero 1 -1 +Test disabling of common subexpression elimination via first parameter +0 0 1 +Test user-provided machine ID 1 -0 -1 +Generated Snowflake IDs are unique 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index c641cfd4bf1..6c0a6cc0f2c 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -1,19 +1,22 @@ -- Test SQL function 'generateSnowflakeID' -SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero -SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ +SELECT 'Negative tests'; +SELECT generateSnowflakeID(1, 2, 3); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT generateSnowflakeID(1, 'not_an_int'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT generateSnowflakeID(1, materialize(2)); -- { serverError ILLEGAL_COLUMN } + + SELECT 'The first bit must be zero'; +SELECT bitAnd(bitShiftRight(generateSnowflakeID(), 63), 1) = 0; + +SELECT 'Test disabling of common subexpression elimination via first parameter'; SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs -SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination +SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- with common subexpression elimination -SELECT generateSnowflakeID('expr', 1) = generateSnowflakeID('expr', 1); -- enabled common subexpression elimination -SELECT generateSnowflakeID('expr', 1) != generateSnowflakeID('expr', 2); -- different machine IDs should produce different results - -SELECT bitAnd(generateSnowflakeID(1023), 1023) = 1023; -- check if the last 10 bits match the machine ID - -SELECT generateSnowflakeID('invalid_machine_id'); -- no output for invalid type - -SELECT generateSnowflakeID(materialize(toUInt64(1))); -- no output for non-const machine ID +SELECT 'Test user-provided machine ID'; +SELECT bitAnd(bitShiftRight(generateSnowflakeID(1, 123), 12), 1024 - 1) = 123; -- the machine id is actually set in the generated snowflake ID (1024 = 2^10) +SELECT 'Generated Snowflake IDs are unique'; SELECT count(*) FROM ( From 4fd832fe482eebd909e642fe2ae2e7468cccebe2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 15 Jul 2024 09:57:32 +0200 Subject: [PATCH 104/178] Fix 02918_parallel_replicas_custom_key_unavailable_replica --- ...02918_parallel_replicas_custom_key_unavailable_replica.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql index b9bc6974c47..c36b5bebd58 100644 --- a/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql +++ b/tests/queries/0_stateless/02918_parallel_replicas_custom_key_unavailable_replica.sql @@ -6,6 +6,9 @@ INSERT INTO 02918_parallel_replicas SELECT toString(number), number % 4 FROM num SET prefer_localhost_replica=0; +--- if we try to query unavaialble replica, connection will be retried +--- but a warning log message will be printed out +SET send_logs_level='error'; -- { echoOn } SELECT y, count() FROM cluster(test_cluster_1_shard_3_replicas_1_unavailable, currentDatabase(), 02918_parallel_replicas) @@ -26,5 +29,6 @@ GROUP BY y ORDER BY y SETTINGS max_parallel_replicas=3, parallel_replicas_custom_key='cityHash64(y)', parallel_replicas_custom_key_filter_type='default'; -- { echoOff } +SET send_logs_level='warning'; DROP TABLE 02918_parallel_replicas; From 2af0edd9e9a509e1bffa15e8da8454a4feb7f0ed Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 15 Jul 2024 08:47:08 +0000 Subject: [PATCH 105/178] Some minor fixups --- base/base/cgroupsv2.cpp | 14 +++++++------- base/base/cgroupsv2.h | 7 +++---- base/base/getMemoryAmount.cpp | 3 +-- src/Common/CgroupsMemoryUsageObserver.cpp | 12 ++++++------ src/Common/getNumberOfPhysicalCPUCores.cpp | 2 +- 5 files changed, 18 insertions(+), 20 deletions(-) diff --git a/base/base/cgroupsv2.cpp b/base/base/cgroupsv2.cpp index 466ebbc3ffb..87f62bf377d 100644 --- a/base/base/cgroupsv2.cpp +++ b/base/base/cgroupsv2.cpp @@ -3,8 +3,9 @@ #include #include -#include +#include +namespace fs = std::filesystem; bool cgroupsV2Enabled() { @@ -13,11 +14,11 @@ bool cgroupsV2Enabled() { /// This file exists iff the host has cgroups v2 enabled. auto controllers_file = default_cgroups_mount / "cgroup.controllers"; - if (!std::filesystem::exists(controllers_file)) + if (!fs::exists(controllers_file)) return false; return true; } - catch (const std::filesystem::filesystem_error &) /// all "underlying OS API errors", typically: permission denied + catch (const fs::filesystem_error &) /// all "underlying OS API errors", typically: permission denied { return false; /// not logging the exception as most callers fall back to cgroups v1 } @@ -33,7 +34,7 @@ bool cgroupsV2MemoryControllerEnabled() /// According to https://docs.kernel.org/admin-guide/cgroup-v2.html, file "cgroup.controllers" defines which controllers are available /// for the current + child cgroups. The set of available controllers can be restricted from level to level using file /// "cgroups.subtree_control". It is therefore sufficient to check the bottom-most nested "cgroup.controllers" file. - auto cgroup_dir = currentCGroupV2Path(); + fs::path cgroup_dir = cgroupV2PathOfProcess(); if (cgroup_dir.empty()) return false; std::ifstream controllers_file(cgroup_dir / "cgroup.controllers"); @@ -47,7 +48,7 @@ bool cgroupsV2MemoryControllerEnabled() #endif } -std::filesystem::path currentCGroupV2Path() +fs::path cgroupV2PathOfProcess() { #if defined(OS_LINUX) chassert(cgroupsV2Enabled()); @@ -63,9 +64,8 @@ std::filesystem::path currentCGroupV2Path() static const std::string v2_prefix = "0::/"; if (!cgroup.starts_with(v2_prefix)) return {}; - - // the 'root' cgroup can have empty path, which is valid cgroup = cgroup.substr(v2_prefix.length()); + /// Note: The 'root' cgroup can have an empty cgroup name, this is valid return default_cgroups_mount / cgroup; #else return {}; diff --git a/base/base/cgroupsv2.h b/base/base/cgroupsv2.h index 2c58682ce31..cfb916ff358 100644 --- a/base/base/cgroupsv2.h +++ b/base/base/cgroupsv2.h @@ -1,7 +1,6 @@ #pragma once #include -#include #if defined(OS_LINUX) /// I think it is possible to mount the cgroups hierarchy somewhere else (e.g. when in containers). @@ -16,7 +15,7 @@ bool cgroupsV2Enabled(); /// Assumes that cgroupsV2Enabled() is enabled. bool cgroupsV2MemoryControllerEnabled(); -/// Detects which cgroup the process belong and returns the path to it in sysfs (for cgroups v2). -/// Returns an empty path if the cgroup cannot be determined. +/// Detects which cgroup v2 the process belongs to and returns the filesystem path to the cgroup. +/// Returns an empty path the cgroup cannot be determined. /// Assumes that cgroupsV2Enabled() is enabled. -std::filesystem::path currentCGroupV2Path(); +std::filesystem::path cgroupV2PathOfProcess(); diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 9bd5ad75445..afdb6ba068a 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -23,8 +23,7 @@ std::optional getCgroupsV2MemoryLimit() if (!cgroupsV2MemoryControllerEnabled()) return {}; - auto current_cgroup = currentCGroupV2Path(); - + std::filesystem::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index e034319b21f..02bde0d80b7 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -25,6 +25,7 @@ #endif using namespace DB; +namespace fs = std::filesystem; namespace DB { @@ -69,7 +70,7 @@ uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & ke struct CgroupsV1Reader : ICgroupsReader { - explicit CgroupsV1Reader(const std::filesystem::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } uint64_t readMemoryUsage() override { @@ -85,7 +86,7 @@ private: struct CgroupsV2Reader : ICgroupsReader { - explicit CgroupsV2Reader(const std::filesystem::path & stat_file_dir) + explicit CgroupsV2Reader(const fs::path & stat_file_dir) : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") { } @@ -129,8 +130,7 @@ std::optional getCgroupsV2Path() if (!cgroupsV2MemoryControllerEnabled()) return {}; - auto current_cgroup = currentCGroupV2Path(); - + fs::path current_cgroup = cgroupV2PathOfProcess(); if (current_cgroup.empty()) return {}; @@ -140,7 +140,7 @@ std::optional getCgroupsV2Path() { const auto current_path = current_cgroup / "memory.current"; const auto stat_path = current_cgroup / "memory.stat"; - if (std::filesystem::exists(current_path) && std::filesystem::exists(stat_path)) + if (fs::exists(current_path) && fs::exists(stat_path)) return {current_cgroup}; current_cgroup = current_cgroup.parent_path(); } @@ -150,7 +150,7 @@ std::optional getCgroupsV2Path() std::optional getCgroupsV1Path() { auto path = default_cgroups_mount / "memory/memory.stat"; - if (!std::filesystem::exists(path)) + if (!fs::exists(path)) return {}; return {default_cgroups_mount / "memory"}; } diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index b16c635f23e..34a1add2f0e 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -38,7 +38,7 @@ uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) if (cgroupsV2Enabled()) { /// First, we identify the path of the cgroup the process belongs - auto cgroup_path = currentCGroupV2Path(); + std::filesystem::path cgroup_path = cgroupV2PathOfProcess(); if (cgroup_path.empty()) return default_cpu_count; From 0dfcaaa28fd4717fcf217d771333077e86401529 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 15 Jul 2024 13:05:53 +0200 Subject: [PATCH 106/178] work with review notes --- src/Interpreters/Squashing.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index a076494a4bc..5fe0ba9a737 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -18,7 +18,6 @@ Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_bloc , min_block_size_bytes(min_block_size_bytes_) , header(header_) { - LOG_TEST(getLogger("Squashing"), "header columns {}", header.columns()); } Chunk Squashing::flush() @@ -46,8 +45,6 @@ Chunk Squashing::squash(Chunk && input_chunk) Chunk Squashing::add(Chunk && input_chunk) { - LOG_TEST(getLogger("Squashing"), "add columns {} rows {}", input_chunk.getNumColumns(), input_chunk.getNumRows()); - if (!input_chunk) return {}; @@ -88,8 +85,6 @@ Chunk Squashing::add(Chunk && input_chunk) Chunk Squashing::convertToChunk(CurrentData && data) const { - LOG_TEST(getLogger("Squashing"), "convertToChunk {}", data.chunks.size()); - if (data.chunks.empty()) return {}; @@ -99,7 +94,10 @@ Chunk Squashing::convertToChunk(CurrentData && data) const // It is imortant that chunk is not empty, it has to have columns even if they are empty // Sometimes there are could be no columns in header but not empty rows in chunks // That happens when we intend to add defaults for the missing columns after - auto aggr_chunk = Chunk(header.getColumns(), header.columns() ? 0 : data.getRows()); + auto aggr_chunk = Chunk(header.getColumns(), 0); + if (header.columns() == 0) + aggr_chunk = Chunk(header.getColumns(), data.getRows()); + aggr_chunk.getChunkInfos().add(std::move(info)); chassert(aggr_chunk); return aggr_chunk; From de029e278ad4461c7ce981146fdc1dcf18ff0b25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 15 Jul 2024 13:30:48 +0200 Subject: [PATCH 107/178] Add additional log masking --- tests/ci/functional_test_check.py | 14 ++++++++++++++ tests/ci/stress_check.py | 15 +++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index ef9f4dc016e..3531a55f8ae 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -24,6 +24,18 @@ from tee_popen import TeePopen NO_CHANGES_MSG = "Nothing to run" +class SensitiveFormatter(logging.Formatter): + @staticmethod + def _filter(s): + return re.sub( + r"(.*)(AZURE_CONNECTION_STRING.*\')(.*)", r"\1AZURE_CONNECTION_STRING\3", s + ) + + def format(self, record): + original = logging.Formatter.format(self, record) + return self._filter(original) + + def get_additional_envs( check_name: str, run_by_hash_num: int, run_by_hash_total: int ) -> List[str]: @@ -213,6 +225,8 @@ def parse_args(): def main(): logging.basicConfig(level=logging.INFO) + for handler in logging.root.handlers: + handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) stopwatch = Stopwatch() diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 486bfc25e22..83a410551a1 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -3,6 +3,7 @@ import csv import logging import os +import re import subprocess import sys from pathlib import Path @@ -19,6 +20,18 @@ from stopwatch import Stopwatch from tee_popen import TeePopen +class SensitiveFormatter(logging.Formatter): + @staticmethod + def _filter(s): + return re.sub( + r"(.*)(AZURE_CONNECTION_STRING.*\')(.*)", r"\1AZURE_CONNECTION_STRING\3", s + ) + + def format(self, record): + original = logging.Formatter.format(self, record) + return self._filter(original) + + def get_additional_envs(check_name: str) -> List[str]: result = [] azure_connection_string = get_parameter_from_ssm("azure_connection_string") @@ -117,6 +130,8 @@ def process_results( def run_stress_test(docker_image_name: str) -> None: logging.basicConfig(level=logging.INFO) + for handler in logging.root.handlers: + handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) stopwatch = Stopwatch() temp_path = Path(TEMP_PATH) From b6b4b046a39994242abbd7d8637ddfb17b0c60a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 15 Jul 2024 13:42:00 +0200 Subject: [PATCH 108/178] Style --- tests/ci/functional_test_check.py | 3 ++- tests/ci/stress_check.py | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 3531a55f8ae..41c7ed963c9 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -226,7 +226,8 @@ def parse_args(): def main(): logging.basicConfig(level=logging.INFO) for handler in logging.root.handlers: - handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) + # pylint: disable=protected-access + handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) # type: ignore stopwatch = Stopwatch() diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 83a410551a1..85da601e379 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -131,7 +131,8 @@ def process_results( def run_stress_test(docker_image_name: str) -> None: logging.basicConfig(level=logging.INFO) for handler in logging.root.handlers: - handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) + # pylint: disable=protected-access + handler.setFormatter(SensitiveFormatter(handler.formatter._fmt)) # type: ignore stopwatch = Stopwatch() temp_path = Path(TEMP_PATH) From bcd08b89531ba0734a689be64f41b46064f0f8b9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 15 Jul 2024 12:50:53 +0000 Subject: [PATCH 109/178] Automatic style fix --- tests/queries/0_stateless/03008_deduplication.python | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03008_deduplication.python b/tests/queries/0_stateless/03008_deduplication.python index dd1058518c9..bfaa496805f 100644 --- a/tests/queries/0_stateless/03008_deduplication.python +++ b/tests/queries/0_stateless/03008_deduplication.python @@ -83,9 +83,11 @@ def instance_insert_statement( template, table_name=table_name, count=count, - insert_columns="'src_4', 4" - if not insert_unique_blocks - else "'src_' || toString(number), number", + insert_columns=( + "'src_4', 4" + if not insert_unique_blocks + else "'src_' || toString(number), number" + ), insert_settings=insert_settings, ) From bba45958c5b3e62d4f99902610a46c4779bc8f40 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 15 Jul 2024 13:07:38 +0000 Subject: [PATCH 110/178] Review fixes. --- .../Resolve/IdentifierResolveScope.cpp | 3 +-- src/Analyzer/Resolve/IdentifierResolveScope.h | 4 +-- src/Analyzer/Resolve/QueryAnalyzer.cpp | 26 +++++++++---------- 3 files changed, 15 insertions(+), 18 deletions(-) diff --git a/src/Analyzer/Resolve/IdentifierResolveScope.cpp b/src/Analyzer/Resolve/IdentifierResolveScope.cpp index 1600efacf4a..eb3e2179440 100644 --- a/src/Analyzer/Resolve/IdentifierResolveScope.cpp +++ b/src/Analyzer/Resolve/IdentifierResolveScope.cpp @@ -12,10 +12,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_, bool is_query_) +IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_) : scope_node(std::move(scope_node_)) , parent_scope(parent_scope_) - , is_query(is_query_) { if (parent_scope) { diff --git a/src/Analyzer/Resolve/IdentifierResolveScope.h b/src/Analyzer/Resolve/IdentifierResolveScope.h index 917e032321d..ab2e27cc14d 100644 --- a/src/Analyzer/Resolve/IdentifierResolveScope.h +++ b/src/Analyzer/Resolve/IdentifierResolveScope.h @@ -128,7 +128,7 @@ constexpr auto PROJECTION_NAME_PLACEHOLDER = "__projection_name_placeholder"; struct IdentifierResolveScope { /// Construct identifier resolve scope using scope node, and parent scope - IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_, bool is_query_); + IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_); QueryTreeNodePtr scope_node; @@ -188,8 +188,6 @@ struct IdentifierResolveScope /// Join retutns NULLs instead of default values bool join_use_nulls = false; - bool is_query; - /// JOINs count size_t joins_count = 0; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 6965036bb0c..a0043c8c04f 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -119,7 +119,7 @@ QueryAnalyzer::~QueryAnalyzer() = default; void QueryAnalyzer::resolve(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, ContextPtr context) { - IdentifierResolveScope scope(node, nullptr /*parent_scope*/, true /*is_query*/); + IdentifierResolveScope scope(node, nullptr /*parent_scope*/); if (!scope.context) scope.context = context; @@ -2169,7 +2169,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I if (apply_transformer->getApplyTransformerType() == ApplyColumnTransformerType::LAMBDA) { auto lambda_expression_to_resolve = expression_node->clone(); - IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope lambda_scope(expression_node, &scope /*parent_scope*/); node_projection_names = resolveLambda(expression_node, lambda_expression_to_resolve, {node}, lambda_scope); auto & lambda_expression_to_resolve_typed = lambda_expression_to_resolve->as(); node = lambda_expression_to_resolve_typed.getExpression(); @@ -3042,7 +3042,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto lambda_expression_clone = lambda_expression_untyped->clone(); - IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); ProjectionNames lambda_projection_names = resolveLambda(lambda_expression_untyped, lambda_expression_clone, function_arguments, lambda_scope); auto & resolved_lambda = lambda_expression_clone->as(); @@ -3297,7 +3297,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi lambda_arguments.push_back(std::make_shared(std::move(column_name_and_type), lambda_to_resolve)); } - IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope lambda_scope(lambda_to_resolve, &scope /*parent_scope*/); lambda_projection_names = resolveLambda(lambda_argument, lambda_to_resolve, lambda_arguments, lambda_scope); if (auto * lambda_list_node_result = lambda_to_resolve_typed.getExpression()->as()) @@ -3518,7 +3518,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( auto node_type = node->getNodeType(); if (!allow_table_expression && (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION)) { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; evaluateScalarSubqueryIfNeeded(node, subquery_scope); @@ -3625,7 +3625,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( else union_node->setIsCTE(false); - IdentifierResolveScope subquery_scope(resolved_identifier_node, &scope /*parent_scope*/, true /*is_query*/); + IdentifierResolveScope subquery_scope(resolved_identifier_node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; /// CTE is being resolved, it's required to forbid to resolve to it again @@ -3758,7 +3758,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( [[fallthrough]]; case QueryTreeNodeType::UNION: { - IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/, true /*is_query*/); + IdentifierResolveScope subquery_scope(node, &scope /*parent_scope*/); subquery_scope.subquery_depth = scope.subquery_depth + 1; std::string projection_name = "_subquery_" + std::to_string(subquery_counter); @@ -3834,7 +3834,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( } /// Check parent scopes until find current query scope. - if (scope_ptr->is_query) + if (scope_ptr->scope_node->getNodeType() == QueryTreeNodeType::QUERY) break; } } @@ -4122,7 +4122,7 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo bool is_column_constant = interpolate_node_typed.getExpression()->getNodeType() == QueryTreeNodeType::CONSTANT; auto & interpolation_to_resolve = interpolate_node_typed.getInterpolateExpression(); - IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope interpolate_scope(interpolation_to_resolve, &scope /*parent_scope*/); auto fake_column_node = std::make_shared(NameAndTypePair(column_to_interpolate_name, interpolate_node_typed.getExpression()->getResultType()), interpolate_node_typed.getExpression()); if (is_column_constant) @@ -4420,7 +4420,7 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table */ alias_column_to_resolve = column_name_to_column_node[alias_column_to_resolve_name]; - IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/, false /*is_query*/); + IdentifierResolveScope alias_column_resolve_scope(alias_column_to_resolve, nullptr /*parent_scope*/); alias_column_resolve_scope.column_name_to_column_node = std::move(column_name_to_column_node); alias_column_resolve_scope.context = scope.context; @@ -5012,7 +5012,7 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS left_subquery->getProjection().getNodes().push_back(projection_node->clone()); left_subquery->getJoinTree() = left_table_expression; - IdentifierResolveScope left_subquery_scope(left_subquery, nullptr /*parent_scope*/, true /*is_query*/); + IdentifierResolveScope left_subquery_scope(left_subquery, nullptr /*parent_scope*/); resolveQuery(left_subquery, left_subquery_scope); const auto & resolved_nodes = left_subquery->getProjection().getNodes(); @@ -5621,7 +5621,7 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier auto & non_recursive_query_mutable_context = non_recursive_query_is_query_node ? non_recursive_query->as().getMutableContext() : non_recursive_query->as().getMutableContext(); - IdentifierResolveScope non_recursive_subquery_scope(non_recursive_query, &scope /*parent_scope*/, true /*is_query*/); + IdentifierResolveScope non_recursive_subquery_scope(non_recursive_query, &scope /*parent_scope*/); non_recursive_subquery_scope.subquery_depth = scope.subquery_depth + 1; if (non_recursive_query_is_query_node) @@ -5652,7 +5652,7 @@ void QueryAnalyzer::resolveUnion(const QueryTreeNodePtr & union_node, Identifier { auto & query_node = queries_nodes[i]; - IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/, true /*is_subquery*/); + IdentifierResolveScope subquery_scope(query_node, &scope /*parent_scope*/); if (recursive_cte_table_node) subquery_scope.expression_argument_name_to_node[union_node_typed.getCTEName()] = recursive_cte_table_node; From 7d42a44944b9876f14749c1e6ebeb309a0750008 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 15 Jul 2024 13:08:31 +0000 Subject: [PATCH 111/178] Review fixes. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index a0043c8c04f..a0be2a53ef6 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -511,7 +511,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden /// exception with this settings enabled(https://github.com/ClickHouse/ClickHouse/issues/52494). subquery_context->setSetting("use_structure_from_insertion_table_in_table_functions", false); - auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_query*/); + auto options = SelectQueryOptions(QueryProcessingStage::Complete, scope.subquery_depth, true /*is_subquery*/); options.only_analyze = only_analyze; auto interpreter = std::make_unique(node->toAST(), subquery_context, subquery_context->getViewSource(), options); From e3d28f92688d63fd7d417553ad77944a380d3d30 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 Jul 2024 15:12:23 +0200 Subject: [PATCH 112/178] Update 02443_detach_attach_partition.sh --- tests/queries/0_stateless/02443_detach_attach_partition.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index d72d771a150..6a47b7d8d61 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -73,7 +73,7 @@ kill -TERM $PID_1 && kill -TERM $PID_2 && kill -TERM $PID_3 && kill -TERM $PID_4 wait $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'threads finished'" -wait_for_queries_to_finish +wait_for_queries_to_finish 60 $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0" $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" From 7fe35a83b6116eac84adbfd71df61e009b348939 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 15 Jul 2024 14:26:30 +0100 Subject: [PATCH 113/178] impl --- programs/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 6b3a0b16624..ce3a4659e0e 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -4,6 +4,9 @@ if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () +set(MAX_LINKER_MEMORY 3500) +include(../cmake/limit_jobs.cmake) + include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake) # The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.), From 981a2c4658cab15cbb6fe17ffabf74bc4562103a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 15 Jul 2024 09:27:22 -0400 Subject: [PATCH 114/178] Update clickhouse-client.xml From e346535641bff8998c126f5fb74f02d63014090a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 15 Jul 2024 16:27:53 +0200 Subject: [PATCH 115/178] fix log in keeper tcp handler --- src/Server/KeeperTCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4849f5827c1..aa79d8814a7 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -623,7 +623,7 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response LOG_INFO( log, "Total time to process a request took too long ({}ms).\nRequest info: {}", - elapsed, + elapsed_ms, request->toString(/*short_format=*/true)); } From dec5931262ab4dfe541abfca3fb1e7b67f3d9a5e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 15 Jul 2024 16:43:41 +0200 Subject: [PATCH 116/178] log session id --- src/Server/KeeperTCPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index aa79d8814a7..b61df45133a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -622,7 +622,8 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response { LOG_INFO( log, - "Total time to process a request took too long ({}ms).\nRequest info: {}", + "Total time to process a request in session {} took too long ({}ms).\nRequest info: {}", + session_id, elapsed_ms, request->toString(/*short_format=*/true)); } From 38027abbecfc7b60594ed1073ce9104133152aa9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 8 Jul 2024 13:06:50 +0000 Subject: [PATCH 117/178] Fix bug in `numbers` when both limit and offset is used but the index cannot be used --- src/Storages/System/StorageSystemNumbers.cpp | 5 +++++ .../03203_system_numbers_limit_and_offset.reference | 3 +++ .../0_stateless/03203_system_numbers_limit_and_offset.sql | 5 +++++ 3 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference create mode 100644 tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index da700a7a4e9..6ddfb5e3213 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -38,6 +38,11 @@ void StorageSystemNumbers::read( size_t max_block_size, size_t num_streams) { + if (limit && *limit < max_block_size) + { + max_block_size = static_cast(*limit); + multithreaded = false; + } query_plan.addStep(std::make_unique( column_names, query_info, storage_snapshot, context, shared_from_this(), max_block_size, num_streams)); } diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference new file mode 100644 index 00000000000..d55fa7c71fc --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference @@ -0,0 +1,3 @@ +9900 +9910 +9920 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql new file mode 100644 index 00000000000..6b7f998c4d4 --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql @@ -0,0 +1,5 @@ +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 990, 3; + From a177bb63077353e15f6780ae775c9519f2795695 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 8 Jul 2024 16:14:20 +0000 Subject: [PATCH 118/178] Really fix the bug --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 35 +++++++------------ src/Storages/System/StorageSystemNumbers.cpp | 5 --- ..._system_numbers_limit_and_offset.reference | 4 +++ .../03203_system_numbers_limit_and_offset.sql | 34 +++++++++++++++++- 4 files changed, 49 insertions(+), 29 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index a3ae035afdd..049fe762640 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -39,15 +39,10 @@ inline void iotaWithStepOptimized(T * begin, size_t count, T first_value, T step class NumbersSource : public ISource { public: - NumbersSource(UInt64 block_size_, UInt64 offset_, std::optional limit_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_) - : ISource(createHeader(column_name)) - , block_size(block_size_) - , next(offset_) - , chunk_step(chunk_step_) - , step(step_) + NumbersSource( + UInt64 block_size_, UInt64 offset_, std::optional end_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_) + : ISource(createHeader(column_name)), block_size(block_size_), next(offset_), chunk_step(chunk_step_), end(end_), step(step_) { - if (limit_.has_value()) - end = limit_.value() + offset_; } String getName() const override { return "Numbers"; } @@ -79,7 +74,6 @@ protected: next += chunk_step; progress(column->size(), column->byteSize()); - return {Columns{std::move(column)}, real_block_size}; } @@ -549,13 +543,21 @@ Pipe ReadFromSystemNumbersStep::makePipe() return pipe; } + const auto end = std::invoke( + [&]() -> std::optional + { + if (numbers_storage.limit.has_value()) + return *(numbers_storage.limit) + numbers_storage.offset; + return {}; + }); + /// Fall back to NumbersSource for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( max_block_size, numbers_storage.offset + i * max_block_size * numbers_storage.step, - numbers_storage.limit, + end, num_streams * max_block_size * numbers_storage.step, numbers_storage.column_name, numbers_storage.step); @@ -571,19 +573,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() pipe.addSource(std::move(source)); } - if (numbers_storage.limit) - { - size_t i = 0; - auto storage_limit = (*numbers_storage.limit - 1) / numbers_storage.step + 1; - /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - pipe.addSimpleTransform( - [&](const Block & header) - { - ++i; - return std::make_shared(header, storage_limit * i / num_streams - storage_limit * (i - 1) / num_streams, 0); - }); - } - return pipe; } diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 6ddfb5e3213..da700a7a4e9 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -38,11 +38,6 @@ void StorageSystemNumbers::read( size_t max_block_size, size_t num_streams) { - if (limit && *limit < max_block_size) - { - max_block_size = static_cast(*limit); - multithreaded = false; - } query_plan.addStep(std::make_unique( column_names, query_info, storage_snapshot, context, shared_from_this(), max_block_size, num_streams)); } diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference index d55fa7c71fc..05deae711c6 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference @@ -1,3 +1,7 @@ +case 1 9900 9910 9920 +case 2 +9990 +18679 31 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql index 6b7f998c4d4..1a1133c8a7f 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql @@ -1,5 +1,37 @@ +SELECT 'case 1'; SELECT number FROM numbers_mt(10000) WHERE (number % 10) = 0 ORDER BY number ASC LIMIT 990, 3; - +SELECT 'case 2'; +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 999, 20 SETTINGS max_block_size = 31; +--- The following query was buggy before, so let's use it as a test case +WITH + toUInt64(-1) AS umax, + toUInt8(ceil(log10(umax))) AS max_digits, + 9 * max_digits AS max_digits_sum, + (x -> ((x > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < x) AND ((x % y) = 0)), range(toUInt64(sqrt(x)) + 1))) = 0))) AS is_prime_slow +SELECT + num, + ds +FROM +( + WITH x -> arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(x)))) AS digits_sum + SELECT + 1 + (number * 2) AS num, + digits_sum(num) AS ds + FROM numbers_mt(10000) + WHERE ds IN ( + WITH x -> ((x > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < x) AND ((x % y) = 0)), range(toUInt64(sqrt(x)) + 1))) = 0)) AS is_prime_slow + SELECT number + FROM numbers(max_digits_sum + 1) + WHERE is_prime_slow(number) + ) +) +WHERE is_prime_slow(num) +ORDER BY num ASC +LIMIT 998, 1 +SETTINGS max_block_size = 64, max_threads=16; From 8b78cf1c6139931132ff84bdf1ae495eef9714be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 8 Jul 2024 16:22:12 +0000 Subject: [PATCH 119/178] Add one more test case --- .../03203_system_numbers_limit_and_offset.reference | 12 ++++++++++++ .../03203_system_numbers_limit_and_offset.sql | 8 ++++++++ 2 files changed, 20 insertions(+) diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference index 05deae711c6..e95864a6fc9 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference @@ -5,3 +5,15 @@ case 1 case 2 9990 18679 31 +0 +10 +20 +30 +40 +50 +60 +70 +80 +90 +100 +110 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql index 1a1133c8a7f..dc9e06b806f 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql @@ -3,11 +3,13 @@ SELECT number FROM numbers_mt(10000) WHERE (number % 10) = 0 ORDER BY number ASC LIMIT 990, 3; + SELECT 'case 2'; SELECT number FROM numbers_mt(10000) WHERE (number % 10) = 0 ORDER BY number ASC LIMIT 999, 20 SETTINGS max_block_size = 31; + --- The following query was buggy before, so let's use it as a test case WITH toUInt64(-1) AS umax, @@ -35,3 +37,9 @@ WHERE is_prime_slow(num) ORDER BY num ASC LIMIT 998, 1 SETTINGS max_block_size = 64, max_threads=16; + +SELECT number +FROM numbers_mt(120) +WHERE (number % 10) = 0 +ORDER BY number ASC +SETTINGS max_block_size = 31, max_threads = 11 From 2442473b253c4d8cf8f4474065b196bd4be4782c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Jul 2024 11:12:28 +0000 Subject: [PATCH 120/178] Fix `generate_series` --- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 50 +++++++++++++++---- .../02970_generate_series.reference | 1 + .../0_stateless/02970_generate_series.sql | 1 + 3 files changed, 41 insertions(+), 11 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 049fe762640..67c228d43f4 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -36,12 +36,32 @@ inline void iotaWithStepOptimized(T * begin, size_t count, T first_value, T step iotaWithStep(begin, count, first_value, step); } +/// The range is defined as [start, end) +UInt64 itemCountInRange(UInt64 start, UInt64 end, UInt64 step) +{ + const auto range_count = end - start; + if (step == 1) + return range_count; + + return (range_count - 1) / step + 1; +} + class NumbersSource : public ISource { public: NumbersSource( - UInt64 block_size_, UInt64 offset_, std::optional end_, UInt64 chunk_step_, const std::string & column_name, UInt64 step_) - : ISource(createHeader(column_name)), block_size(block_size_), next(offset_), chunk_step(chunk_step_), end(end_), step(step_) + UInt64 block_size_, + UInt64 offset_, + std::optional end_, + const std::string & column_name, + UInt64 step_in_chunk_, + UInt64 step_between_chunks_) + : ISource(createHeader(column_name)) + , block_size(block_size_) + , next(offset_) + , end(end_) + , step_in_chunk(step_in_chunk_) + , step_between_chunks(step_between_chunks_) { } String getName() const override { return "Numbers"; } @@ -59,7 +79,10 @@ protected: { if (end.value() <= next) return {}; - real_block_size = std::min(block_size, end.value() - next); + + auto max_items_to_generate = itemCountInRange(next, *end, step_in_chunk); + + real_block_size = std::min(block_size, max_items_to_generate); } auto column = ColumnUInt64::create(real_block_size); ColumnUInt64::Container & vec = column->getData(); @@ -69,9 +92,9 @@ protected: UInt64 * current_end = &vec[real_block_size]; - iotaWithStepOptimized(pos, static_cast(current_end - pos), curr, step); + iotaWithStepOptimized(pos, static_cast(current_end - pos), curr, step_in_chunk); - next += chunk_step; + next += step_between_chunks; progress(column->size(), column->byteSize()); return {Columns{std::move(column)}, real_block_size}; @@ -80,9 +103,9 @@ protected: private: UInt64 block_size; UInt64 next; - UInt64 chunk_step; std::optional end; /// not included - UInt64 step; + UInt64 step_in_chunk; + UInt64 step_between_chunks; }; struct RangeWithStep @@ -552,19 +575,24 @@ Pipe ReadFromSystemNumbersStep::makePipe() }); /// Fall back to NumbersSource + /// Range in a single block + const auto block_range = max_block_size * numbers_storage.step; + /// Step between chunks in a single source. + /// It is bigger than block_range in case of multiple threads, because we have to account for other sources as well. + const auto step_between_chunks = num_streams * block_range; for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( max_block_size, - numbers_storage.offset + i * max_block_size * numbers_storage.step, + numbers_storage.offset + i * block_range, end, - num_streams * max_block_size * numbers_storage.step, numbers_storage.column_name, - numbers_storage.step); + numbers_storage.step, + step_between_chunks); if (numbers_storage.limit && i == 0) { - auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1; + auto rows_appr = itemCountInRange(numbers_storage.offset, *numbers_storage.limit, numbers_storage.step); if (limit > 0 && limit < rows_appr) rows_appr = query_info_limit; source->addTotalRowsApprox(rows_appr); diff --git a/tests/queries/0_stateless/02970_generate_series.reference b/tests/queries/0_stateless/02970_generate_series.reference index 6e6f3c81587..f8a8e1891b7 100644 --- a/tests/queries/0_stateless/02970_generate_series.reference +++ b/tests/queries/0_stateless/02970_generate_series.reference @@ -5,6 +5,7 @@ 501 50 17928 +17928 0 10 13 diff --git a/tests/queries/0_stateless/02970_generate_series.sql b/tests/queries/0_stateless/02970_generate_series.sql index 0844714b3a6..edae884a561 100644 --- a/tests/queries/0_stateless/02970_generate_series.sql +++ b/tests/queries/0_stateless/02970_generate_series.sql @@ -5,6 +5,7 @@ SELECT count() FROM generate_series(7, 77, 10); SELECT count() FROM generate_series(0, 1000, 2); SELECT count() FROM generate_series(0, 999, 20); SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1; +SELECT sum(generate_series) FROM generate_series(4, 1008, 4) WHERE generate_series % 7 = 1 SETTINGS max_block_size = 71; SELECT * FROM generate_series(5, 4); SELECT * FROM generate_series(0, 0); From a92eb1a4c38be7868fd0cb80261ae882c8fbb70e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Jul 2024 11:20:05 +0000 Subject: [PATCH 121/178] Avoid creating unnecessary sources --- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 67c228d43f4..da8a6933d6f 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -582,6 +582,10 @@ Pipe ReadFromSystemNumbersStep::makePipe() const auto step_between_chunks = num_streams * block_range; for (size_t i = 0; i < num_streams; ++i) { + const auto source_start = numbers_storage.offset + i * block_range; + if (numbers_storage.limit.has_value() && *numbers_storage.limit < source_start) + break; + auto source = std::make_shared( max_block_size, numbers_storage.offset + i * block_range, From 17352b6ed1cdcad359d2f17a96040f7fdaba9422 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 10 Jul 2024 12:06:11 +0000 Subject: [PATCH 122/178] Fix the case when offset is bigger than limit --- src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp | 8 +++++--- .../03203_system_numbers_limit_and_offset.reference | 1 + .../0_stateless/03203_system_numbers_limit_and_offset.sql | 8 +++++++- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index da8a6933d6f..b070bbe739b 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -582,13 +582,15 @@ Pipe ReadFromSystemNumbersStep::makePipe() const auto step_between_chunks = num_streams * block_range; for (size_t i = 0; i < num_streams; ++i) { - const auto source_start = numbers_storage.offset + i * block_range; - if (numbers_storage.limit.has_value() && *numbers_storage.limit < source_start) + const auto source_offset = i * block_range; + if (numbers_storage.limit.has_value() && *numbers_storage.limit < source_offset) break; + const auto source_start = numbers_storage.offset + source_offset; + auto source = std::make_shared( max_block_size, - numbers_storage.offset + i * block_range, + source_start, end, numbers_storage.column_name, numbers_storage.step, diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference index e95864a6fc9..960459f42ac 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference @@ -17,3 +17,4 @@ case 2 90 100 110 +4250 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql index dc9e06b806f..e258b0bd2d2 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql @@ -42,4 +42,10 @@ SELECT number FROM numbers_mt(120) WHERE (number % 10) = 0 ORDER BY number ASC -SETTINGS max_block_size = 31, max_threads = 11 +SETTINGS max_block_size = 31, max_threads = 11; + +SELECT number +FROM numbers_mt(4242, 9) +WHERE (number % 10) = 0 +ORDER BY number ASC +SETTINGS max_block_size = 31, max_threads = 11; From 6be9d7b2544ccb5642999b53409c0d9326163a62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 11 Jul 2024 13:40:44 +0200 Subject: [PATCH 123/178] Make the tests works with old analyzer --- .../0_stateless/03203_system_numbers_limit_and_offset.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql index e258b0bd2d2..b72ab4a98fd 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql @@ -21,10 +21,10 @@ SELECT ds FROM ( - WITH x -> arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(x)))) AS digits_sum + WITH arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(num)))) AS digits_sum SELECT 1 + (number * 2) AS num, - digits_sum(num) AS ds + digits_sum AS ds FROM numbers_mt(10000) WHERE ds IN ( WITH x -> ((x > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < x) AND ((x % y) = 0)), range(toUInt64(sqrt(x)) + 1))) = 0)) AS is_prime_slow From 35eabcc9d1cf3a5339c20ea34d006edfc858ae68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 13 Jul 2024 16:01:15 +0000 Subject: [PATCH 124/178] Split test cases --- ...ystem_numbers_limit_and_offset_complex.reference} | 6 ------ ...3203_system_numbers_limit_and_offset_complex.sql} | 12 ------------ ..._system_numbers_limit_and_offset_simple.reference | 6 ++++++ .../03203_system_numbers_limit_and_offset_simple.sql | 11 +++++++++++ 4 files changed, 17 insertions(+), 18 deletions(-) rename tests/queries/0_stateless/{03203_system_numbers_limit_and_offset.reference => 03203_system_numbers_limit_and_offset_complex.reference} (60%) rename tests/queries/0_stateless/{03203_system_numbers_limit_and_offset.sql => 03203_system_numbers_limit_and_offset_complex.sql} (82%) create mode 100644 tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference create mode 100644 tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference similarity index 60% rename from tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference rename to tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference index 960459f42ac..d38b21d2d05 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.reference +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.reference @@ -1,9 +1,3 @@ -case 1 -9900 -9910 -9920 -case 2 -9990 18679 31 0 10 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql similarity index 82% rename from tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql rename to tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql index b72ab4a98fd..8e8620361fa 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql @@ -1,15 +1,3 @@ -SELECT 'case 1'; -SELECT number FROM numbers_mt(10000) -WHERE (number % 10) = 0 -ORDER BY number ASC -LIMIT 990, 3; - -SELECT 'case 2'; -SELECT number FROM numbers_mt(10000) -WHERE (number % 10) = 0 -ORDER BY number ASC -LIMIT 999, 20 SETTINGS max_block_size = 31; - --- The following query was buggy before, so let's use it as a test case WITH toUInt64(-1) AS umax, diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference new file mode 100644 index 00000000000..6aad3ee0b4b --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.reference @@ -0,0 +1,6 @@ +case 1 +9900 +9910 +9920 +case 2 +9990 diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql new file mode 100644 index 00000000000..df01ddf4312 --- /dev/null +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_simple.sql @@ -0,0 +1,11 @@ +SELECT 'case 1'; +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 990, 3; + +SELECT 'case 2'; +SELECT number FROM numbers_mt(10000) +WHERE (number % 10) = 0 +ORDER BY number ASC +LIMIT 999, 20 SETTINGS max_block_size = 31; From 8e469af2a41b444bbeb58b63fa0ece7d883553c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Sat, 13 Jul 2024 20:36:01 +0000 Subject: [PATCH 125/178] Make query really work with old analyzer --- ...ystem_numbers_limit_and_offset_complex.sql | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql index 8e8620361fa..756e08da27d 100644 --- a/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql +++ b/tests/queries/0_stateless/03203_system_numbers_limit_and_offset_complex.sql @@ -1,27 +1,26 @@ --- The following query was buggy before, so let's use it as a test case WITH - toUInt64(-1) AS umax, - toUInt8(ceil(log10(umax))) AS max_digits, - 9 * max_digits AS max_digits_sum, - (x -> ((x > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < x) AND ((x % y) = 0)), range(toUInt64(sqrt(x)) + 1))) = 0))) AS is_prime_slow + (num > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < num) AND ((num % y) = 0)), range(toUInt64(sqrt(num)) + 1))) = 0) AS is_prime_slow SELECT num, - ds + ds, FROM ( - WITH arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(num)))) AS digits_sum + WITH + arraySum(arrayMap(y -> toUInt8(y), splitByString('', toString(num)))) AS digits_sum SELECT 1 + (number * 2) AS num, digits_sum AS ds FROM numbers_mt(10000) WHERE ds IN ( - WITH x -> ((x > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < x) AND ((x % y) = 0)), range(toUInt64(sqrt(x)) + 1))) = 0)) AS is_prime_slow + WITH + (number > 1) AND (arraySum(arrayMap(y -> ((y > 1) AND (y < number) AND ((number % y) = 0)), range(toUInt64(sqrt(number)) + 1))) = 0) AS is_prime_slow SELECT number - FROM numbers(max_digits_sum + 1) - WHERE is_prime_slow(number) + FROM numbers(180 + 1) + WHERE is_prime_slow ) ) -WHERE is_prime_slow(num) +WHERE is_prime_slow ORDER BY num ASC LIMIT 998, 1 SETTINGS max_block_size = 64, max_threads=16; From cfb2183d118b783d142a721c89c04b1835dfa3ed Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 15 Jul 2024 17:25:43 +0200 Subject: [PATCH 126/178] Update 01396_inactive_replica_cleanup_nodes_zookeeper.sh --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index b81bb75891d..bff85b3e29f 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT --max_block_size 1 --min_insert_block_size_rows 1 --min_inser for _ in {1..60}; do $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" - [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 100)) ]] && break; + [[ $($CLICKHOUSE_CLIENT --query "SELECT sum(toUInt32(extract(message, 'Removed (\d+) old log entries'))) FROM system.text_log WHERE event_date >= yesterday() AND logger_name LIKE '%' || '$CLICKHOUSE_DATABASE' || '%r1%(ReplicatedMergeTreeCleanupThread)%' AND message LIKE '%Removed % old log entries%'") -gt $((SCALE - 10)) ]] && break; sleep 1 done From e06f1ed108f12b24fff716dc380bb5adce27a921 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 15 Jul 2024 17:59:00 +0200 Subject: [PATCH 127/178] Update the MySQL interface doc with compatibility notes --- docs/en/interfaces/mysql.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index 42820505406..9ad13ae4f05 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -6,7 +6,18 @@ sidebar_label: MySQL Interface # MySQL Interface -ClickHouse supports the MySQL wire protocol. This allow tools that are MySQL-compatible to interact with ClickHouse seamlessly (e.g. [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md)). +ClickHouse supports the MySQL wire protocol. This allows certain clients that do not have native ClickHouse connectors leverage the MySQL protocol instead, and it has been validated with the following BI tools: + +- [Looker Studio](../integrations/data-visualization/looker-studio-and-clickhouse.md) +- [Tableau Online](../integrations/tableau-online) +- [QuickSight](../integrations/quicksight) + +If you are trying other untested clients or integrations, keep in mind that there could be the following limitations: + +- SSL implementation might not be fully compatible; there could be potential [TLS SNI](https://www.cloudflare.com/learning/ssl/what-is-sni/) issues. +- A particular tool might require dialect features (e.g., MySQL-specific functions or settings) that are not implemented yet. + +If there is a native driver available (e.g., [DBeaver](../integrations/dbeaver)), it is always preferred to use it instead of the MySQL interface. Additionally, while most of the MySQL language clients should work fine, MySQL interface is not guaranteed to be a drop-in replacement for a codebase with existing MySQL queries. ## Enabling the MySQL Interface On ClickHouse Cloud From 293b5c81d088223f9428e4b9e73f036bc8165d7d Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 15 Jul 2024 18:10:46 +0200 Subject: [PATCH 128/178] Add an entry about MySQL interface issues reporting --- docs/en/interfaces/mysql.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index 9ad13ae4f05..30a8d9569a9 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -19,6 +19,8 @@ If you are trying other untested clients or integrations, keep in mind that ther If there is a native driver available (e.g., [DBeaver](../integrations/dbeaver)), it is always preferred to use it instead of the MySQL interface. Additionally, while most of the MySQL language clients should work fine, MySQL interface is not guaranteed to be a drop-in replacement for a codebase with existing MySQL queries. +If your use case involves a particular tool that does not have a native ClickHouse driver, and you would like to use it via the MySQL interface and you found certain incompatibilities - please [create an issue](https://github.com/ClickHouse/ClickHouse/issues) in the ClickHouse repository. + ## Enabling the MySQL Interface On ClickHouse Cloud 1. After creating your ClickHouse Cloud Service, on the credentials screen, select the MySQL tab From 10c553b06450dfb34d63b8b37b8fbdd19e98ad5f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 15 Jul 2024 18:36:19 +0200 Subject: [PATCH 129/178] empty commit From 50f068efe930e264de938da46040a1e056200032 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 17:00:53 +0000 Subject: [PATCH 130/178] add dedicated runner to libfuzzer, update docker --- docker/test/libfuzzer/run_libfuzzer.py | 70 +--------------------- tests/ci/build_download_helper.py | 2 +- tests/fuzz/build.sh | 3 + utils/libfuzzer/runner.py | 80 ++++++++++++++++++++++++++ 4 files changed, 86 insertions(+), 69 deletions(-) create mode 100644 utils/libfuzzer/runner.py diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py index fa67805dfa5..25d5ede028d 100755 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -1,77 +1,11 @@ #!/usr/bin/env python3 -import configparser -import logging -import os -from pathlib import Path -import subprocess - -DEBUGGER = os.getenv("DEBUGGER", "") -FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") - - -def run_fuzzer(fuzzer: str): - logging.info(f"Running fuzzer {fuzzer}...") - - corpus_dir = f"{fuzzer}.in" - with Path(corpus_dir) as path: - if not path.exists() or not path.is_dir(): - corpus_dir = "" - - options_file = f"{fuzzer}.options" - custom_libfuzzer_options = "" - - with Path(options_file) as path: - if path.exists() and path.is_file(): - parser = configparser.ConfigParser() - parser.read(path) - - if parser.has_section("asan"): - os.environ["ASAN_OPTIONS"] = ( - f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}" - ) - - if parser.has_section("msan"): - os.environ["MSAN_OPTIONS"] = ( - f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}" - ) - - if parser.has_section("ubsan"): - os.environ["UBSAN_OPTIONS"] = ( - f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}" - ) - - if parser.has_section("libfuzzer"): - custom_libfuzzer_options = " ".join( - "-%s=%s" % (key, value) - for key, value in parser["libfuzzer"].items() - ) - - cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" - if custom_libfuzzer_options: - cmd_line += f" {custom_libfuzzer_options}" - - if not "-dict=" in cmd_line and Path(f"{fuzzer}.dict").exists(): - cmd_line += f" -dict={fuzzer}.dict" - - cmd_line += " < /dev/null" - - logging.info(f"...will execute: {cmd_line}") - subprocess.check_call(cmd_line, shell=True) +import runner def main(): - logging.basicConfig(level=logging.INFO) - - subprocess.check_call("ls -al", shell=True) - - with Path() as current: - for fuzzer in current.iterdir(): - if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): - run_fuzzer(fuzzer) - + runner.run() exit(0) - if __name__ == "__main__": main() diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 8482abb26e0..0b7dd6dd0c9 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -275,5 +275,5 @@ def download_fuzzers( check_name, reports_path, result_path, - lambda x: x.endswith(("_fuzzer", ".dict", ".options", "_seed_corpus.zip")), + lambda x: x.endswith(("_fuzzer", ".dict", ".options", "_seed_corpus.zip", "runner.py")), ) diff --git a/tests/fuzz/build.sh b/tests/fuzz/build.sh index 12f41f6e079..cf871d2de58 100755 --- a/tests/fuzz/build.sh +++ b/tests/fuzz/build.sh @@ -1,5 +1,8 @@ #!/bin/bash -eu +# copy runner +cp $SRC/utils/libfuzzer/runner.py $OUT/ + # copy fuzzer options and dictionaries cp $SRC/tests/fuzz/*.dict $OUT/ cp $SRC/tests/fuzz/*.options $OUT/ diff --git a/utils/libfuzzer/runner.py b/utils/libfuzzer/runner.py new file mode 100644 index 00000000000..435eb49383b --- /dev/null +++ b/utils/libfuzzer/runner.py @@ -0,0 +1,80 @@ +#!/usr/bin/env python3 + +import configparser +import logging +import os +from pathlib import Path +import subprocess + +DEBUGGER = os.getenv("DEBUGGER", "") +FUZZER_ARGS = os.getenv("FUZZER_ARGS", "") + + +def run_fuzzer(fuzzer: str): + logging.info(f"Running fuzzer {fuzzer}...") + + corpus_dir = f"{fuzzer}.in" + with Path(corpus_dir) as path: + if not path.exists() or not path.is_dir(): + corpus_dir = "" + + options_file = f"{fuzzer}.options" + custom_libfuzzer_options = "" + fuzzer_arguments = "" + + with Path(options_file) as path: + if path.exists() and path.is_file(): + parser = configparser.ConfigParser() + parser.read(path) + + if parser.has_section("asan"): + os.environ["ASAN_OPTIONS"] = ( + f"{os.environ['ASAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['asan'].items())}" + ) + + if parser.has_section("msan"): + os.environ["MSAN_OPTIONS"] = ( + f"{os.environ['MSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['msan'].items())}" + ) + + if parser.has_section("ubsan"): + os.environ["UBSAN_OPTIONS"] = ( + f"{os.environ['UBSAN_OPTIONS']}:{':'.join('%s=%s' % (key, value) for key, value in parser['ubsan'].items())}" + ) + + if parser.has_section("libfuzzer"): + custom_libfuzzer_options = " ".join( + "-%s=%s" % (key, value) + for key, value in parser["libfuzzer"].items() + ) + + if parser.has_section("fuzzer_arguments"): + fuzzer_arguments = " ".join( + ("%s" % key) if value == "" else ("%s=%s" % (key, value)) + for key, value in parser["fuzzer_arguments"].items() + ) + + cmd_line = f"{DEBUGGER} ./{fuzzer} {FUZZER_ARGS} {corpus_dir}" + if custom_libfuzzer_options: + cmd_line += f" {custom_libfuzzer_options}" + if fuzzer_arguments: + cmd_line += f" {fuzzer_arguments}" + + if not "-dict=" in cmd_line and Path(f"{fuzzer}.dict").exists(): + cmd_line += f" -dict={fuzzer}.dict" + + cmd_line += " < /dev/null" + + logging.info(f"...will execute: {cmd_line}") + subprocess.check_call(cmd_line, shell=True) + + +def run(): + logging.basicConfig(level=logging.INFO) + + subprocess.check_call("ls -al", shell=True) + + with Path() as current: + for fuzzer in current.iterdir(): + if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): + run_fuzzer(fuzzer) From 874b4d1af09496be6cc6601186379e8ad925b05a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 15 Jul 2024 19:15:45 +0200 Subject: [PATCH 131/178] fix tidy build --- src/Client/ConnectionPool.h | 2 +- src/Interpreters/Context.cpp | 1 - utils/keeper-bench/Runner.cpp | 1 + 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index d83ecdd75f9..0fcb3c4e7e1 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -103,7 +103,7 @@ public: Entry get(const ConnectionTimeouts & timeouts, /// NOLINT const Settings & settings, - bool force_connected = true) override; + bool force_connected) override; std::string getDescription() const { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..2602afd8b78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 587e015b340..cd7f1437cf7 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include From 97c9739abcb6b6e5da113ac12a35078fe694aff6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 15 Jul 2024 17:21:50 +0000 Subject: [PATCH 132/178] Automatic style fix --- docker/test/libfuzzer/run_libfuzzer.py | 1 + tests/ci/build_download_helper.py | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py index 25d5ede028d..06c816c8943 100755 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ b/docker/test/libfuzzer/run_libfuzzer.py @@ -7,5 +7,6 @@ def main(): runner.run() exit(0) + if __name__ == "__main__": main() diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 0b7dd6dd0c9..c3e516f836d 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -275,5 +275,7 @@ def download_fuzzers( check_name, reports_path, result_path, - lambda x: x.endswith(("_fuzzer", ".dict", ".options", "_seed_corpus.zip", "runner.py")), + lambda x: x.endswith( + ("_fuzzer", ".dict", ".options", "_seed_corpus.zip", "runner.py") + ), ) From 5fc4fada610b9d7a62528eaecb600bd5d16d48ae Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 15 Jul 2024 18:18:15 +0200 Subject: [PATCH 133/178] add info about previous release --- tests/ci/ci_utils.py | 5 +++-- tests/ci/create_release.py | 21 +++++++++++++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 44bd37fe260..25875e55df6 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -49,14 +49,15 @@ class GHActions: class Shell: @classmethod def run_strict(cls, command): - subprocess.run( - command + " 2>&1", + res = subprocess.run( + command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE, text=True, check=True, ) + return res.stdout.strip() @classmethod def run(cls, command): diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index f377e12d11c..e5723e00e2a 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -14,6 +14,7 @@ from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper from autoscale_runners_lambda.lambda_shared.pr import Labels +from ci_utils import Shell from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -65,6 +66,8 @@ class ReleaseInfo: commit_sha: str # lts or stable codename: str + previous_release_tag: str + previous_release_sha: str @staticmethod def from_file(file_path: str) -> "ReleaseInfo": @@ -79,6 +82,8 @@ class ReleaseInfo: version = None release_branch = None release_tag = None + previous_release_tag = None + previous_release_sha = None codename = None assert release_type in ("patch", "new") if release_type == "new": @@ -101,6 +106,11 @@ class ReleaseInfo: codename = ( VersionType.STABLE ) # dummy value (artifactory won't be updated for new release) + previous_release_tag = expected_prev_tag + previous_release_sha = Shell.run_strict( + f"git rev-parse {previous_release_tag}" + ) + assert previous_release_sha if release_type == "patch": with checkout(commit_ref): _, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}") @@ -118,6 +128,7 @@ class ReleaseInfo: ) if version.patch == 1: expected_version = copy(version) + previous_release_tag = f"v{version.major}.{version.minor}.1.1-new" expected_version.bump() expected_tag_prefix = ( f"v{expected_version.major}.{expected_version.minor}." @@ -128,6 +139,7 @@ class ReleaseInfo: f"v{version.major}.{version.minor}.{version.patch-1}." ) expected_tag_suffix = f"-{version.get_stable_release_type()}" + previous_release_tag = git.latest_tag if git.latest_tag.startswith( expected_tag_prefix ) and git.latest_tag.endswith(expected_tag_suffix): @@ -137,8 +149,15 @@ class ReleaseInfo: False ), f"BUG: Unexpected latest tag [{git.latest_tag}] expected [{expected_tag_prefix}*{expected_tag_suffix}]" + previous_release_sha = Shell.run_strict( + f"git rev-parse {previous_release_tag}" + ) + assert previous_release_sha + assert ( release_branch + and previous_release_tag + and previous_release_sha and commit_sha and release_tag and version @@ -150,6 +169,8 @@ class ReleaseInfo: release_tag=release_tag, version=version.string, codename=codename, + previous_release_tag=previous_release_tag, + previous_release_sha=previous_release_sha, ) with open(outfile, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(res), indent=2), file=f) From 5efbb42a2a0efbb5a32267fec60ab73eddaea4cf Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 15 Jul 2024 19:31:41 +0200 Subject: [PATCH 134/178] Add QuickSight to spell check ignore --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a2a2fcd967a..987c4e8cac1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -760,6 +760,7 @@ QueryCacheMisses QueryPreempted QueryThread QuickAssist +QuickSight QuoteMeta RBAC RClickHouse From de2e789c06e255b7a31a6caf530043fad8b248a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 15 Jul 2024 19:44:30 +0200 Subject: [PATCH 135/178] Change env variable to run tests with SMT --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 90fb9611151..8df1dd2a8eb 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3386,13 +3386,13 @@ def parse_args(): parser.add_argument( "--replace-replicated-with-shared", action="store_true", - default=os.environ.get("USE_META_IN_KEEPER_FOR_MERGE_TREE", False), + default=os.environ.get("REPLACE_RMT_WITH_SMT", False), help="Replace ReplicatedMergeTree engine with SharedMergeTree", ) parser.add_argument( "--replace-non-replicated-with-shared", action="store_true", - default=False, + default=os.environ.get("REPLACE_MT_WITH_SMT", False), help="Replace ordinary MergeTree engine with SharedMergeTree", ) From 5361ade8e709f3ebe8916aa80669f4bec8b5c726 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 19:10:07 +0000 Subject: [PATCH 136/178] optimize --- docker/test/libfuzzer/Dockerfile | 2 +- docker/test/libfuzzer/run_libfuzzer.py | 12 ------------ tests/ci/build_download_helper.py | 4 +--- tests/fuzz/build.sh | 3 --- utils/libfuzzer/runner.py | 6 +++++- 5 files changed, 7 insertions(+), 20 deletions(-) delete mode 100755 docker/test/libfuzzer/run_libfuzzer.py diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index e6eb2ae336e..9ded2ca4e3a 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -39,7 +39,7 @@ ENV FUZZER_ARGS="-max_total_time=60" SHELL ["/bin/bash", "-c"] CMD set -o pipefail \ - && timeout -s 9 1h /run_libfuzzer.py 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log + && timeout -s 9 1h ./utils/libfuzzer/runner.py 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer diff --git a/docker/test/libfuzzer/run_libfuzzer.py b/docker/test/libfuzzer/run_libfuzzer.py deleted file mode 100755 index 06c816c8943..00000000000 --- a/docker/test/libfuzzer/run_libfuzzer.py +++ /dev/null @@ -1,12 +0,0 @@ -#!/usr/bin/env python3 - -import runner - - -def main(): - runner.run() - exit(0) - - -if __name__ == "__main__": - main() diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index c3e516f836d..8482abb26e0 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -275,7 +275,5 @@ def download_fuzzers( check_name, reports_path, result_path, - lambda x: x.endswith( - ("_fuzzer", ".dict", ".options", "_seed_corpus.zip", "runner.py") - ), + lambda x: x.endswith(("_fuzzer", ".dict", ".options", "_seed_corpus.zip")), ) diff --git a/tests/fuzz/build.sh b/tests/fuzz/build.sh index cf871d2de58..12f41f6e079 100755 --- a/tests/fuzz/build.sh +++ b/tests/fuzz/build.sh @@ -1,8 +1,5 @@ #!/bin/bash -eu -# copy runner -cp $SRC/utils/libfuzzer/runner.py $OUT/ - # copy fuzzer options and dictionaries cp $SRC/tests/fuzz/*.dict $OUT/ cp $SRC/tests/fuzz/*.options $OUT/ diff --git a/utils/libfuzzer/runner.py b/utils/libfuzzer/runner.py index 435eb49383b..bbe648dbbc2 100644 --- a/utils/libfuzzer/runner.py +++ b/utils/libfuzzer/runner.py @@ -69,7 +69,7 @@ def run_fuzzer(fuzzer: str): subprocess.check_call(cmd_line, shell=True) -def run(): +def main(): logging.basicConfig(level=logging.INFO) subprocess.check_call("ls -al", shell=True) @@ -78,3 +78,7 @@ def run(): for fuzzer in current.iterdir(): if (current / fuzzer).is_file() and os.access(current / fuzzer, os.X_OK): run_fuzzer(fuzzer) + + +if __name__ == "__main__": + main() From 0176e0b2ae534fee35a6e2bcab8a74a0783e8137 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 Jul 2024 19:27:54 +0000 Subject: [PATCH 137/178] fix --- docker/test/libfuzzer/Dockerfile | 4 ---- tests/ci/libfuzzer_test_check.py | 3 ++- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/docker/test/libfuzzer/Dockerfile b/docker/test/libfuzzer/Dockerfile index 9ded2ca4e3a..3ffae0cd921 100644 --- a/docker/test/libfuzzer/Dockerfile +++ b/docker/test/libfuzzer/Dockerfile @@ -33,13 +33,9 @@ RUN apt-get update \ COPY requirements.txt / RUN pip3 install --no-cache-dir -r /requirements.txt -COPY * / - ENV FUZZER_ARGS="-max_total_time=60" SHELL ["/bin/bash", "-c"] -CMD set -o pipefail \ - && timeout -s 9 1h ./utils/libfuzzer/runner.py 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> clickhouse/libfuzzer diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 1f5936c3fec..d9e33229932 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -74,7 +74,8 @@ def get_run_command( f"--volume={repo_path}/tests:/usr/share/clickhouse-test " f"--volume={result_path}:/test_output " "--security-opt seccomp=unconfined " # required to issue io_uring sys-calls - f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" + f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image} " + "python3 ./utils/runner.py" ) From c17d5926c6f9f71dff07cb828a4e3bb53b3e6b81 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Jul 2024 21:29:52 +0200 Subject: [PATCH 138/178] Fix typo in new_delete.cpp --- src/Common/memory.h | 4 ++-- src/Common/new_delete.cpp | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/memory.h b/src/Common/memory.h index caa0418fa56..dbef069b408 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -80,7 +80,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) throw std::bad_alloc{}; } -inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept +inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept { #if USE_GWP_ASAN if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) @@ -99,7 +99,7 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept return malloc(size); } -inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept +inline ALWAYS_INLINE void * newNoExcept(std::size_t size, std::align_val_t align) noexcept { #if USE_GWP_ASAN if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index e8151fbe201..80e05fc4ea0 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -87,7 +87,7 @@ void * operator new(std::size_t size, const std::nothrow_t &) noexcept { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace); - void * ptr = Memory::newNoExept(size); + void * ptr = Memory::newNoExcept(size); trace.onAlloc(ptr, actual_size); return ptr; } @@ -96,7 +96,7 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace); - void * ptr = Memory::newNoExept(size); + void * ptr = Memory::newNoExcept(size); trace.onAlloc(ptr, actual_size); return ptr; } @@ -105,7 +105,7 @@ void * operator new(std::size_t size, std::align_val_t align, const std::nothrow { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace, align); - void * ptr = Memory::newNoExept(size, align); + void * ptr = Memory::newNoExcept(size, align); trace.onAlloc(ptr, actual_size); return ptr; } @@ -114,7 +114,7 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace, align); - void * ptr = Memory::newNoExept(size, align); + void * ptr = Memory::newNoExcept(size, align); trace.onAlloc(ptr, actual_size); return ptr; } From bc5e2afc86317d832e6e25fb7d7719e44bdc1876 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 16 Jul 2024 00:52:17 +0200 Subject: [PATCH 139/178] Update StorageDictionary.cpp From 04735f42297c6526b837e537c74ccbf0d38c0581 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 02:27:58 +0200 Subject: [PATCH 140/178] Fix clang tidy --- src/Interpreters/Context.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..2602afd8b78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include From 68aedc219187b0ffe89a3382c7f8136085e72ecf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 03:58:55 +0200 Subject: [PATCH 141/178] Fix something in Fast Test --- docker/test/fasttest/run.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 0d975d64010..58c436b469b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -9,7 +9,11 @@ trap 'kill $(jobs -pr) ||:' EXIT stage=${stage:-} # Compiler version, normally set by Dockerfile -export LLVM_VERSION=${LLVM_VERSION:-17} +export LLVM_VERSION=${LLVM_VERSION:-18} + +# For some reason it is needed for tests with 'expect' but I don't know, why +groupadd --system --gid 1000 clickhouse +useradd --system --gid 1000 --uid 1000 clickhouse # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about From 26f8b81081dd75941bb72b00642391529b9d5bda Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 10:54:00 +0800 Subject: [PATCH 142/178] update trusted contributors --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index e981e28a454..e3aceaab0de 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -45,6 +45,7 @@ TRUSTED_CONTRIBUTORS = { "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc "jrdi", # ClickHouse contributor, TinyBird + "XuJia0210", # ClickHouse, Inc ] } From 919bbf57a7bca538f4a75c0e63af10555b75aa44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:07:39 +0200 Subject: [PATCH 143/178] Fix hashing of empty tuples --- src/Functions/FunctionsHashing.h | 20 ++++++++++++---- .../03205_hashing_empty_tuples.reference | 22 ++++++++++++++++++ .../03205_hashing_empty_tuples.sql | 23 +++++++++++++++++++ 3 files changed, 60 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03205_hashing_empty_tuples.reference create mode 100644 tests/queries/0_stateless/03205_hashing_empty_tuples.sql diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 27717ea3611..95c54ac9528 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1184,7 +1184,7 @@ private: if (icolumn->size() != vec_to.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", - icolumn->getName(), icolumn->size(), vec_to.size(), getName()); + icolumn->getName(), icolumn->size(), vec_to.size(), getName()); if constexpr (Keyed) if (key_cols.size() != vec_to.size() && key_cols.size() != 1) @@ -1223,6 +1223,9 @@ private: else executeGeneric(key_cols, icolumn, vec_to); } + /// Return a fixed random-looking magic number when input is empty. + static constexpr auto filler = 0xe28dbde7fe22e41c; + void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. @@ -1231,6 +1234,11 @@ private: const auto & tuple_columns = tuple->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } @@ -1239,6 +1247,11 @@ private: const auto & tuple_columns = tuple_const->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); @@ -1300,10 +1313,7 @@ public: constexpr size_t first_data_argument = Keyed; if (arguments.size() <= first_data_argument) - { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + vec_to.assign(input_rows_count, static_cast(filler)); KeyColumnsType key_cols{}; if constexpr (Keyed) diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.reference b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference new file mode 100644 index 00000000000..e24b5809aee --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference @@ -0,0 +1,22 @@ +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.sql b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql new file mode 100644 index 00000000000..4a97f30ced3 --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql @@ -0,0 +1,23 @@ +SELECT sipHash64(()); +SELECT sipHash64((), ()); +SELECT sipHash64((), 1); +SELECT sipHash64(1, ()); +SELECT sipHash64(1, (), 1); +SELECT sipHash64((), 1, ()); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((1, 2), ()); +SELECT sipHash64((), (1, 2), ()); +SELECT sipHash64((1, 2), (), (3, 4)); + +SELECT sipHash64(materialize(())); +SELECT sipHash64(materialize(()), materialize(())); +SELECT sipHash64(materialize(()), 1); +SELECT sipHash64(1, materialize(())); +SELECT sipHash64(1, materialize(()), 1); +SELECT sipHash64((), 1, materialize(())); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64((1, 2), materialize(())); +SELECT sipHash64(materialize(()), (1, 2), ()); +SELECT sipHash64((1, 2), materialize(()), (3, 4)); From 2c62a6b5283c68aac604a97591571e05e6877fab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:10:26 +0200 Subject: [PATCH 144/178] Update pr.py --- .../lambda_shared_package/lambda_shared/pr.py | 21 +------------------ 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index e3aceaab0de..d970662d38b 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -3,48 +3,29 @@ import re from typing import Tuple -# Individual trusted contirbutors who are not in any trusted organization. +# Individual trusted contributors who are not in any trusted organization. # Can be changed in runtime: we will append users that we learned to be in # a trusted org, to save GitHub API calls. TRUSTED_CONTRIBUTORS = { e.lower() for e in [ - "achimbab", # Kakao corp "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", # SEMRush "bharatnc", # Many contributions. - "bobrik", # Seasoned contributor, CloudFlare "cwurm", # ClickHouse, Inc "den-crane", # Documentation contributor - "hagen1778", # Roman Khavronenko, seasoned contributor - "hczhcz", - "hexiaoting", # Seasoned contributor "ildus", # adjust, ex-pgpro - "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "kreuzerkrieg", - "nikvas0", "nvartolomei", # Seasoned contributor, CloudFlare - "spongedu", # Seasoned contributor "taiyang-li", "ucasFL", # Amos Bird's friend "vdimir", # ClickHouse, Inc - "YiuRULE", - "zlobober", # Developer of YT - "ilejn", # Arenadata, responsible for Kerberized Kafka "thomoco", # ClickHouse, Inc - "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse, Inc - "myrrc", # Mike Kot, DoubleCloud "thevar1able", # ClickHouse, Inc - "aalexfvk", - "MikhailBurdukov", "tsolodov", # ClickHouse, Inc - "kitaisreal", - "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc - "jrdi", # ClickHouse contributor, TinyBird "XuJia0210", # ClickHouse, Inc ] } From 398440c242a9e10ef900815ebb603c67506ffcbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:21:20 +0200 Subject: [PATCH 145/178] Delete bad test `02805_distributed_queries_timeouts` --- .../0_stateless/02805_distributed_queries_timeouts.reference | 0 .../queries/0_stateless/02805_distributed_queries_timeouts.sql | 3 --- 2 files changed, 3 deletions(-) delete mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.reference delete mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.sql diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference b/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql deleted file mode 100644 index 0b7337d1255..00000000000 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ /dev/null @@ -1,3 +0,0 @@ -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 5d96b684a640ef7cc0f911d25ab8fbde4db9f590 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 11:30:13 +0800 Subject: [PATCH 146/178] update format --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index d970662d38b..9320cc4ce8c 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -26,7 +26,7 @@ TRUSTED_CONTRIBUTORS = { "thevar1able", # ClickHouse, Inc "tsolodov", # ClickHouse, Inc "justindeguzman", # ClickHouse, Inc - "XuJia0210", # ClickHouse, Inc + "XuJia0210", # ClickHouse, Inc ] } From c05b2bfd39bdb12290f2c698bfdbdec41021a45e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:33:33 +0200 Subject: [PATCH 147/178] More clarity in the test `03001_consider_lwd_when_merge` --- tests/queries/0_stateless/03001_consider_lwd_when_merge.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03001_consider_lwd_when_merge.sql b/tests/queries/0_stateless/03001_consider_lwd_when_merge.sql index 988d7058f21..2b10c72ae1b 100644 --- a/tests/queries/0_stateless/03001_consider_lwd_when_merge.sql +++ b/tests/queries/0_stateless/03001_consider_lwd_when_merge.sql @@ -7,12 +7,14 @@ SETTINGS max_bytes_to_merge_at_max_space_in_pool = 80000, exclude_deleted_rows_f INSERT INTO lwd_merge SELECT number FROM numbers(10000); INSERT INTO lwd_merge SELECT number FROM numbers(10000, 10000); -OPTIMIZE TABLE lwd_merge; +SET optimize_throw_if_noop = 1; + +OPTIMIZE TABLE lwd_merge; -- { serverError CANNOT_ASSIGN_OPTIMIZE } SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_merge' AND active = 1; DELETE FROM lwd_merge WHERE id % 10 > 0; -OPTIMIZE TABLE lwd_merge; +OPTIMIZE TABLE lwd_merge; -- { serverError CANNOT_ASSIGN_OPTIMIZE } SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'lwd_merge' AND active = 1; ALTER TABLE lwd_merge MODIFY SETTING exclude_deleted_rows_for_part_size_in_merge = 1; From 1d2f3a1ed5274764a2a051994ebd69464b6b1212 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:56:26 +0200 Subject: [PATCH 148/178] Update pr.py --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index 9320cc4ce8c..95130fc2a0f 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -9,7 +9,6 @@ from typing import Tuple TRUSTED_CONTRIBUTORS = { e.lower() for e in [ - "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", # SEMRush "bharatnc", # Many contributions. @@ -19,11 +18,9 @@ TRUSTED_CONTRIBUTORS = { "nvartolomei", # Seasoned contributor, CloudFlare "taiyang-li", "ucasFL", # Amos Bird's friend - "vdimir", # ClickHouse, Inc "thomoco", # ClickHouse, Inc "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse, Inc - "thevar1able", # ClickHouse, Inc "tsolodov", # ClickHouse, Inc "justindeguzman", # ClickHouse, Inc "XuJia0210", # ClickHouse, Inc From e292358ade930e2779792365407fa229cd37d079 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 06:12:42 +0200 Subject: [PATCH 149/178] Update run.sh --- docker/test/fasttest/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 58c436b469b..57e7b6b2f56 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,9 +11,9 @@ stage=${stage:-} # Compiler version, normally set by Dockerfile export LLVM_VERSION=${LLVM_VERSION:-18} -# For some reason it is needed for tests with 'expect' but I don't know, why -groupadd --system --gid 1000 clickhouse -useradd --system --gid 1000 --uid 1000 clickhouse +# For some reason, it is needed for tests with 'expect', but I don't know, why. +sudo groupadd --system --gid 1000 clickhouse +sudo useradd --system --gid 1000 --uid 1000 clickhouse # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about From f3047cc78dc310e72b913964429f925876f27fd1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 12 Jul 2024 10:35:16 +0800 Subject: [PATCH 150/178] fixed type mismatch in cross join --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- tests/queries/0_stateless/00202_cross_join.reference | 1 + tests/queries/0_stateless/00202_cross_join.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index fa8ebd2c0f0..0c7cad4360d 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -125,7 +125,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s if (isCrossOrComma(kind)) { data->type = Type::CROSS; - sample_block_with_columns_to_add = right_sample_block; + sample_block_with_columns_to_add = materializeBlock(right_sample_block); } else if (table_join->getClauses().empty()) { diff --git a/tests/queries/0_stateless/00202_cross_join.reference b/tests/queries/0_stateless/00202_cross_join.reference index a8db281730a..e134631383d 100644 --- a/tests/queries/0_stateless/00202_cross_join.reference +++ b/tests/queries/0_stateless/00202_cross_join.reference @@ -43,3 +43,4 @@ 2 2 2 3 2 4 +1 1 1 1 1 diff --git a/tests/queries/0_stateless/00202_cross_join.sql b/tests/queries/0_stateless/00202_cross_join.sql index 8d62c56b3f1..e4929d038c3 100644 --- a/tests/queries/0_stateless/00202_cross_join.sql +++ b/tests/queries/0_stateless/00202_cross_join.sql @@ -5,3 +5,5 @@ SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN SET allow_experimental_analyzer = 1; SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; + +SELECT * FROM ( SELECT 1 AS a, toLowCardinality(1), 1) AS t1 CROSS JOIN (SELECT toLowCardinality(1 AS a), 1 AS b) AS t2; From 9c1532e02f5f1dfeb2f3833afe585e26d08ccbf0 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 15 Jul 2024 09:46:51 +0800 Subject: [PATCH 151/178] add tests --- tests/queries/0_stateless/00202_cross_join.reference | 1 - tests/queries/0_stateless/00202_cross_join.sql | 2 -- .../queries/0_stateless/03205_column_type_check.reference | 2 ++ tests/queries/0_stateless/03205_column_type_check.sql | 7 +++++++ 4 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03205_column_type_check.reference create mode 100644 tests/queries/0_stateless/03205_column_type_check.sql diff --git a/tests/queries/0_stateless/00202_cross_join.reference b/tests/queries/0_stateless/00202_cross_join.reference index e134631383d..a8db281730a 100644 --- a/tests/queries/0_stateless/00202_cross_join.reference +++ b/tests/queries/0_stateless/00202_cross_join.reference @@ -43,4 +43,3 @@ 2 2 2 3 2 4 -1 1 1 1 1 diff --git a/tests/queries/0_stateless/00202_cross_join.sql b/tests/queries/0_stateless/00202_cross_join.sql index e4929d038c3..8d62c56b3f1 100644 --- a/tests/queries/0_stateless/00202_cross_join.sql +++ b/tests/queries/0_stateless/00202_cross_join.sql @@ -5,5 +5,3 @@ SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN SET allow_experimental_analyzer = 1; SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; - -SELECT * FROM ( SELECT 1 AS a, toLowCardinality(1), 1) AS t1 CROSS JOIN (SELECT toLowCardinality(1 AS a), 1 AS b) AS t2; diff --git a/tests/queries/0_stateless/03205_column_type_check.reference b/tests/queries/0_stateless/03205_column_type_check.reference new file mode 100644 index 00000000000..3b6c93a0610 --- /dev/null +++ b/tests/queries/0_stateless/03205_column_type_check.reference @@ -0,0 +1,2 @@ +1 nan 1048575 2 +1 1 1 1 1 diff --git a/tests/queries/0_stateless/03205_column_type_check.sql b/tests/queries/0_stateless/03205_column_type_check.sql new file mode 100644 index 00000000000..ab122821eb0 --- /dev/null +++ b/tests/queries/0_stateless/03205_column_type_check.sql @@ -0,0 +1,7 @@ +SELECT * FROM (SELECT toUInt256(1)) AS t, (SELECT greatCircleAngle(toLowCardinality(toNullable(toUInt256(1048575))), 257, -9223372036854775808, 1048576), 1048575, materialize(2)) AS u; + + +SET join_algorithm='hash'; +SET allow_experimental_join_condition=1; +SELECT * FROM ( SELECT 1 AS a, toLowCardinality(1), 1) AS t1 CROSS JOIN (SELECT toLowCardinality(1 AS a), 1 AS b) AS t2; + From 1f752eb313804fcdb1e360a69b195a2481fd970a Mon Sep 17 00:00:00 2001 From: Yinzuo Jiang Date: Tue, 16 Jul 2024 15:59:10 +0800 Subject: [PATCH 152/178] Update build-cross-riscv.md --- docs/en/development/build-cross-riscv.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-cross-riscv.md b/docs/en/development/build-cross-riscv.md index 759d97823e2..dd97b6081e8 100644 --- a/docs/en/development/build-cross-riscv.md +++ b/docs/en/development/build-cross-riscv.md @@ -11,7 +11,7 @@ This is for the case when you have Linux machine and want to use it to build `cl The cross-build for RISC-V 64 is based on the [Build instructions](../development/build.md), follow them first. -## Install Clang-16 +## Install Clang-18 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do ``` From a9b8c2a29a69d85be827048f02ca85a84ec7afca Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 10:48:43 +0200 Subject: [PATCH 153/178] CI: Do not block Tests_3 unless MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI --- tests/ci/functional_test_check.py | 28 +++++++++++- tests/ci/integration_test_check.py | 19 ++++++++ tests/ci/merge_pr.py | 70 ++++-------------------------- 3 files changed, 54 insertions(+), 63 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 41c7ed963c9..74dd4d8fbd7 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -17,9 +17,19 @@ from download_release_packages import download_last_release from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from get_robot_token import get_parameter_from_ssm from pr_info import PRInfo -from report import ERROR, SUCCESS, JobReport, StatusType, TestResults, read_test_results +from report import ( + ERROR, + SUCCESS, + JobReport, + StatusType, + TestResults, + read_test_results, + FAILURE, +) from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI +from ci_utils import Utils NO_CHANGES_MSG = "Nothing to run" @@ -351,7 +361,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 80ac1935d95..6245f0490fc 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -23,10 +23,13 @@ from report import ( TestResult, TestResults, read_test_results, + FAILURE, ) from stopwatch import Stopwatch import integration_tests_runner as runner +from ci_config import CI +from ci_utils import Utils def get_json_params_dict( @@ -233,7 +236,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 94456506879..4b57467ebdc 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -27,7 +27,6 @@ from report import SUCCESS, FAILURE from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY from synchronizer_utils import SYNC_BRANCH_PREFIX from ci_config import CI -from ci_utils import Utils # The team name for accepted approvals TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core") @@ -249,74 +248,21 @@ def main(): repo = gh.get_repo(args.repo) if args.set_ci_status: + # set Mergeable check status and exit assert args.wf_status in (FAILURE, SUCCESS) - # set mergeable check status and exit commit = get_commit(gh, args.pr_info.sha) statuses = get_commit_filtered_statuses(commit) - max_failed_tests_per_job = 0 - job_name_with_max_failures = None - total_failed_tests = 0 - failed_to_get_info = False has_failed_statuses = False for status in statuses: - if not CI.is_required(status.context) or status.context in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # CI.StatusNames.SYNC or CI.StatusNames.PR_CHECK should not be checked - continue print(f"Check status [{status.context}], [{status.state}]") - if status.state == FAILURE: + if not CI.is_required(status.context) and status.state != SUCCESS: has_failed_statuses = True - failed_cnt = Utils.get_failed_tests_number(status.description) - if failed_cnt is None: - failed_to_get_info = True - print( - f"WARNING: failed to get number of failed tests from [{status.description}]" - ) - else: - if failed_cnt > max_failed_tests_per_job: - job_name_with_max_failures = status.context - max_failed_tests_per_job = failed_cnt - total_failed_tests += failed_cnt - print( - f"Failed test cases in [{status.context}] is [{failed_cnt}], total failures [{total_failed_tests}]" - ) - elif status.state != SUCCESS and status.context not in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) - has_failed_statuses = True - print( - f"Unexpected status for [{status.context}]: [{status.state}] - block further testing" - ) - failed_to_get_info = True - - can_continue = True - if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI: - print( - f"Required check has [{total_failed_tests}] failed - block further testing" - ) - can_continue = False - if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI: - print( - f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing" - ) - can_continue = False - if failed_to_get_info: - print("Unexpected commit status state - block further testing") - can_continue = False - if args.wf_status != SUCCESS and not has_failed_statuses: - # workflow failed but reason is unknown as no failed statuses present - can_continue = False - print( - "WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation." - ) if args.wf_status == SUCCESS or has_failed_statuses: - # do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted + # set Mergeable check if workflow is successful (green) + # or if we have GH statuses with failures (red) + # to avoid false-green on a died runner state = trigger_mergeable_check( commit, statuses, @@ -333,10 +279,10 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - - if not can_continue: + if args.wf_status == SUCCESS and not has_failed_statuses: + sys.exit(0) + else: sys.exit(1) - sys.exit(0) # An ugly and not nice fix to patch the wrong organization URL, # see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710 From 77c4e6ae8ca03022c99c1f3f328ef1c8d282ed7b Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 16 Jul 2024 12:21:08 +0200 Subject: [PATCH 154/178] Stateless tests: disable sequential tests in parallel tests --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index c0bfc12bc75..2b535f8dd23 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -12,7 +12,7 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME)) USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} -RUN_SEQUENTIAL_TESTS_IN_PARALLEL=1 +RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 From e8627601b52e28a4d4203cd365cd5e4693e2af57 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 Jul 2024 13:08:25 +0200 Subject: [PATCH 155/178] Add a note about reading subcolumns from Array type in documentation --- docs/en/sql-reference/data-types/array.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/docs/en/sql-reference/data-types/array.md b/docs/en/sql-reference/data-types/array.md index e5a8ce5d18b..1737fdd88b2 100644 --- a/docs/en/sql-reference/data-types/array.md +++ b/docs/en/sql-reference/data-types/array.md @@ -96,3 +96,22 @@ Result: │ 1 │ [2] │ [[4,1]] │ └───────────┴───────────┴───────────┘ ``` + +## Reading nested subcolumns from Array + +If nested type `T` inside `Array` has subcolumns (for example, if it's a [named tuple](./tuple.md)), you can read its subcolumns from an `Array(T)` type with the same subcolumn names. The type of a subcolumn will be `Array` of the type of original subcolumn. + +**Example** + +```sql +CREATE TABLE t_arr (arr Array(Tuple(field1 UInt32, field2 String))) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_arr VALUES ([(1, 'Hello'), (2, 'World')]), ([(3, 'This'), (4, 'is'), (5, 'subcolumn')]); +SELECT arr.field1, toTypeName(arr.field1), arr.field2, toTypeName(arr.field2) from t_arr; +``` + +```test +┌─arr.field1─┬─toTypeName(arr.field1)─┬─arr.field2────────────────┬─toTypeName(arr.field2)─┐ +│ [1,2] │ Array(UInt32) │ ['Hello','World'] │ Array(String) │ +│ [3,4,5] │ Array(UInt32) │ ['This','is','subcolumn'] │ Array(String) │ +└────────────┴────────────────────────┴───────────────────────────┴────────────────────────┘ +``` From 630a29f537ecae672dcc295f3f29cea5478253a2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Jul 2024 15:11:17 +0200 Subject: [PATCH 156/178] Fix bsdtar for @nikitamikhaylov --- tests/ci/compress_files.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/compress_files.py b/tests/ci/compress_files.py index 70c0aca6a36..f49c872b5a0 100644 --- a/tests/ci/compress_files.py +++ b/tests/ci/compress_files.py @@ -58,14 +58,14 @@ def decompress_fast(archive_path: Path, result_path: Optional[Path] = None) -> N archive_path, result_path, ) - program_part = "--use-compress-program='zstd --threads=0'" + program_part = "--use-compress-program='zstd --threads=0 -d'" elif PIGZ.exists(): logging.info( "pigz found, will compress and decompress faster ('%s' -> '%s')", archive_path, result_path, ) - program_part = "--use-compress-program='pigz'" + program_part = "--use-compress-program='pigz -d'" else: program_part = "-z" logging.info( From 2af361eb14617d067732795424c22a6f511c9f18 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 16:18:41 +0200 Subject: [PATCH 157/178] minor fix --- tests/ci/merge_pr.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 4b57467ebdc..b005be472c7 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -256,7 +256,12 @@ def main(): has_failed_statuses = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") - if not CI.is_required(status.context) and status.state != SUCCESS: + if ( + not CI.is_required(status.context) + and status.context not in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) + and status.state != SUCCESS + ): + print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True if args.wf_status == SUCCESS or has_failed_statuses: From 58ce070f8bd632793d149d2e65f05889f641bd87 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Jul 2024 15:03:46 +0000 Subject: [PATCH 158/178] Fix clang tidy after #66402 --- src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index dc1749b3196..ff40b1fcc2d 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -71,8 +71,7 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre { if (join_node->getStrictness() != JoinStrictness::All) allow_where = false; - - if (join_node->getKind() == JoinKind::Left) + else if (join_node->getKind() == JoinKind::Left) allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression); else if (join_node->getKind() == JoinKind::Right) allow_where = join_node->getRightTableExpression()->isEqual(*table_expression); From d1a70944c7740b011c9416a6e9cf57727111d0a1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 16 Jul 2024 15:09:36 +0000 Subject: [PATCH 159/178] Add test for join kind to test_storage_postgresql/test.py::test_filter_push_down --- tests/integration/test_storage_postgresql/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index ffcff36c47e..12823f1f72d 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -871,7 +871,7 @@ def test_filter_pushdown(started_cluster): ) assert result1 == result2 - for kind in ["INNER", "LEFT", "RIGHT", "FULL"]: + for kind in ["INNER", "LEFT", "RIGHT", "FULL", "ANY LEFT", "SEMI RIGHT"]: for value in [0, 10]: compare_results( "SELECT * FROM ch_table {kind} JOIN {pg_table} as p ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL", From db7fb78a04280179b7ca09f588c957d2e0eab6be Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 19:49:21 +0200 Subject: [PATCH 160/178] major fix --- tests/ci/ci_config.py | 6 ++++-- tests/ci/merge_pr.py | 6 +----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9b9ddee5326..f5e39f343b2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -415,7 +415,8 @@ class CI: JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -460,7 +461,8 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600, - reference_job_name=JobNames.STATELESS_TEST_RELEASE, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), JobNames.JEPSEN_KEEPER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index b005be472c7..4d8facafb84 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -256,11 +256,7 @@ def main(): has_failed_statuses = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") - if ( - not CI.is_required(status.context) - and status.context not in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) - and status.state != SUCCESS - ): + if CI.is_required(status.context) and status.state != SUCCESS: print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True From ea3a333a0711a33ad9193e67939b5f8a970834ea Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 16 Jul 2024 22:25:45 +0200 Subject: [PATCH 161/178] Fix formatting error --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..5e2802eb08b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server. fqdn(); ``` -Aliases: `fullHostName`, 'FQDN'. +Aliases: `fullHostName`, `FQDN`. **Returned value** From 8c0ade413cca3ea63b47909c54c80de76c1e2a26 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 16 Jul 2024 22:28:12 +0200 Subject: [PATCH 162/178] Update other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5e2802eb08b..b7e4094f30e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -76,7 +76,7 @@ WHERE macro = 'test'; └───────┴──────────────┘ ``` -## FQDN +## fqdn Returns the fully qualified domain name of the ClickHouse server. From d579e9f64f1142934eeb8628f9b2764b5a022ec9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 16 Jul 2024 23:12:02 +0200 Subject: [PATCH 163/178] Fix logical error in PrometheusRequestHandler. --- src/Server/PrometheusRequestHandler.cpp | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index 1f3e038a1f5..87c106c3fc0 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -30,15 +30,8 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe response.setContentType("text/plain; version=0.0.4; charset=UTF-8"); WriteBufferFromHTTPServerResponse wb(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout, write_event); - try - { - metrics_writer->write(wb); - wb.finalize(); - } - catch (...) - { - wb.finalize(); - } + metrics_writer->write(wb); + wb.finalize(); } catch (...) { From 5ca7241865f863d68b3a3b96011b9238e49ac3f8 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 00:23:14 +0200 Subject: [PATCH 164/178] CI: AWS ASG decrease capacity from runners --- tests/ci/worker/init_runner.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index ec7793db2aa..91bffa876b2 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init script" +echo "Running init script v1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner @@ -90,7 +90,6 @@ terminate_delayed() { # IF `sleep` IS CHANGED, CHANGE ANOTHER VALUE IN `pgrep` sleep=13.14159265358979323846 echo "Going to terminate the runner's instance in $sleep seconds" - INSTANCE_ID=$(ec2metadata --instance-id) # We execute it with `at` to not have it as an orphan process, but launched independently # GH Runners kill all remain processes echo "sleep '$sleep'; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \ @@ -111,11 +110,17 @@ declare -f terminate_delayed >> /tmp/actions-hooks/common.sh terminate_and_exit() { # Terminate instance and exit from the script instantly echo "Going to terminate the runner's instance" - INSTANCE_ID=$(ec2metadata --instance-id) aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" exit 0 } +terminate_decrease_and_exit() { + # Terminate instance and exit from the script instantly + echo "Going to terminate the runner's instance and decrease asg capacity" + aws autoscaling terminate-instance-in-auto-scaling-group --instance-id "$INSTANCE_ID" --should-decrement-desired-capacity + exit 0 +} + declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh check_spot_instance_is_old() { @@ -324,7 +329,7 @@ while true; do sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \ || continue echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down" - terminate_and_exit + terminate_decrease_and_exit fi fi else From a1a802cee9f1e75cfdaed7c86aac63fd4e3010db Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 00:32:13 +0200 Subject: [PATCH 165/178] fix deployment script --- tests/ci/worker/deploy-runner-init.sh | 4 +++- tests/ci/worker/init_runner.sh | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/worker/deploy-runner-init.sh b/tests/ci/worker/deploy-runner-init.sh index 06edede48fa..96fbd82a99c 100755 --- a/tests/ci/worker/deploy-runner-init.sh +++ b/tests/ci/worker/deploy-runner-init.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +set -e + usage() { echo "Usage: $0 ENVIRONMENT" >&2 echo "Valid values for ENVIRONMENT: staging, production" >&2 @@ -55,7 +57,7 @@ EOF body() { local first_line - first_line=$(sed -n '/^# THE SCRIPT START$/{=;q}' "$SOURCE_SCRIPT") + first_line=$(sed -n '/^# THE SCRIPT START$/{=;q;}' "$SOURCE_SCRIPT") if [ -z "$first_line" ]; then echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2 exit 1 diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 91bffa876b2..d6cdb6d9c57 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init script v1" +echo "Running init v1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner From a6ee9a98f54396394460a6e8492c371b53e47fbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 01:59:22 +0200 Subject: [PATCH 166/178] Fix clang-tidy --- src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index dc1749b3196..a7f56999d73 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -69,9 +69,6 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre bool allow_where = true; if (const auto * join_node = join_tree->as()) { - if (join_node->getStrictness() != JoinStrictness::All) - allow_where = false; - if (join_node->getKind() == JoinKind::Left) allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression); else if (join_node->getKind() == JoinKind::Right) From 694c0074ab3ccafae265485062b2075b0b98d0b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 02:08:37 +0200 Subject: [PATCH 167/178] Allow to run clang-tidy with clang-19 --- .clang-tidy | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.clang-tidy b/.clang-tidy index de19059d09e..bb63bf2eea6 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -27,6 +27,8 @@ Checks: [ '-bugprone-not-null-terminated-result', '-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged '-bugprone-unchecked-optional-access', + '-bugprone-crtp-constructor-accessibility', + '-bugprone-suspicious-stringview-data-usage', '-cert-dcl16-c', '-cert-dcl37-c', @@ -36,6 +38,7 @@ Checks: [ '-cert-msc51-cpp', '-cert-oop54-cpp', '-cert-oop57-cpp', + '-cert-err33-c', # Misreports on clang-19: it warns about all functions containing 'remove' in the name, not only about the standard library. '-clang-analyzer-optin.performance.Padding', @@ -99,6 +102,7 @@ Checks: [ '-modernize-use-emplace', '-modernize-use-nodiscard', '-modernize-use-trailing-return-type', + '-modernize-use-designated-initializers', '-performance-enum-size', '-performance-inefficient-string-concatenation', From 0d1789292bc480c33c0f2b2ee54058976dea74fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 02:11:09 +0200 Subject: [PATCH 168/178] Add sudo for convenience --- docker/test/util/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index f13bb576f79..babddbd228c 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ && apt-get install \ + sudo \ apt-transport-https \ apt-utils \ ca-certificates \ From c913a36ed6580e87ebb641cb3024c7182725d8ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 02:13:57 +0200 Subject: [PATCH 169/178] Minor modifications --- docker/test/fasttest/Dockerfile | 5 ++++- docker/test/fasttest/run.sh | 4 ---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 2512268be0f..72e33226f38 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -26,7 +26,10 @@ RUN apt-get update \ zstd \ --yes --no-install-recommends \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ + && groupadd --system --gid 1000 clickhouse \ + && useradd --system --gid 1000 --uid 1000 clickhouse +# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why. COPY requirements.txt / RUN pip3 install --no-cache-dir -r /requirements.txt diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 57e7b6b2f56..970bf12a81a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,10 +11,6 @@ stage=${stage:-} # Compiler version, normally set by Dockerfile export LLVM_VERSION=${LLVM_VERSION:-18} -# For some reason, it is needed for tests with 'expect', but I don't know, why. -sudo groupadd --system --gid 1000 clickhouse -sudo useradd --system --gid 1000 --uid 1000 clickhouse - # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about # it being undefined. Also read it as array so that we can pass an empty list From a40edea5c99ac290a3b04a015065c0cad5266ab0 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 17 Jul 2024 09:37:28 +0800 Subject: [PATCH 170/178] trigger CI From 60f45cb993ccf15812605d51b6e7ce8489c98d05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 03:52:35 +0200 Subject: [PATCH 171/178] A fix --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 72e33226f38..4cac2ee6135 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -28,7 +28,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ && groupadd --system --gid 1000 clickhouse \ - && useradd --system --gid 1000 --uid 1000 clickhouse + && useradd --system --gid 1000 --uid 1000 -m clickhouse # ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why. COPY requirements.txt / From 10c19d2be123845cb2d452a4e3e735e0e28e41cf Mon Sep 17 00:00:00 2001 From: jiaosenvip <62206110+jiaosenvip@users.noreply.github.com> Date: Wed, 17 Jul 2024 11:11:34 +0800 Subject: [PATCH 172/178] Conditional statement error The conditional statement is used incorrectly here, and the two cases are correctly covered after modification. --- docs/zh/sql-reference/functions/conditional-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/sql-reference/functions/conditional-functions.md b/docs/zh/sql-reference/functions/conditional-functions.md index e0f8f7a4777..378e4d897c0 100644 --- a/docs/zh/sql-reference/functions/conditional-functions.md +++ b/docs/zh/sql-reference/functions/conditional-functions.md @@ -82,14 +82,14 @@ FROM LEFT_RIGHT SELECT left, right, - if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller + if(left < right, 'left is smaller than right', 'right is smaller or equal than left') AS is_smaller FROM LEFT_RIGHT WHERE isNotNull(left) AND isNotNull(right) ┌─left─┬─right─┬─is_smaller──────────────────────────┐ │ 1 │ 3 │ left is smaller than right │ -│ 2 │ 2 │ right is greater or equal than left │ -│ 3 │ 1 │ right is greater or equal than left │ +│ 2 │ 2 │ right is smaller or equal than left │ +│ 3 │ 1 │ right is smaller or equal than left │ └──────┴───────┴─────────────────────────────────────┘ ``` From 968d28f1f4bae02eae2f7b96c774ddfd9656c308 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 07:24:46 +0200 Subject: [PATCH 173/178] Remove outdated articles from the docs --- docs/en/sql-reference/data-types/lowcardinality.md | 3 +-- docs/zh/changelog/index.md | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md index 133ac2bd72e..a92f6f4dce6 100644 --- a/docs/en/sql-reference/data-types/lowcardinality.md +++ b/docs/en/sql-reference/data-types/lowcardinality.md @@ -56,7 +56,6 @@ Functions: ## Related content -- [Reducing ClickHouse Storage Cost with the Low Cardinality Type – Lessons from an Instana Engineer](https://altinity.com/blog/2020-5-20-reducing-clickhouse-storage-cost-with-the-low-cardinality-type-lessons-from-an-instana-engineer) -- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf) - Blog: [Optimizing ClickHouse with Schemas and Codecs](https://clickhouse.com/blog/optimize-clickhouse-codecs-compression-schema) - Blog: [Working with time series data in ClickHouse](https://clickhouse.com/blog/working-with-time-series-data-and-functions-ClickHouse) +- [String Optimization (video presentation in Russian)](https://youtu.be/rqf-ILRgBdY?list=PL0Z2YDlm0b3iwXCpEFiOOYmwXzVmjJfEt). [Slides in English](https://github.com/ClickHouse/clickhouse-presentations/raw/master/meetup19/string_optimization.pdf) diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index c91d8bcf4d1..cd77a8c03cf 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -356,7 +356,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" #### 新功能 {#new-feature-1} -- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 这个新功能是由Altinity的特殊要求添加到错误修正版本中的。 +- 添加 `deduplicate_blocks_in_dependent_materialized_views` 用于控制具有实例化视图的表中幂等插入的行为的选项。 [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) ### ClickHouse版本v20.1.2.4,2020-01-22 {#clickhouse-release-v20-1-2-4-2020-01-22} From 005256429fb4d6a9aec88b3bb5662382166f6f9b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Jul 2024 11:49:15 +0200 Subject: [PATCH 174/178] Don't run test 03198_dynamic_read_subcolumns in azure --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 27fca179580..155009cc2fe 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings, no-s3-storage +-- Tags: no-random-settings, no-object-storage SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; From 2958803255f0f646147eca4f9432f72496318bbf Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 11:26:38 +0200 Subject: [PATCH 175/178] do not decrease from lambda --- tests/ci/autoscale_runners_lambda/app.py | 77 ++++++++++--------- .../test_autoscale.py | 25 ++++-- 2 files changed, 60 insertions(+), 42 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 520abcb41bb..aed5cf3a8dd 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -99,7 +99,8 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - scale_down, scale_up = get_scales(runner_type) + #scale_down, scale_up = get_scales(runner_type) + _, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state effective_capacity = max( @@ -110,7 +111,7 @@ def set_capacity( # How much nodes are free (positive) or need to be added (negative) capacity_reserve = effective_capacity - running - queued stop = False - if capacity_reserve < 0: + if capacity_reserve <= 0: # This part is about scaling up capacity_deficit = -capacity_reserve # It looks that we are still OK, since no queued jobs exist @@ -158,41 +159,43 @@ def set_capacity( ) return - # Now we will calculate if we need to scale down - stop = stop or asg["DesiredCapacity"] == asg["MinSize"] - new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) - new_capacity = max(new_capacity, asg["MinSize"]) - new_capacity = min(new_capacity, asg["MaxSize"]) - stop = stop or asg["DesiredCapacity"] == new_capacity - if stop: - logging.info( - "Do not decrease ASG %s capacity, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - return - - logging.info( - "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - new_capacity, - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - if not dry_run: - client.set_desired_capacity( - AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=new_capacity, - ) + # FIXME: try decreasing capacity from runners that finished their jobs and have no job assigned + # IMPORTANT: Runner init script must be of version that supports ASG decrease + # # Now we will calculate if we need to scale down + # stop = stop or asg["DesiredCapacity"] == asg["MinSize"] + # new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) + # new_capacity = max(new_capacity, asg["MinSize"]) + # new_capacity = min(new_capacity, asg["MaxSize"]) + # stop = stop or asg["DesiredCapacity"] == new_capacity + # if stop: + # logging.info( + # "Do not decrease ASG %s capacity, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # return + # + # logging.info( + # "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # new_capacity, + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # if not dry_run: + # client.set_desired_capacity( + # AutoScalingGroupName=asg["AutoScalingGroupName"], + # DesiredCapacity=new_capacity, + # ) def main(dry_run: bool = True) -> None: diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 3aeab49ffc7..d1a1f9b358f 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -97,19 +97,34 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), # Decrease capacity - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + # FIXME: Tests changed for lambda that can only scale up + # TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), + # TestCase( + # "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + # ), TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], -1 ), - TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + # TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), + TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), + # TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], -1), + # TestCase( + # "style-checker", + # 1, + # 13, + # 20, + # [Queue("in_progress", 5, "style-checker")], + # 5, + # ), TestCase( "style-checker", 1, 13, 20, [Queue("in_progress", 5, "style-checker")], - 5, + -1, ), ) for t in test_cases: From c40473ff80c0cdf09b85b7dda695a41ee2df0112 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Jul 2024 10:50:52 +0000 Subject: [PATCH 176/178] Automatic style fix --- tests/ci/autoscale_runners_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index aed5cf3a8dd..c790a5081dd 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -99,7 +99,7 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - #scale_down, scale_up = get_scales(runner_type) + # scale_down, scale_up = get_scales(runner_type) _, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state From 3ade6de95df356adfaf7f6a6e3fb22d2545c6c50 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Jul 2024 13:13:51 +0200 Subject: [PATCH 177/178] Add description about no-object-storage tag --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 155009cc2fe..5d473064c68 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,5 @@ -- Tags: no-random-settings, no-object-storage +-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; From 245bc587ae5d9cb6751153b7bf24dcf88487d5dd Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 17 Jul 2024 17:57:28 +0200 Subject: [PATCH 178/178] add 'fqdn' to list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 4e7bf7f686e..b4b18f9bbf6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1653,6 +1653,7 @@ formated formatschema formatter formatters +fqdn frac freezed fromDaysSinceYearZero