From defed923313e2cf8c33d3b0890d6a2b86e563c45 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 12 Mar 2024 11:38:27 +0000 Subject: [PATCH 001/243] do nothing in `waitForOutdatedPartsToBeLoaded()` if loading is not required --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d56cf761cf4..85389828e57 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1979,6 +1979,15 @@ void MergeTreeData::waitForOutdatedPartsToBeLoaded() const TSA_NO_THREAD_SAFETY_ if (isStaticStorage()) return; + /// If waiting is not required, do NOT log and do NOT enable/disable turbo mode to make `waitForOutdatedPartsToBeLoaded` a lightweight check + { + std::unique_lock lock(outdated_data_parts_mutex); + if (outdated_data_parts_loading_canceled) + throw Exception(ErrorCodes::NOT_INITIALIZED, "Loading of outdated data parts was already canceled"); + if (outdated_data_parts_loading_finished) + return; + } + /// We need to load parts as fast as possible getOutdatedPartsLoadingThreadPool().enableTurboMode(); SCOPE_EXIT({ From 051103b0e0fef928cc41aafd00de7565b776dfd8 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 15:39:21 +0000 Subject: [PATCH 002/243] Fix db iterator wait during async metrics collection --- src/Databases/DatabaseAtomic.cpp | 4 +- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseDictionary.cpp | 2 +- src/Databases/DatabaseDictionary.h | 2 +- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseFilesystem.h | 2 +- src/Databases/DatabaseHDFS.h | 2 +- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 44 +++++++++++++------ src/Databases/DatabaseOrdinary.h | 3 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabaseS3.cpp | 2 +- src/Databases/DatabaseS3.h | 2 +- src/Databases/DatabasesCommon.cpp | 4 +- src/Databases/DatabasesCommon.h | 2 +- src/Databases/DatabasesOverlay.cpp | 2 +- src/Databases/DatabasesOverlay.h | 2 +- src/Databases/IDatabase.h | 13 +++++- .../MySQL/DatabaseMaterializedMySQL.cpp | 5 +-- .../MySQL/DatabaseMaterializedMySQL.h | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.h | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 6 +-- .../DatabaseMaterializedPostgreSQL.h | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.h | 2 +- src/Interpreters/DatabaseCatalog.cpp | 3 ++ src/Interpreters/DatabaseCatalog.h | 6 +-- src/Interpreters/InterpreterDropQuery.cpp | 1 + .../ServerAsynchronousMetrics.cpp | 5 ++- src/Server/ReplicasStatusHandler.cpp | 5 ++- 33 files changed, 85 insertions(+), 56 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 0bf7c8af4b4..37b6123acef 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -416,9 +416,9 @@ void DatabaseAtomic::assertCanBeDetached(bool cleanup) } DatabaseTablesIteratorPtr -DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name) const +DatabaseAtomic::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const { - auto base_iter = DatabaseOrdinary::getTablesIterator(local_context, filter_by_table_name); + auto base_iter = DatabaseOrdinary::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded); return std::make_unique(std::move(typeid_cast(*base_iter))); } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 404478f7cd1..b59edd479ba 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -46,7 +46,7 @@ public: void drop(ContextPtr /*context*/) override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 76fdb4fa961..adb9a659fcd 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -80,7 +80,7 @@ StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, ContextPtr return createStorageDictionary(getDatabaseName(), load_result, getContext()); } -DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const { return std::make_unique(listTables(filter_by_table_name), getDatabaseName()); } diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index 469801d183e..a18ea833710 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -34,7 +34,7 @@ public: StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; bool empty() const override; diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 05af0acf978..b27a816a60d 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -229,7 +229,7 @@ std::vector> DatabaseFilesystem::getTablesForBacku * Returns an empty iterator because the database does not have its own tables * But only caches them for quick access */ -DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction &) const +DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const { return std::make_unique(Tables{}, getDatabaseName()); } diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index 3338aa28c21..4b9db5e574d 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -45,7 +45,7 @@ public: std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const override; protected: StoragePtr getTableImpl(const String & name, ContextPtr context, bool throw_on_error) const; diff --git a/src/Databases/DatabaseHDFS.h b/src/Databases/DatabaseHDFS.h index b586a912e16..d19918000cf 100644 --- a/src/Databases/DatabaseHDFS.h +++ b/src/Databases/DatabaseHDFS.h @@ -45,7 +45,7 @@ public: void shutdown() override; std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const override; protected: StoragePtr getTableImpl(const String & name, ContextPtr context) const; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 623c7fbee98..fb1b3ee626b 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -152,7 +152,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const return loadTable(table_name); } -DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const { std::lock_guard lock(mutex); Strings filtered_tables; diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 2b1b119754d..4347649117d 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -62,7 +62,7 @@ public: bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 95bdcfc7dce..3859c2fe0ce 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -438,24 +438,40 @@ void DatabaseOrdinary::stopLoading() stop_load_table.clear(); } -DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const { - // Wait for every table (matching the filter) to be loaded and started up before we make the snapshot. - // It is important, because otherwise table might be: - // - not attached and thus will be missed in the snapshot; - // - not started, which is not good for DDL operations. - LoadTaskPtrs tasks_to_wait; + if (!skip_not_loaded) + { + // Wait for every table (matching the filter) to be loaded and started up before we make the snapshot. + // It is important, because otherwise table might be: + // - not attached and thus will be missed in the snapshot; + // - not started, which is not good for DDL operations. + LoadTaskPtrs tasks_to_wait; + { + std::lock_guard lock(mutex); + if (!filter_by_table_name) + tasks_to_wait.reserve(startup_table.size()); + for (const auto & [table_name, task] : startup_table) + if (!filter_by_table_name || filter_by_table_name(table_name)) + tasks_to_wait.emplace_back(task); + } + waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), tasks_to_wait); + } + return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded); +} + +Strings DatabaseOrdinary::getAllTableNames(ContextPtr context) const +{ + std::set unique_names; { std::lock_guard lock(mutex); - if (!filter_by_table_name) - tasks_to_wait.reserve(startup_table.size()); - for (const auto & [table_name, task] : startup_table) - if (!filter_by_table_name || filter_by_table_name(table_name)) - tasks_to_wait.emplace_back(task); + for (const auto & [table_name, _] : tables) + unique_names.emplace(table_name); + // Not yet loaded table are not listed in `tables`, so we have to add table names from tasks + for (const auto & [table_name, _] : startup_table) + unique_names.emplace(table_name); } - waitLoad(currentPoolOr(TablesLoaderForegroundPoolId), tasks_to_wait); - - return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name); + return {unique_names.begin(), unique_names.end()}; } void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 7089540337a..fa5827903cb 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -56,7 +56,8 @@ public: LoadTaskPtr startupDatabaseAsync(AsyncLoader & async_loader, LoadJobSet startup_after, LoadingStrictnessLevel mode) override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; + Strings getAllTableNames(ContextPtr context) const override; void alterTable( ContextPtr context, diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3b6a712510d..0c2cf1bb011 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1323,13 +1323,13 @@ void DatabaseReplicated::drop(ContextPtr context_) void DatabaseReplicated::stopReplication() { - stopLoading(); if (ddl_worker) ddl_worker->shutdown(); } void DatabaseReplicated::shutdown() { + stopLoading(); stopReplication(); ddl_worker_initialized = false; ddl_worker = nullptr; diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 159a5242dbe..1589cc1c75d 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -303,7 +303,7 @@ std::vector> DatabaseS3::getTablesForBackup(const * Returns an empty iterator because the database does not have its own tables * But only caches them for quick access */ -DatabaseTablesIteratorPtr DatabaseS3::getTablesIterator(ContextPtr, const FilterByNameFunction &) const +DatabaseTablesIteratorPtr DatabaseS3::getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const { return std::make_unique(Tables{}, getDatabaseName()); } diff --git a/src/Databases/DatabaseS3.h b/src/Databases/DatabaseS3.h index 5e7375dbd58..7e38da0fe63 100644 --- a/src/Databases/DatabaseS3.h +++ b/src/Databases/DatabaseS3.h @@ -56,7 +56,7 @@ public: void shutdown() override; std::vector> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const override; static Configuration parseArguments(ASTs engine_args, ContextPtr context); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index f8d6ad69ba8..733af31bdff 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -214,7 +214,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, Con return tryGetTableNoWait(table_name); } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const { std::lock_guard lock(mutex); if (!filter_by_table_name) @@ -347,7 +347,7 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name backQuote(database_name), backQuote(table_name)); } -std::vector> DatabaseWithOwnTablesBase::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const +std::vector> DatabaseWithOwnTablesBase::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context, bool skip_not_loaded) const { std::vector> res; diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 81a3c55a435..2eecf8a564f 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -35,7 +35,7 @@ public: StoragePtr detachTable(ContextPtr context, const String & table_name) override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(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; diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index c8705254e73..2772db5e066 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -254,7 +254,7 @@ void DatabasesOverlay::shutdown() db->shutdown(); } -DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context_, const FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context_, const FilterByNameFunction & filter_by_table_name, bool /*skip_not_loaded*/) const { Tables tables; for (const auto & db : databases) diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index b58df506f70..e0c31e009cc 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -52,7 +52,7 @@ public: void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr restore_coordination, UInt64 timeout_ms) override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; bool empty() const override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 75662bfebe3..b00f2fe4baf 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -229,7 +229,18 @@ public: /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name = {}) const = 0; /// NOLINT + /// 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 + + /// Returns list of table names. + virtual Strings getAllTableNames(ContextPtr context) const + { + // NOTE: This default implementation wait for all tables to be loaded and started up. It should be reimplemented for databases that support async loading. + Strings result; + for (auto table_it = getTablesIterator(context); table_it->isValid(); table_it->next()) + result.emplace_back(table_it->name()); + return result; + } /// Is the database empty. virtual bool empty() const = 0; diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 0f0d73ae16f..d8360a24bcb 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -185,9 +185,9 @@ StoragePtr DatabaseMaterializedMySQL::tryGetTable(const String & name, ContextPt } DatabaseTablesIteratorPtr -DatabaseMaterializedMySQL::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const +DatabaseMaterializedMySQL::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const { - DatabaseTablesIteratorPtr iterator = DatabaseAtomic::getTablesIterator(context_, filter_by_table_name); + DatabaseTablesIteratorPtr iterator = DatabaseAtomic::getTablesIterator(context_, filter_by_table_name, skip_not_loaded); if (context_->isInternalQuery()) return iterator; return std::make_unique(std::move(iterator), this); @@ -201,7 +201,6 @@ void DatabaseMaterializedMySQL::checkIsInternalQuery(ContextPtr context_, const void DatabaseMaterializedMySQL::stopReplication() { - stopLoading(); materialize_thread.stopSynchronization(); started_up = false; } diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index d2976adcadb..a6418e6fc5c 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -73,7 +73,7 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context_) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; void checkIsInternalQuery(ContextPtr context_, const char * method) const; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index d9b0f7f9ac7..b2e199735db 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -105,7 +105,7 @@ bool DatabaseMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name) const +DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const { Tables tables; std::lock_guard lock(mutex); diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index e5b1f434d2f..084a8339be3 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -58,7 +58,7 @@ public: bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_nam, bool skip_not_loaded) const override; ASTPtr getCreateDatabaseQuery() const override; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index b44bc136b1f..5ef44d3826c 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -455,8 +455,6 @@ void DatabaseMaterializedPostgreSQL::shutdown() void DatabaseMaterializedPostgreSQL::stopReplication() { - stopLoading(); - std::lock_guard lock(handler_mutex); if (replication_handler) replication_handler->shutdown(); @@ -484,10 +482,10 @@ void DatabaseMaterializedPostgreSQL::drop(ContextPtr local_context) DatabaseTablesIteratorPtr DatabaseMaterializedPostgreSQL::getTablesIterator( - ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const + ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const { /// Modify context into nested_context and pass query to Atomic database. - return DatabaseAtomic::getTablesIterator(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), filter_by_table_name); + return DatabaseAtomic::getTablesIterator(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), filter_by_table_name, skip_not_loaded); } void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index dfa53fa61d7..cf1333d03c8 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -45,7 +45,7 @@ public: void stopLoading() override; DatabaseTablesIteratorPtr - getTablesIterator(ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; + getTablesIterator(ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; StoragePtr tryGetTable(const String & name, ContextPtr context) const override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 3ba7333c98e..137b9d5cef9 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -46,7 +46,7 @@ public: void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; bool isTableExist(const String & name, ContextPtr context) const override; StoragePtr tryGetTable(const String & name, ContextPtr context) const override; diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index b7a82fd9d0f..e758ea35de5 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -46,7 +46,7 @@ bool DatabaseSQLite::empty() const } -DatabaseTablesIteratorPtr DatabaseSQLite::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction &) const +DatabaseTablesIteratorPtr DatabaseSQLite::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction &, bool) const { std::lock_guard lock(mutex); diff --git a/src/Databases/SQLite/DatabaseSQLite.h b/src/Databases/SQLite/DatabaseSQLite.h index e5e93bbc8ce..6bd84a4d297 100644 --- a/src/Databases/SQLite/DatabaseSQLite.h +++ b/src/Databases/SQLite/DatabaseSQLite.h @@ -32,7 +32,7 @@ public: StoragePtr tryGetTable(const String & name, ContextPtr context) const override; - DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; + DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const override; bool empty() const override; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 9d9f418934f..7231181941a 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1602,6 +1602,9 @@ void DatabaseCatalog::reloadDisksTask() for (auto & database : getDatabases()) { + // WARNING: In case of `async_load_databases = true` getTablesIterator() call wait for all table in the database to be loaded. + // WARNING: It means that no database will be able to update configuration until all databases are fully loaded. + // TODO: We can split this task by table or by database to make loaded table operate as usual. auto it = database.second->getTablesIterator(getContext()); while (it->isValid()) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 6f05a3cea0f..61ec2d9e320 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -418,11 +418,7 @@ public: Names getAllRegisteredNames() const override { - Names result; - if (database) - for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next()) - result.emplace_back(table_it->name()); - return result; + return database ? database->getAllTableNames(context) : {}; } private: diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 72aa4cc63e3..237f49582a6 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -400,6 +400,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, table_context->setInternalQuery(true); /// Do not hold extra shared pointers to tables std::vector> tables_to_drop; + // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { auto table_ptr = iterator->table(); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index fe7ccd64ffe..d26cfcf9fe9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -278,7 +278,8 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr bool is_system = db.first == DatabaseCatalog::SYSTEM_DATABASE; - for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + // Note that we skip not yet loaded tables, so metrics could possibly be lower than expected on fully loaded database just after server start if `async_load_databases = true`. + for (auto iterator = db.second->getTablesIterator(getContext(), {}, true); iterator->isValid(); iterator->next()) { ++total_number_of_tables; if (is_system) @@ -408,7 +409,7 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats() if (!db.second->canContainMergeTreeTables()) continue; - for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(getContext(), {}, true); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); if (!table) diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 91c6bd722d3..67823117758 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -51,7 +51,10 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe if (!db.second->canContainMergeTreeTables()) continue; - for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + // Note that in case `async_load_databases = true` we do not want replica status handler to be hanging + // and waiting (in getTablesIterator() call) for every table to be load, so we just skip not-yet-loaded tables. + // If they have some lag it will be reflected as soon as they are load. + for (auto iterator = db.second->getTablesIterator(getContext(), {}, true); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); if (!table) From 46debe89c22e8f423c66c380caf3e90642e81a77 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 18:17:55 +0000 Subject: [PATCH 003/243] fix --- src/Databases/DatabaseReplicated.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 0c2cf1bb011..ffaefe08b5f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1329,7 +1329,6 @@ void DatabaseReplicated::stopReplication() void DatabaseReplicated::shutdown() { - stopLoading(); stopReplication(); ddl_worker_initialized = false; ddl_worker = nullptr; From 87037d6d7df0bd825e40236c1f3a932865bd93de Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 20 Mar 2024 16:17:30 +0000 Subject: [PATCH 004/243] fix build --- src/Interpreters/DatabaseCatalog.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 61ec2d9e320..629bc6c5f4a 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -418,7 +418,9 @@ public: Names getAllRegisteredNames() const override { - return database ? database->getAllTableNames(context) : {}; + if (database) + return database->getAllTableNames(context); + return {}; } private: From a91eaa78ee0ecd1e1fc78fbb8a0379403b599d9f Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 21 Mar 2024 11:46:44 +0100 Subject: [PATCH 005/243] Update src/Interpreters/ServerAsynchronousMetrics.cpp Co-authored-by: Antonio Andelic --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index d26cfcf9fe9..7703a352130 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -279,7 +279,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr bool is_system = db.first == DatabaseCatalog::SYSTEM_DATABASE; // Note that we skip not yet loaded tables, so metrics could possibly be lower than expected on fully loaded database just after server start if `async_load_databases = true`. - for (auto iterator = db.second->getTablesIterator(getContext(), {}, true); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(getContext(), {}, /*skip_not_loaded=*/true); iterator->isValid(); iterator->next()) { ++total_number_of_tables; if (is_system) From 82c171b748c8f3de04369eb04769bb5ed5ef554b Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Fri, 22 Mar 2024 11:30:15 +0000 Subject: [PATCH 006/243] add ranking functions + make the supported table more obvious --- .../sql-reference/window-functions/index.md | 176 ++++++++++++++---- 1 file changed, 142 insertions(+), 34 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 9b2ded7b6ce..2f44c36acb4 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -12,25 +12,23 @@ Some of the calculations that you can do are similar to those that can be done w ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported. -| Feature | Support or workaround | +| Feature | Supported? | |------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| ad hoc window specification (`count(*) over (partition by id order by time desc)`) | supported | -| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | supported | -| `WINDOW` clause (`select ... from table window w as (partition by id)`) | supported | -| `ROWS` frame | supported | -| `RANGE` frame | supported, the default | -| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | not supported, specify the number of seconds instead (`RANGE` works with any numeric type). | -| `GROUPS` frame | not supported | -| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | all aggregate functions are supported | -| `rank()`, `dense_rank()`, `row_number()` | supported | -| `lag/lead(value, offset)` | Not supported. Workarounds: | -| | 1) replace with `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead` | -| | 2) use `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | -| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | +| ad hoc window specification (`count(*) over (partition by id order by time desc)`) | ✅ | +| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | ✅ | +| `WINDOW` clause (`select ... from table window w as (partition by id)`) | ✅ | +| `ROWS` frame | ✅ | +| `RANGE` frame | ✅ (the default) | +| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | +| `GROUPS` frame | ❌ | +| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | +| `rank()`, `dense_rank()`, `row_number()` | ✅ | +| `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | +| ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions -There are also the following window function that's specific to ClickHouse: +There is also the following ClickHouse specific window function: ### nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL X UNITS]) @@ -89,6 +87,62 @@ These functions can be used only as a window function. Let's have a look at some examples of how window functions can be used. +### Numbering rows + +```sql +CREATE TABLE salaries +( + `team` String, + `player` String, + `salary` UInt32, + `position` String +) +Engine = Memory; + +INSERT INTO salaries FORMAT Values + ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), + ('Port Elizabeth Barbarians', 'Charles Juarez', 190000, 'F'), + ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Scott Harrison', 150000, 'D'), + ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'); +``` + +```sql +SELECT player, salary, + row_number() OVER (ORDER BY salary) AS row +FROM salaries; +``` + +```text +┌─player──────────┬─salary─┬─row─┐ +│ Michael Stanley │ 150000 │ 1 │ +│ Scott Harrison │ 150000 │ 2 │ +│ Charles Juarez │ 190000 │ 3 │ +│ Gary Chen │ 195000 │ 4 │ +│ Robert George │ 195000 │ 5 │ +└─────────────────┴────────┴─────┘ +``` + +```sql +SELECT player, salary, + row_number() OVER (ORDER BY salary) AS row, + rank() OVER (ORDER BY salary) AS rank, + dense_rank() OVER (ORDER BY salary) AS denseRank +FROM salaries; +``` + +```text +┌─player──────────┬─salary─┬─row─┬─rank─┬─denseRank─┐ +│ Michael Stanley │ 150000 │ 1 │ 1 │ 1 │ +│ Scott Harrison │ 150000 │ 2 │ 1 │ 1 │ +│ Charles Juarez │ 190000 │ 3 │ 3 │ 2 │ +│ Gary Chen │ 195000 │ 4 │ 4 │ 3 │ +│ Robert George │ 195000 │ 5 │ 4 │ 3 │ +└─────────────────┴────────┴─────┴──────┴───────────┘ +``` + +### Partitioning by column + ```sql CREATE TABLE wf_partition ( @@ -120,6 +174,8 @@ ORDER BY └──────────┴───────┴───────┴──────────────┘ ``` +### Frame bounding + ```sql CREATE TABLE wf_frame ( @@ -131,14 +187,19 @@ ENGINE = Memory; INSERT INTO wf_frame FORMAT Values (1,1,1), (1,2,2), (1,3,3), (1,4,4), (1,5,5); +``` --- frame is bounded by bounds of a partition (BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +```sql +-- Frame is bounded by bounds of a partition (BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) SELECT part_key, value, order, - groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC - Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS frame_values + groupArray(value) OVER ( + PARTITION BY part_key + ORDER BY order ASC + Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) AS frame_values FROM wf_frame ORDER BY part_key ASC, @@ -151,7 +212,9 @@ ORDER BY │ 1 │ 4 │ 4 │ [1,2,3,4,5] │ │ 1 │ 5 │ 5 │ [1,2,3,4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` +```sql -- short form - no bound expression, no order by SELECT part_key, @@ -169,14 +232,19 @@ ORDER BY │ 1 │ 4 │ 4 │ [1,2,3,4,5] │ │ 1 │ 5 │ 5 │ [1,2,3,4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` --- frame is bounded by the beggining of a partition and the current row +```sql +-- frame is bounded by the beginning of a partition and the current row SELECT part_key, value, order, - groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC - Rows BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS frame_values + groupArray(value) OVER ( + PARTITION BY part_key + ORDER BY order ASC + Rows BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS frame_values FROM wf_frame ORDER BY part_key ASC, @@ -189,8 +257,10 @@ ORDER BY │ 1 │ 4 │ 4 │ [1,2,3,4] │ │ 1 │ 5 │ 5 │ [1,2,3,4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` --- short form (frame is bounded by the beggining of a partition and the current row) +```sql +-- short form (frame is bounded by the beginning of a partition and the current row) SELECT part_key, value, @@ -207,8 +277,10 @@ ORDER BY │ 1 │ 4 │ 4 │ [1,2,3,4] │ │ 1 │ 5 │ 5 │ [1,2,3,4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` --- frame is bounded by the beggining of a partition and the current row, but order is backward +```sql +-- frame is bounded by the beginning of a partition and the current row, but order is backward SELECT part_key, value, @@ -225,14 +297,19 @@ ORDER BY │ 1 │ 4 │ 4 │ [5,4] │ │ 1 │ 5 │ 5 │ [5] │ └──────────┴───────┴───────┴──────────────┘ +``` +```sql -- sliding frame - 1 PRECEDING ROW AND CURRENT ROW SELECT part_key, value, order, - groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC - Rows BETWEEN 1 PRECEDING AND CURRENT ROW) AS frame_values + groupArray(value) OVER ( + PARTITION BY part_key + ORDER BY order ASC + Rows BETWEEN 1 PRECEDING AND CURRENT ROW + ) AS frame_values FROM wf_frame ORDER BY part_key ASC, @@ -245,14 +322,19 @@ ORDER BY │ 1 │ 4 │ 4 │ [3,4] │ │ 1 │ 5 │ 5 │ [4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` +```sql -- sliding frame - Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING SELECT part_key, value, order, - groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC - Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS frame_values + groupArray(value) OVER ( + PARTITION BY part_key + ORDER BY order ASC + Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING + ) AS frame_values FROM wf_frame ORDER BY part_key ASC, @@ -264,7 +346,9 @@ ORDER BY │ 1 │ 4 │ 4 │ [3,4,5] │ │ 1 │ 5 │ 5 │ [4,5] │ └──────────┴───────┴───────┴──────────────┘ +``` +```sql -- row_number does not respect the frame, so rn_1 = rn_2 = rn_3 != rn_4 SELECT part_key, @@ -278,8 +362,11 @@ SELECT FROM wf_frame WINDOW w1 AS (PARTITION BY part_key ORDER BY order DESC), - w2 AS (PARTITION BY part_key ORDER BY order DESC - Rows BETWEEN 1 PRECEDING AND CURRENT ROW) + w2 AS ( + PARTITION BY part_key + ORDER BY order DESC + Rows BETWEEN 1 PRECEDING AND CURRENT ROW + ) ORDER BY part_key ASC, value ASC; @@ -290,7 +377,9 @@ ORDER BY │ 1 │ 4 │ 4 │ [5,4] │ 2 │ 2 │ 2 │ 2 │ │ 1 │ 5 │ 5 │ [5] │ 1 │ 1 │ 1 │ 1 │ └──────────┴───────┴───────┴──────────────┴──────┴──────┴──────┴──────┘ +``` +```sql -- first_value and last_value respect the frame SELECT groupArray(value) OVER w1 AS frame_values_1, @@ -313,7 +402,9 @@ ORDER BY │ [1,2,3,4] │ 1 │ 4 │ [3,4] │ 3 │ 4 │ │ [1,2,3,4,5] │ 1 │ 5 │ [4,5] │ 4 │ 5 │ └────────────────┴───────────────┴──────────────┴────────────────┴───────────────┴──────────────┘ +``` +```sql -- second value within the frame SELECT groupArray(value) OVER w1 AS frame_values_1, @@ -330,7 +421,9 @@ ORDER BY │ [1,2,3,4] │ 2 │ │ [2,3,4,5] │ 3 │ └────────────────┴──────────────┘ +``` +```sql -- second value within the frame + Null for missing values SELECT groupArray(value) OVER w1 AS frame_values_1, @@ -351,6 +444,8 @@ ORDER BY ## Real world examples +The following examples solve common real-world problems. + ### Maximum/total salary per department. ```sql @@ -369,7 +464,9 @@ INSERT INTO employees FORMAT Values ('IT', 'Tim', 200), ('IT', 'Anna', 300), ('IT', 'Elen', 500); +``` +```sql SELECT department, employee_name AS emp, @@ -386,8 +483,10 @@ FROM max(salary) OVER wndw AS max_salary_per_dep, sum(salary) OVER wndw AS total_salary_per_dep FROM employees - WINDOW wndw AS (PARTITION BY department - rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + WINDOW wndw AS ( + PARTITION BY department + rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) ORDER BY department ASC, employee_name ASC @@ -421,7 +520,9 @@ INSERT INTO warehouse VALUES ('sku1', '2020-01-01', 1), ('sku1', '2020-02-01', 1), ('sku1', '2020-03-01', 1); +``` +```sql SELECT item, ts, @@ -461,13 +562,18 @@ insert into sensors values('cpu_temp', '2020-01-01 00:00:00', 87), ('cpu_temp', '2020-01-01 00:00:05', 87), ('cpu_temp', '2020-01-01 00:00:06', 87), ('cpu_temp', '2020-01-01 00:00:07', 87); +``` + +```sql SELECT metric, ts, value, - avg(value) OVER - (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN 2 PRECEDING AND CURRENT ROW) - AS moving_avg_temp + avg(value) OVER ( + PARTITION BY metric + ORDER BY ts ASC + Rows BETWEEN 2 PRECEDING AND CURRENT ROW + ) AS moving_avg_temp FROM sensors ORDER BY metric ASC, @@ -536,7 +642,9 @@ insert into sensors values('ambient_temp', '2020-01-01 00:00:00', 16), ('ambient_temp', '2020-03-01 12:00:00', 16), ('ambient_temp', '2020-03-01 12:00:00', 16), ('ambient_temp', '2020-03-01 12:00:00', 16); +``` +```sql SELECT metric, ts, From 2df818866797c23fc38063663441280059fad565 Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Fri, 22 Mar 2024 11:54:04 +0000 Subject: [PATCH 007/243] Agg functions --- .../sql-reference/window-functions/index.md | 48 +++++++++++++++++-- 1 file changed, 44 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 2f44c36acb4..19821781d0e 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -101,9 +101,9 @@ Engine = Memory; INSERT INTO salaries FORMAT Values ('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'), - ('Port Elizabeth Barbarians', 'Charles Juarez', 190000, 'F'), + ('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'), ('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'), - ('Port Elizabeth Barbarians', 'Scott Harrison', 150000, 'D'), + ('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'), ('Port Elizabeth Barbarians', 'Robert George', 195000, 'M'); ``` @@ -141,6 +141,46 @@ FROM salaries; └─────────────────┴────────┴─────┴──────┴───────────┘ ``` +### Aggregation functions + +Compare each player's salary to the average for their team. + +```sql +SELECT player, salary, team, + avg(salary) OVER (PARTITION BY team) AS teamAvg, + salary - teamAvg AS diff +FROM salaries; +``` + +```text +┌─player──────────┬─salary─┬─team──────────────────────┬─teamAvg─┬───diff─┐ +│ Charles Juarez │ 190000 │ New Coreystad Archdukes │ 170000 │ 20000 │ +│ Scott Harrison │ 150000 │ New Coreystad Archdukes │ 170000 │ -20000 │ +│ Gary Chen │ 195000 │ Port Elizabeth Barbarians │ 180000 │ 15000 │ +│ Michael Stanley │ 150000 │ Port Elizabeth Barbarians │ 180000 │ -30000 │ +│ Robert George │ 195000 │ Port Elizabeth Barbarians │ 180000 │ 15000 │ +└─────────────────┴────────┴───────────────────────────┴─────────┴────────┘ +``` + +Compare each player's salary to the maximum for their team. + +```sql +SELECT player, salary, team, + max(salary) OVER (PARTITION BY team) AS teamAvg, + salary - teamAvg AS diff +FROM salaries; +``` + +```text +┌─player──────────┬─salary─┬─team──────────────────────┬─teamAvg─┬───diff─┐ +│ Charles Juarez │ 190000 │ New Coreystad Archdukes │ 190000 │ 0 │ +│ Scott Harrison │ 150000 │ New Coreystad Archdukes │ 190000 │ -40000 │ +│ Gary Chen │ 195000 │ Port Elizabeth Barbarians │ 195000 │ 0 │ +│ Michael Stanley │ 150000 │ Port Elizabeth Barbarians │ 195000 │ -45000 │ +│ Robert George │ 195000 │ Port Elizabeth Barbarians │ 195000 │ 0 │ +└─────────────────┴────────┴───────────────────────────┴─────────┴────────┘ +``` + ### Partitioning by column ```sql @@ -446,7 +486,7 @@ ORDER BY The following examples solve common real-world problems. -### Maximum/total salary per department. +### Maximum/total salary per department ```sql CREATE TABLE employees @@ -502,7 +542,7 @@ FROM └────────────┴──────┴────────┴────────────────────┴──────────────────────┴──────────────────┘ ``` -### Cumulative sum. +### Cumulative sum ```sql CREATE TABLE warehouse From e91dc87f824a623c941292a0876223d931dc6b4b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 21 Mar 2024 17:18:04 +0100 Subject: [PATCH 008/243] Speed up dynamic resize of filesystem cache --- src/Interpreters/Cache/EvictionCandidates.cpp | 72 ++++++++---- src/Interpreters/Cache/EvictionCandidates.h | 18 +++ src/Interpreters/Cache/FileCache.cpp | 46 ++++---- src/Interpreters/Cache/FileCache.h | 1 + src/Interpreters/Cache/IFileCachePriority.h | 14 ++- .../Cache/LRUFileCachePriority.cpp | 103 +++++++++--------- src/Interpreters/Cache/LRUFileCachePriority.h | 23 +++- src/Interpreters/Cache/Metadata.cpp | 79 +++++++------- src/Interpreters/Cache/Metadata.h | 18 ++- .../Cache/SLRUFileCachePriority.cpp | 30 +++++ .../Cache/SLRUFileCachePriority.h | 7 ++ 11 files changed, 276 insertions(+), 135 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index f1ae2baa347..e24d671b66e 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -1,6 +1,8 @@ #include #include +#include +namespace fs = std::filesystem; namespace ProfileEvents { @@ -33,40 +35,68 @@ void EvictionCandidates::add(LockedKey & locked_key, const FileSegmentMetadataPt } void EvictionCandidates::evict(FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock & lock) +{ + evictImpl(false, query_context, lock); +} + +std::vector EvictionCandidates::evictFromMemory( + FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock & lock) +{ + return evictImpl(true, query_context, lock); +} + +std::vector EvictionCandidates::evictImpl( + bool remove_only_metadata, + FileCacheQueryLimit::QueryContext * query_context, + const CachePriorityGuard::Lock & lock) { if (candidates.empty()) - return; + return {}; auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); - for (auto & [key, key_candidates] : candidates) + std::vector evicted_paths; + try { - auto locked_key = key_candidates.key_metadata->tryLock(); - if (!locked_key) - continue; /// key could become invalid after we released the key lock above, just skip it. - - auto & to_evict = key_candidates.candidates; - while (!to_evict.empty()) + for (auto & [key, key_candidates] : candidates) { - auto & candidate = to_evict.back(); - chassert(candidate->releasable()); + auto locked_key = key_candidates.key_metadata->tryLock(); + if (!locked_key) + continue; /// key could become invalid after we released the key lock above, just skip it. - const auto segment = candidate->file_segment; - auto queue_it = segment->getQueueIterator(); - chassert(queue_it); + auto & to_evict = key_candidates.candidates; + while (!to_evict.empty()) + { + auto & candidate = to_evict.back(); + chassert(candidate->releasable()); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + const auto segment = candidate->file_segment; + auto queue_it = segment->getQueueIterator(); + chassert(queue_it); - locked_key->removeFileSegment(segment->offset(), segment->lock()); - queue_it->remove(lock); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); - if (query_context) - query_context->remove(segment->key(), segment->offset(), lock); + if (remove_only_metadata) + evicted_paths.push_back(segment->getPath()); - to_evict.pop_back(); + locked_key->removeFileSegment( + segment->offset(), segment->lock(), /* can_be_broken */false, remove_only_metadata); + + queue_it->remove(lock); + if (query_context) + query_context->remove(segment->key(), segment->offset(), lock); + + to_evict.pop_back(); + } } } + catch (...) + { + for (const auto & path : evicted_paths) + fs::remove(path); + throw; + } + return evicted_paths; } - } diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index e817d33d5fe..ec29692d8d3 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -7,12 +7,23 @@ namespace DB class EvictionCandidates { public: + EvictionCandidates() = default; + EvictionCandidates(EvictionCandidates && other) noexcept + { + candidates = std::move(other.candidates); + candidates_size = std::move(other.candidates_size); + invalidated_queue_entries = std::move(other.invalidated_queue_entries); + } ~EvictionCandidates(); void add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate); + void add(const EvictionCandidates & other, const CachePriorityGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } + void evict(FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock &); + std::vector evictFromMemory(FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock &); + size_t size() const { return candidates_size; } auto begin() const { return candidates.begin(); } @@ -28,6 +39,13 @@ private: std::unordered_map candidates; size_t candidates_size = 0; + + std::vector invalidated_queue_entries; + + std::vector evictImpl( + bool remove_only_metadata, + FileCacheQueryLimit::QueryContext * query_context, + const CachePriorityGuard::Lock & lock); }; using EvictionCandidatesPtr = std::unique_ptr; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 916bdb8f898..4e41c308bf2 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -180,6 +180,7 @@ void FileCache::initialize() } metadata.startup(); + is_initialized = true; } @@ -1340,34 +1341,33 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { - cache_is_being_resized.store(true, std::memory_order_relaxed); - SCOPE_EXIT({ - cache_is_being_resized.store(false, std::memory_order_relaxed); - }); - - auto cache_lock = lockCache(); - bool updated = false; - try + std::vector evicted_paths; { - updated = main_priority->modifySizeLimits( + cache_is_being_resized.store(true, std::memory_order_relaxed); + SCOPE_EXIT({ + cache_is_being_resized.store(false, std::memory_order_relaxed); + }); + + auto cache_lock = lockCache(); + FileCacheReserveStat stat; + auto eviction_candidates = main_priority->collectCandidatesForEviction( + new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock); + + evicted_paths = eviction_candidates.evictFromMemory(nullptr, cache_lock); + + main_priority->modifySizeLimits( new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); } - catch (...) - { - actual_settings.max_size = main_priority->getSizeLimit(cache_lock); - actual_settings.max_elements = main_priority->getElementsLimit(cache_lock); - throw; - } - if (updated) - { - LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", - actual_settings.max_size, new_settings.max_size, - actual_settings.max_elements, new_settings.max_elements); + for (const auto & path : evicted_paths) + fs::remove(path); - actual_settings.max_size = main_priority->getSizeLimit(cache_lock); - actual_settings.max_elements = main_priority->getElementsLimit(cache_lock); - } + LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); + + actual_settings.max_size = new_settings.max_size; + actual_settings.max_elements = new_settings.max_elements; } if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 1433a067e7e..087e1ad344d 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -18,6 +18,7 @@ #include #include #include +#include #include diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 09d71cebb01..bcde64bce98 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -107,7 +107,19 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) = 0; - virtual bool modifySizeLimits(size_t max_size_, size_t max_elements_, double size_ratio_, const CachePriorityGuard::Lock &) = 0; + /// Collect eviction `candidates_num` candidates for eviction. + virtual EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CachePriorityGuard::Lock &) = 0; + + virtual bool modifySizeLimits( + size_t max_size_, + size_t max_elements_, + double size_ratio_, + const CachePriorityGuard::Lock &) = 0; protected: IFileCachePriority(size_t max_size_, size_t max_elements_); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 08e65b577ca..4189170c5ef 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -16,9 +16,6 @@ namespace ProfileEvents { extern const Event FilesystemCacheEvictionSkippedFileSegments; extern const Event FilesystemCacheEvictionTries; - extern const Event FilesystemCacheEvictMicroseconds; - extern const Event FilesystemCacheEvictedBytes; - extern const Event FilesystemCacheEvictedFileSegments; } namespace DB @@ -205,17 +202,22 @@ bool LRUFileCachePriority::canFit( /// NOLINT IteratorPtr, bool) const { - return canFit(size, 0, 0, lock); + return canFit(size, 1, 0, 0, lock); } bool LRUFileCachePriority::canFit( size_t size, + size_t elements, size_t released_size_assumption, size_t released_elements_assumption, - const CachePriorityGuard::Lock &) const + const CachePriorityGuard::Lock &, + const size_t * max_size_, + const size_t * max_elements_) const { - return (max_size == 0 || (state->current_size + size - released_size_assumption <= max_size)) - && (max_elements == 0 || state->current_elements_num + 1 - released_elements_assumption <= max_elements); + return (max_size == 0 + || (state->current_size + size - released_size_assumption <= (max_size_ ? *max_size_ : max_size))) + && (max_elements == 0 + || state->current_elements_num + elements - released_elements_assumption <= (max_elements_ ? *max_elements_ : max_elements)); } bool LRUFileCachePriority::collectCandidatesForEviction( @@ -230,6 +232,38 @@ bool LRUFileCachePriority::collectCandidatesForEviction( if (canFit(size, lock)) return true; + auto can_fit = [&] + { + return canFit(size, 1, stat.stat.releasable_size, stat.stat.releasable_count, lock); + }; + iterateForEviction(res, stat, can_fit, lock); + return can_fit(); +} + +EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CachePriorityGuard::Lock & lock) +{ + EvictionCandidates res; + auto stop_condition = [&, this]() + { + return canFit(0, 0, stat.stat.releasable_size, stat.stat.releasable_count, + lock, &desired_size, &desired_elements_count) + || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); + }; + iterateForEviction(res, stat, stop_condition, lock); + return res; +} + +void LRUFileCachePriority::iterateForEviction( + EvictionCandidates & res, + FileCacheReserveStat & stat, + StopConditionFunc stop_condition, + const CachePriorityGuard::Lock & lock) +{ ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionTries); IterateFunc iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) @@ -244,27 +278,23 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } else { - stat.update(segment_metadata->size(), file_segment->getKind(), false); ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictionSkippedFileSegments); + stat.update(segment_metadata->size(), file_segment->getKind(), false); } return IterationResult::CONTINUE; }; - auto can_fit = [&] - { - return canFit(size, stat.stat.releasable_size, stat.stat.releasable_count, lock); - }; - iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) { - return can_fit() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); + return stop_condition() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); }, lock); - - return can_fit(); } -LRUFileCachePriority::LRUIterator LRUFileCachePriority::move(LRUIterator & it, LRUFileCachePriority & other, const CachePriorityGuard::Lock &) +LRUFileCachePriority::LRUIterator LRUFileCachePriority::move( + LRUIterator & it, + LRUFileCachePriority & other, + const CachePriorityGuard::Lock &) { const auto & entry = *it.getEntry(); if (entry.size == 0) @@ -309,45 +339,20 @@ IFileCachePriority::PriorityDumpPtr LRUFileCachePriority::dump(const CachePriori } bool LRUFileCachePriority::modifySizeLimits( - size_t max_size_, size_t max_elements_, double /* size_ratio_ */, const CachePriorityGuard::Lock & lock) + size_t max_size_, size_t max_elements_, double /* size_ratio_ */, const CachePriorityGuard::Lock &) { if (max_size == max_size_ && max_elements == max_elements_) return false; /// Nothing to change. - auto check_limits_satisfied = [&]() + if (state->current_size > max_size_ || state->current_elements_num > max_elements_) { - return (max_size_ == 0 || state->current_size <= max_size_) - && (max_elements_ == 0 || state->current_elements_num <= max_elements_); - }; - - if (check_limits_satisfied()) - { - max_size = max_size_; - max_elements = max_elements_; - return true; + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot modify size limits to {} in size and {} in elements: " + "not enough space freed. Current size: {}/{}, elements: {}/{}", + max_size_, max_elements_, + state->current_size, max_size, state->current_elements_num, max_elements); } - auto iterate_func = [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { - chassert(segment_metadata->file_segment->assertCorrectness()); - - if (!segment_metadata->releasable()) - return IterationResult::CONTINUE; - - auto segment = segment_metadata->file_segment; - locked_key.removeFileSegment(segment->offset(), segment->lock()); - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize()); - return IterationResult::REMOVE_AND_CONTINUE; - }; - - auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); - iterate( - [&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { return check_limits_satisfied() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); }, - lock); - max_size = max_size_; max_elements = max_elements_; return true; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 49977c79b81..16034f379f3 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -55,6 +55,13 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; + EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CachePriorityGuard::Lock &) override; + void shuffle(const CachePriorityGuard::Lock &) override; struct LRUPriorityDump : public IPriorityDump @@ -81,7 +88,14 @@ private: void updateElementsCount(int64_t num); void updateSize(int64_t size); - bool canFit(size_t size, size_t released_size_assumption, size_t released_elements_assumption, const CachePriorityGuard::Lock &) const; + bool canFit( + size_t size, + size_t elements, + size_t released_size_assumption, + size_t released_elements_assumption, + const CachePriorityGuard::Lock &, + const size_t * max_size_ = nullptr, + const size_t * max_elements_ = nullptr) const; LRUQueue::iterator remove(LRUQueue::iterator it, const CachePriorityGuard::Lock &); @@ -96,6 +110,13 @@ private: LRUIterator move(LRUIterator & it, LRUFileCachePriority & other, const CachePriorityGuard::Lock &); LRUIterator add(EntryPtr entry, const CachePriorityGuard::Lock &); + + using StopConditionFunc = std::function; + void iterateForEviction( + EvictionCandidates & res, + FileCacheReserveStat & stat, + StopConditionFunc stop_condition, + const CachePriorityGuard::Lock &); }; class LRUFileCachePriority::LRUIterator : public IFileCachePriority::Iterator diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index b79605622b6..65bbcb24cfa 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -899,32 +899,34 @@ bool LockedKey::removeAllFileSegments(bool if_releasable) return removed_all; } -KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, bool can_be_broken) +KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, bool can_be_broken, bool remove_only_metadata) { auto it = key_metadata->find(offset); if (it == key_metadata->end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no offset {}", offset); auto file_segment = it->second->file_segment; - return removeFileSegmentImpl(it, file_segment->lock(), can_be_broken); + return removeFileSegmentImpl(it, file_segment->lock(), can_be_broken, remove_only_metadata); } KeyMetadata::iterator LockedKey::removeFileSegment( size_t offset, const FileSegmentGuard::Lock & segment_lock, - bool can_be_broken) + bool can_be_broken, + bool remove_only_metadata) { auto it = key_metadata->find(offset); if (it == key_metadata->end()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no offset {} in key {}", offset, getKey()); - return removeFileSegmentImpl(it, segment_lock, can_be_broken); + return removeFileSegmentImpl(it, segment_lock, can_be_broken, remove_only_metadata); } KeyMetadata::iterator LockedKey::removeFileSegmentImpl( KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock, - bool can_be_broken) + bool can_be_broken, + bool remove_only_metadata) { auto file_segment = it->second->file_segment; @@ -939,47 +941,50 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( file_segment->detach(segment_lock, *this); - try + if (!remove_only_metadata) { - const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (file_segment->segment_kind == FileSegmentKind::Temporary) + try { - /// FIXME: For temporary file segment the requirement is not as strong because - /// the implementation of "temporary data in cache" creates files in advance. - if (fs::exists(path)) + const auto path = key_metadata->getFileSegmentPath(*file_segment); + if (file_segment->segment_kind == FileSegmentKind::Temporary) + { + /// FIXME: For temporary file segment the requirement is not as strong because + /// the implementation of "temporary data in cache" creates files in advance. + if (fs::exists(path)) + fs::remove(path); + } + else if (file_segment->downloaded_size == 0) + { + chassert(!fs::exists(path)); + } + else if (fs::exists(path)) + { fs::remove(path); - } - else if (file_segment->downloaded_size == 0) - { - chassert(!fs::exists(path)); - } - else if (fs::exists(path)) - { - fs::remove(path); - /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. - int flags = file_segment->getFlagsForLocalRead(); - /// Files are created with flags from file_segment->getFlagsForLocalRead() - /// plus optionally O_DIRECT is added, depends on query setting, so remove both. - OpenedFileCache::instance().remove(path, flags); - OpenedFileCache::instance().remove(path, flags | O_DIRECT); + /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. + int flags = file_segment->getFlagsForLocalRead(); + /// Files are created with flags from file_segment->getFlagsForLocalRead() + /// plus optionally O_DIRECT is added, depends on query setting, so remove both. + OpenedFileCache::instance().remove(path, flags); + OpenedFileCache::instance().remove(path, flags | O_DIRECT); - LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path); - } - else if (!can_be_broken) - { + LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path); + } + else if (!can_be_broken) + { #ifdef ABORT_ON_LOGICAL_ERROR - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); #else - LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}", - path, getKey(), file_segment->offset()); + LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}", + path, getKey(), file_segment->offset()); #endif + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); } return key_metadata->erase(it); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index c02127cdef3..6f63132c568 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -269,8 +269,16 @@ struct LockedKey : private boost::noncopyable bool removeAllFileSegments(bool if_releasable = true); - KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &, bool can_be_broken = false); - KeyMetadata::iterator removeFileSegment(size_t offset, bool can_be_broken = false); + KeyMetadata::iterator removeFileSegment( + size_t offset, + const FileSegmentGuard::Lock &, + bool can_be_broken = false, + bool remove_only_metadata = false); + + KeyMetadata::iterator removeFileSegment( + size_t offset, + bool can_be_broken = false, + bool remove_only_metadata = false); void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &); @@ -289,7 +297,11 @@ struct LockedKey : private boost::noncopyable std::string toString() const; private: - KeyMetadata::iterator removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock &, bool can_be_broken = false); + KeyMetadata::iterator removeFileSegmentImpl( + KeyMetadata::iterator it, + const FileSegmentGuard::Lock &, + bool can_be_broken = false, + bool remove_only_metadata_ = false); const std::shared_ptr key_metadata; KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`. diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index c97d05d4b84..a405c237d71 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -169,6 +169,36 @@ bool SLRUFileCachePriority::collectCandidatesForEviction( return true; } +EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CachePriorityGuard::Lock & lock) +{ + const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio); + const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); + + auto res = probationary_queue.collectCandidatesForEviction( + desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); + + chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); + chassert(res.size() == stat.stat.releasable_count); + + if (max_candidates_to_evict && res.size() == max_candidates_to_evict) + return res; + + const auto desired_protected_size = getRatio(max_size, size_ratio); + const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); + + auto res_add = protected_queue.collectCandidatesForEviction( + desired_protected_size, desired_protected_elements_num, + max_candidates_to_evict ? max_candidates_to_evict - res.size() : 0, stat, lock); + + res.add(res_add, lock); + return res; +} + void SLRUFileCachePriority::increasePriority(SLRUIterator & iterator, const CachePriorityGuard::Lock & lock) { /// If entry is already in protected queue, diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index f90918f7878..b5d75fc7955 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -52,6 +52,13 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; + EvictionCandidates collectCandidatesForEviction( + size_t desired_size, + size_t desired_elements_count, + size_t max_candidates_to_evict, + FileCacheReserveStat & stat, + const CachePriorityGuard::Lock &) override; + void shuffle(const CachePriorityGuard::Lock &) override; PriorityDumpPtr dump(const CachePriorityGuard::Lock &) override; From a85886c2e0b1d2997b1b5192fe7a489181668041 Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Fri, 22 Mar 2024 16:26:43 +0000 Subject: [PATCH 009/243] AggregatingMergeTree: Split table creation and MV definition + add more to example --- .../mergetree-family/aggregatingmergetree.md | 51 +++++++++++++++---- 1 file changed, 42 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 62191d9b5e4..7a449f400fd 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -68,6 +68,12 @@ In the results of `SELECT` query, the values of `AggregateFunction` type have im ## Example of an Aggregated Materialized View {#example-of-an-aggregated-materialized-view} +The following examples assumes that you have a database named `test` so make sure you create that if it doesn't already exist: + +```sql +CREATE DATABASE test; +``` + We will create the table `test.visits` that contain the raw data: ``` sql @@ -80,17 +86,24 @@ CREATE TABLE test.visits ) ENGINE = MergeTree ORDER BY (StartDate, CounterID); ``` +Next, we need to create an `AggregatingMergeTree` table that will store `AggregationFunction`s that keep track of the total number of visits and the number of unique users. + `AggregatingMergeTree` materialized view that watches the `test.visits` table, and use the `AggregateFunction` type: ``` sql -CREATE MATERIALIZED VIEW test.mv_visits -( +CREATE TABLE test.agg_visits ( StartDate DateTime64 NOT NULL, CounterID UInt64, Visits AggregateFunction(sum, Nullable(Int32)), Users AggregateFunction(uniq, Nullable(Int32)) ) -ENGINE = AggregatingMergeTree() ORDER BY (StartDate, CounterID) +ENGINE = AggregatingMergeTree() ORDER BY (StartDate, CounterID); +``` + +And then let's create a materialized view that populates `test.agg_visits` from `test.visits` : + +```sql +CREATE MATERIALIZED VIEW test.visits_mv TO test.agg_visits AS SELECT StartDate, CounterID, @@ -104,25 +117,45 @@ Inserting data into the `test.visits` table. ``` sql INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) - VALUES (1667446031, 1, 3, 4) -INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) - VALUES (1667446031, 1, 6, 3) + VALUES (1667446031000, 1, 3, 4), (1667446031000, 1, 6, 3); ``` -The data is inserted in both the table and the materialized view `test.mv_visits`. +The data is inserted in both `test.visits` and `test.agg_visits`. To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`: -``` sql +```sql SELECT StartDate, sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users -FROM test.mv_visits +FROM test.agg_visits GROUP BY StartDate ORDER BY StartDate; ``` +```text +┌───────────────StartDate─┬─Visits─┬─Users─┐ +│ 2022-11-03 03:27:11.000 │ 9 │ 2 │ +└─────────────────────────┴────────┴───────┘ +``` + +And how about if we add another couple of records to `test.visits`, but this time we'll use a different timestamp for one of the records: + +```sql +INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) + VALUES (1669446031000, 2, 5, 10), (1667446031000, 3, 7, 5); +``` + +If we then run the `SELECT` query again, we'll see the following output: + +```text +┌───────────────StartDate─┬─Visits─┬─Users─┐ +│ 2022-11-03 03:27:11.000 │ 16 │ 3 │ +│ 2022-11-26 07:00:31.000 │ 5 │ 1 │ +└─────────────────────────┴────────┴───────┘ +``` + ## Related Content - Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states) From 513bb7ddbb2a0ab0dcab55d6637025b7fa36ab8b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Mar 2024 13:54:06 +0100 Subject: [PATCH 010/243] Minor --- src/Interpreters/Cache/EvictionCandidates.cpp | 9 ++++++++- src/Interpreters/Cache/EvictionCandidates.h | 2 +- src/Interpreters/Cache/SLRUFileCachePriority.cpp | 2 +- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index e24d671b66e..06125f8efc2 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -40,7 +40,8 @@ void EvictionCandidates::evict(FileCacheQueryLimit::QueryContext * query_context } std::vector EvictionCandidates::evictFromMemory( - FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock & lock) + FileCacheQueryLimit::QueryContext * query_context, + const CachePriorityGuard::Lock & lock) { return evictImpl(true, query_context, lock); } @@ -99,4 +100,10 @@ std::vector EvictionCandidates::evictImpl( } return evicted_paths; } + +void EvictionCandidates::insert(EvictionCandidates && other, const CachePriorityGuard::Lock &) +{ + candidates.insert(make_move_iterator(other.candidates.begin()), make_move_iterator(other.candidates.end())); +} + } diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index ec29692d8d3..afe6880d497 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -18,7 +18,7 @@ public: void add(LockedKey & locked_key, const FileSegmentMetadataPtr & candidate); - void add(const EvictionCandidates & other, const CachePriorityGuard::Lock &) { candidates.insert(other.candidates.begin(), other.candidates.end()); } + void insert(EvictionCandidates && other, const CachePriorityGuard::Lock &); void evict(FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock &); diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index a405c237d71..681ceef04ea 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -195,7 +195,7 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, max_candidates_to_evict ? max_candidates_to_evict - res.size() : 0, stat, lock); - res.add(res_add, lock); + res.insert(std::move(res_add), lock); return res; } From 071a8ff95f5656ab18433dc03b19fce12e5855ab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Mar 2024 18:55:46 +0100 Subject: [PATCH 011/243] less unformatted exceptions --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 34 +++++------ .../MergeTree/MergeTreeDataMergerMutator.h | 10 ++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 41 +++++++------ .../MergeTree/ReplicatedMergeTreeQueue.h | 8 +-- src/Storages/StorageMergeTree.cpp | 58 +++++++++---------- src/Storages/StorageMergeTree.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 20 +++---- 9 files changed, 91 insertions(+), 94 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9f3b48f88c..7a2ddc77724 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7773,11 +7773,11 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & return result; } -bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, String & out_reason) +bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, PreformattedMessage & out_reason) { if (left->getProjectionParts().size() != right->getProjectionParts().size()) { - out_reason = fmt::format( + out_reason = PreformattedMessage::create( "Parts have different number of projections: {} in part '{}' and {} in part '{}'", left->getProjectionParts().size(), left->name, @@ -7791,7 +7791,7 @@ bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const { if (!right->hasProjection(name)) { - out_reason = fmt::format( + out_reason = PreformattedMessage::create( "The part '{}' doesn't have projection '{}' while part '{}' does", right->name, name, left->name ); return false; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8305c7c6ce9..9081d384a26 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -418,7 +418,7 @@ public: static ReservationPtr tryReserveSpace(UInt64 expected_size, const IDataPartStorage & data_part_storage); static ReservationPtr reserveSpace(UInt64 expected_size, const IDataPartStorage & data_part_storage); - static bool partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, String & out_reason); + static bool partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, PreformattedMessage & out_reason); StoragePolicyPtr getStoragePolicy() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 53d49b51e8f..2d49e1df19b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -136,7 +136,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( const AllowedMergingPredicate & can_merge_callback, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, const PartitionIdsHint * partitions_hint) { MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn, partitions_hint); @@ -145,7 +145,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (data_parts.empty()) { - out_disable_reason = "There are no parts in the table"; + out_disable_reason = PreformattedMessage::create("There are no parts in the table"); return SelectPartsDecision::CANNOT_SELECT; } @@ -153,7 +153,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (info.parts_selected_precondition == 0) { - out_disable_reason = "No parts satisfy preconditions for merge"; + out_disable_reason = PreformattedMessage::create("No parts satisfy preconditions for merge"); return SelectPartsDecision::CANNOT_SELECT; } @@ -177,9 +177,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( /*optimize_skip_merged_partitions=*/true); } - if (!out_disable_reason.empty()) - out_disable_reason += ". "; - out_disable_reason += "There is no need to merge parts according to merge selector algorithm"; + if (!out_disable_reason.text.empty()) + out_disable_reason.text += ". "; + out_disable_reason.text += "There is no need to merge parts according to merge selector algorithm"; return SelectPartsDecision::CANNOT_SELECT; } @@ -196,7 +196,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart auto metadata_snapshot = data.getInMemoryMetadataPtr(); - String out_reason; + PreformattedMessage out_reason; MergeSelectingInfo info = getPossibleMergeRanges(data_parts, can_merge_callback, txn, out_reason); if (info.parts_selected_precondition == 0) @@ -223,7 +223,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart for (size_t i = 0; i < all_partition_ids.size(); ++i) { auto future_part = std::make_shared(); - String out_disable_reason; + PreformattedMessage out_disable_reason; /// This method should have been const, but something went wrong... it's const with dry_run = true auto status = const_cast(this)->selectPartsToMergeFromRanges( future_part, /*aggressive*/ false, max_total_size_to_merge, merge_with_ttl_allowed, @@ -232,7 +232,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart if (status == SelectPartsDecision::SELECTED) res.insert(all_partition_ids[i]); else - LOG_TEST(log, "Nothing to merge in partition {}: {}", all_partition_ids[i], out_disable_reason); + LOG_TEST(log, "Nothing to merge in partition {}: {}", all_partition_ids[i], out_disable_reason.text); } String best_partition_id_to_optimize = getBestPartitionToOptimizeEntire(info.partitions_info); @@ -331,7 +331,7 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo const MergeTreeData::DataPartsVector & data_parts, const AllowedMergingPredicate & can_merge_callback, const MergeTreeTransactionPtr & txn, - String & out_disable_reason) const + PreformattedMessage & out_disable_reason) const { MergeSelectingInfo res; @@ -444,7 +444,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( const StorageMetadataPtr & metadata_snapshot, const IMergeSelector::PartsRanges & parts_ranges, const time_t & current_time, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool dry_run) { const auto data_settings = data.getSettings(); @@ -515,7 +515,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( if (parts_to_merge.empty()) { - out_disable_reason = "Did not find any parts to merge (with usual merge selectors)"; + out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors)"); return SelectPartsDecision::CANNOT_SELECT; } } @@ -573,20 +573,20 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti bool final, const StorageMetadataPtr & metadata_snapshot, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool optimize_skip_merged_partitions) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); if (parts.empty()) { - out_disable_reason = "There are no parts inside partition"; + out_disable_reason = PreformattedMessage::create("There are no parts inside partition"); return SelectPartsDecision::CANNOT_SELECT; } if (!final && parts.size() == 1) { - out_disable_reason = "There is only one part inside partition"; + out_disable_reason = PreformattedMessage::create("There is only one part inside partition"); return SelectPartsDecision::CANNOT_SELECT; } @@ -595,7 +595,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti if (final && optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { - out_disable_reason = "Partition skipped due to optimize_skip_merged_partitions"; + out_disable_reason = PreformattedMessage::create("Partition skipped due to optimize_skip_merged_partitions"); return SelectPartsDecision::NOTHING_TO_MERGE; } @@ -636,7 +636,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti static_cast((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)); } - out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); + out_disable_reason = PreformattedMessage::create("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); return SelectPartsDecision::CANNOT_SELECT; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 669ee040af3..aad34bfb914 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -43,7 +43,7 @@ public: using AllowedMergingPredicate = std::function; + PreformattedMessage &)>; explicit MergeTreeDataMergerMutator(MergeTreeData & data_); @@ -92,7 +92,7 @@ public: const MergeTreeData::DataPartsVector & data_parts, const AllowedMergingPredicate & can_merge_callback, const MergeTreeTransactionPtr & txn, - String & out_disable_reason) const; + PreformattedMessage & out_disable_reason) const; /// The third step of selecting parts to merge: takes ranges that we can merge, and selects parts that we want to merge SelectPartsDecision selectPartsToMergeFromRanges( @@ -103,7 +103,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const IMergeSelector::PartsRanges & parts_ranges, const time_t & current_time, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool dry_run = false); String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; @@ -129,7 +129,7 @@ public: const AllowedMergingPredicate & can_merge, bool merge_with_ttl_allowed, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, const PartitionIdsHint * partitions_hint = nullptr); /** Select all the parts in the specified partition for merge, if possible. @@ -144,7 +144,7 @@ public: bool final, const StorageMetadataPtr & metadata_snapshot, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool optimize_skip_merged_partitions = false); /** Creates a task to merge parts. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 42f564f40da..d7168ff57be 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2266,7 +2266,7 @@ bool BaseMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, - String & out_reason) const + PreformattedMessage & out_reason) const { if (left) return canMergeTwoParts(left, right, out_reason); @@ -2278,7 +2278,7 @@ template bool BaseMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String & out_reason) const + PreformattedMessage & out_reason) const { /// A sketch of a proof of why this method actually works: /// @@ -2322,19 +2322,19 @@ bool BaseMergePredicate::canMergeTwoParts( { if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { - out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; + out_reason = PreformattedMessage::create("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { - out_reason = "Quorum insert for part " + part->name + " is currently in progress"; + out_reason = PreformattedMessage::create("Quorum insert for part {} is currently in progress", part->name); return false; } if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { - out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; + out_reason = PreformattedMessage::create("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } } @@ -2348,7 +2348,7 @@ bool BaseMergePredicate::canMergeTwoParts( { if (partition_ids_hint && !partition_ids_hint->contains(left->info.partition_id)) { - out_reason = fmt::format("Uncommitted block were not loaded for unexpected partition {}", left->info.partition_id); + out_reason = PreformattedMessage::create("Uncommitted block were not loaded for unexpected partition {}", left->info.partition_id); return false; } @@ -2360,8 +2360,7 @@ bool BaseMergePredicate::canMergeTwoParts( auto block_it = block_numbers.upper_bound(left_max_block); if (block_it != block_numbers.end() && *block_it < right_min_block) { - out_reason = "Block number " + toString(*block_it) + " is still being inserted between parts " - + left->name + " and " + right->name; + out_reason = PreformattedMessage::create("Block number {} is still being inserted between parts {} and {}", *block_it, left->name, right->name); return false; } } @@ -2380,7 +2379,7 @@ bool BaseMergePredicate::canMergeTwoParts( String containing_part = virtual_parts_->getContainingPart(part->info); if (containing_part != part->name) { - out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; + out_reason = PreformattedMessage::create("Part {} has already been assigned a merge into {}", part->name, containing_part); return false; } } @@ -2397,9 +2396,9 @@ bool BaseMergePredicate::canMergeTwoParts( Strings covered = virtual_parts_->getPartsCoveredBy(gap_part_info); if (!covered.empty()) { - out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() - + " to " + covered.back() + ") that are still not present or being processed by " - + " other background process on this replica between " + left->name + " and " + right->name; + out_reason = PreformattedMessage::create("There are {} parts (from {} to {}) " + "that are still not present or being processed by other background process " + "on this replica between {} and {}", covered.size(), covered.front(), covered.back(), left->name, right->name); return false; } } @@ -2415,8 +2414,8 @@ bool BaseMergePredicate::canMergeTwoParts( if (left_mutation_ver != right_mutation_ver) { - out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " - + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; + out_reason = PreformattedMessage::create("Current mutation versions of parts {} and {} differ: " + "{} and {} respectively", left->name, right->name, left_mutation_ver, right_mutation_ver); return false; } } @@ -2427,23 +2426,23 @@ bool BaseMergePredicate::canMergeTwoParts( template bool BaseMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, - String & out_reason) const + PreformattedMessage & out_reason) const { if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { - out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); + out_reason = PreformattedMessage::create("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { - out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); + out_reason = PreformattedMessage::create("Quorum insert for part {} is currently in progress", part->name); return false; } if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { - out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); + out_reason = PreformattedMessage::create("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } @@ -2458,7 +2457,7 @@ bool BaseMergePredicate::canMergeSinglePart( String containing_part = virtual_parts_->getContainingPart(part->info); if (containing_part != part->name) { - out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); + out_reason = PreformattedMessage::create("Part {} has already been assigned a merge into {}", part->name, containing_part); return false; } } @@ -2467,7 +2466,7 @@ bool BaseMergePredicate::canMergeSinglePart( } -bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String & out_reason) const +bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, PreformattedMessage & out_reason) const { std::lock_guard lock(queue.state_mutex); for (const auto & entry : queue.queue) @@ -2480,7 +2479,7 @@ bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const Mer if (part->info.isDisjoint(MergeTreePartInfo::fromPartName(part_name, queue.format_version))) continue; - out_reason = fmt::format("Part {} participates in REPLACE_RANGE {} ({})", part_name, entry->new_part_name, entry->znode_name); + out_reason = PreformattedMessage::create("Part {} participates in REPLACE_RANGE {} ({})", part_name, entry->new_part_name, entry->znode_name); return true; } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index b17e7819946..85f3aacc766 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -505,19 +505,19 @@ public: bool operator()(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction * txn, - String & out_reason) const; + PreformattedMessage & out_reason) const; /// Can we assign a merge with these two parts? /// (assuming that no merge was assigned after the predicate was constructed) /// If we can't and out_reason is not nullptr, set it to the reason why we can't merge. bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, - String & out_reason) const; + PreformattedMessage & out_reason) const; /// Can we assign a merge this part and some other part? /// For example a merge of a part and itself is needed for TTL. /// This predicate is checked for the first part of each range. - bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String & out_reason) const; + bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, PreformattedMessage & out_reason) const; CommittingBlocks getCommittingBlocks(zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, LoggerPtr log_); @@ -561,7 +561,7 @@ public: /// Returns true if part is needed for some REPLACE_RANGE entry. /// We should not drop part in this case, because replication queue may stuck without that part. - bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String & out_reason) const; + bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, PreformattedMessage & out_reason) const; /// Return nonempty optional of desired mutation version and alter version. /// If we have no alter (modify/drop) mutations in mutations queue, than we return biggest possible diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c87681a1418..c41943b269d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -933,7 +933,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( bool aggressive, const String & partition_id, bool final, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, TableLockHolder & /* table_lock_holder */, std::unique_lock & lock, const MergeTreeTransactionPtr & txn, @@ -951,7 +951,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( CurrentlyMergingPartsTaggerPtr merging_tagger; MergeList::EntryPtr merge_entry; - auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, String & disable_reason) -> bool + auto can_merge = [this, &lock](const DataPartPtr & left, const DataPartPtr & right, const MergeTreeTransaction * tx, PreformattedMessage & disable_reason) -> bool { if (tx) { @@ -960,7 +960,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if ((left && !left->version.isVisible(tx->getSnapshot(), Tx::EmptyTID)) || (right && !right->version.isVisible(tx->getSnapshot(), Tx::EmptyTID))) { - disable_reason = "Some part is not visible in transaction"; + disable_reason = PreformattedMessage::create("Some part is not visible in transaction"); return false; } @@ -968,7 +968,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if ((left && left->version.isRemovalTIDLocked()) || (right && right->version.isRemovalTIDLocked())) { - disable_reason = "Some part is locked for removal in another cuncurrent transaction"; + disable_reason = PreformattedMessage::create("Some part is locked for removal in another cuncurrent transaction"); return false; } } @@ -979,7 +979,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( { if (currently_merging_mutating_parts.contains(right)) { - disable_reason = "Some part currently in a merging or mutating process"; + disable_reason = PreformattedMessage::create("Some part currently in a merging or mutating process"); return false; } else @@ -988,13 +988,13 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (currently_merging_mutating_parts.contains(left) || currently_merging_mutating_parts.contains(right)) { - disable_reason = "Some part currently in a merging or mutating process"; + disable_reason = PreformattedMessage::create("Some part currently in a merging or mutating process"); return false; } if (getCurrentMutationVersion(left, lock) != getCurrentMutationVersion(right, lock)) { - disable_reason = "Some parts have different mutation version"; + disable_reason = PreformattedMessage::create("Some parts have different mutation version"); return false; } @@ -1004,7 +1004,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( auto max_possible_level = getMaxLevelInBetween(left, right); if (max_possible_level > std::max(left->info.level, right->info.level)) { - disable_reason = fmt::format("There is an outdated part in a gap between two active parts ({}, {}) with merge level {} higher than these active parts have", left->name, right->name, max_possible_level); + disable_reason = PreformattedMessage::create("There is an outdated part in a gap between two active parts ({}, {}) with merge level {} higher than these active parts have", left->name, right->name, max_possible_level); return false; } @@ -1013,11 +1013,11 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; - auto is_background_memory_usage_ok = [](String & disable_reason) -> bool + auto is_background_memory_usage_ok = [](PreformattedMessage & disable_reason) -> bool { if (canEnqueueBackgroundTask()) return true; - disable_reason = fmt::format("Current background tasks memory usage ({}) is more than the limit ({})", + disable_reason = PreformattedMessage::create("Current background tasks memory usage ({}) is more than the limit ({})", formatReadableSizeWithBinarySuffix(background_memory_tracker.get()), formatReadableSizeWithBinarySuffix(background_memory_tracker.getSoftLimit())); return false; @@ -1045,7 +1045,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( out_disable_reason); } else - out_disable_reason = "Current value of max_source_parts_size is zero"; + out_disable_reason = PreformattedMessage::create("Current value of max_source_parts_size is zero"); } } else @@ -1086,7 +1086,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for(lock, timeout)) { - out_disable_reason = fmt::format("Timeout ({} ms) while waiting for already running merges before running OPTIMIZE with FINAL", timeout_ms); + out_disable_reason = PreformattedMessage::create("Timeout ({} ms) while waiting for already running merges before running OPTIMIZE with FINAL", timeout_ms); break; } } @@ -1102,9 +1102,9 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (select_decision != SelectPartsDecision::SELECTED) { - if (!out_disable_reason.empty()) - out_disable_reason += ". "; - out_disable_reason += "Cannot select parts for optimization"; + if (!out_disable_reason.text.empty()) + out_disable_reason.text += ". "; + out_disable_reason.text += "Cannot select parts for optimization"; return {}; } @@ -1125,7 +1125,7 @@ bool StorageMergeTree::merge( const Names & deduplicate_by_columns, bool cleanup, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool optimize_skip_merged_partitions) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -1180,7 +1180,7 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p } MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( - const StorageMetadataPtr & metadata_snapshot, String & /* disable_reason */, TableLockHolder & /* table_lock_holder */, + const StorageMetadataPtr & metadata_snapshot, PreformattedMessage & /* disable_reason */, TableLockHolder & /* table_lock_holder */, std::unique_lock & /*currently_processing_in_background_mutex_lock*/) { if (current_mutations_by_version.empty()) @@ -1396,7 +1396,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (merger_mutator.merges_blocker.isCancelled()) return false; - String out_reason; + PreformattedMessage out_reason; merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, out_reason, shared_lock, lock, txn); if (!merge_entry && !current_mutations_by_version.empty()) @@ -1559,14 +1559,12 @@ bool StorageMergeTree::optimize( auto txn = local_context->getCurrentTransaction(); - String disable_reason; + PreformattedMessage disable_reason; if (!partition && final) { if (cleanup && this->merging_params.mode != MergingParams::Mode::Replacing) { - constexpr const char * message = "Cannot OPTIMIZE with CLEANUP table: {}"; - disable_reason = "only ReplacingMergeTree can be CLEANUP"; - throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); + throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, "Cannot OPTIMIZE with CLEANUP table: only ReplacingMergeTree can be CLEANUP"); } if (cleanup && !getSettings()->allow_experimental_replacing_merge_with_cleanup) @@ -1592,12 +1590,12 @@ bool StorageMergeTree::optimize( local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; - if (disable_reason.empty()) - disable_reason = "unknown reason"; - LOG_INFO(log, message, disable_reason); + if (disable_reason.text.empty()) + disable_reason = PreformattedMessage::create("unknown reason"); + LOG_INFO(log, message, disable_reason.text); if (local_context->getSettingsRef().optimize_throw_if_noop) - throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); + throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason.text); return false; } } @@ -1620,12 +1618,12 @@ bool StorageMergeTree::optimize( local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; - if (disable_reason.empty()) - disable_reason = "unknown reason"; - LOG_INFO(log, message, disable_reason); + if (disable_reason.text.empty()) + disable_reason = PreformattedMessage::create("unknown reason"); + LOG_INFO(log, message, disable_reason.text); if (local_context->getSettingsRef().optimize_throw_if_noop) - throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason); + throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, message, disable_reason.text); return false; } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c384a391291..02217e6d138 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -175,7 +175,7 @@ private: const Names & deduplicate_by_columns, bool cleanup, const MergeTreeTransactionPtr & txn, - String & out_disable_reason, + PreformattedMessage & out_disable_reason, bool optimize_skip_merged_partitions = false); void renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction); @@ -202,7 +202,7 @@ private: bool aggressive, const String & partition_id, bool final, - String & disable_reason, + PreformattedMessage & disable_reason, TableLockHolder & table_lock_holder, std::unique_lock & lock, const MergeTreeTransactionPtr & txn, @@ -211,7 +211,7 @@ private: MergeMutateSelectedEntryPtr selectPartsToMutate( - const StorageMetadataPtr & metadata_snapshot, String & disable_reason, + const StorageMetadataPtr & metadata_snapshot, PreformattedMessage & disable_reason, TableLockHolder & table_lock_holder, std::unique_lock & currently_processing_in_background_mutex_lock); /// For current mutations queue, returns maximum version of mutation for a part, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c41403e312b..2feaca6ba48 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3791,7 +3791,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() merge_pred.emplace(queue.getMergePredicate(zookeeper, partitions_to_merge_in)); } - String out_reason; + PreformattedMessage out_reason; if (can_assign_merge && merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, *merge_pred, merge_with_ttl_allowed, NO_TRANSACTION_PTR, out_reason, &partitions_to_merge_in) == SelectPartsDecision::SELECTED) @@ -5773,7 +5773,7 @@ bool StorageReplicatedMergeTree::optimize( future_merged_part->uuid = UUIDHelpers::generateV4(); constexpr const char * unknown_disable_reason = "unknown reason"; - String disable_reason = unknown_disable_reason; + PreformattedMessage disable_reason = PreformattedMessage::create(unknown_disable_reason); SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; if (partition_id.empty()) @@ -5796,10 +5796,10 @@ bool StorageReplicatedMergeTree::optimize( if (select_decision != SelectPartsDecision::SELECTED) { constexpr const char * message_fmt = "Cannot select parts for optimization: {}"; - assert(disable_reason != unknown_disable_reason); + assert(disable_reason.text != unknown_disable_reason); if (!partition_id.empty()) - disable_reason += fmt::format(" (in partition {})", partition_id); - return handle_noop(message_fmt, disable_reason); + disable_reason.text += fmt::format(" (in partition {})", partition_id); + return handle_noop(message_fmt, disable_reason.text); } ReplicatedMergeTreeLogEntryData merge_entry; @@ -8465,9 +8465,9 @@ void StorageReplicatedMergeTree::movePartitionToShard( } /// canMergeSinglePart is overlapping with dropPart, let's try to use the same code. - String out_reason; + PreformattedMessage out_reason; if (!merge_pred.canMergeSinglePart(part, out_reason)) - throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Part is busy, reason: {}", out_reason); + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Part is busy, reason: {}", out_reason.text); } { @@ -8725,18 +8725,18 @@ bool StorageReplicatedMergeTree::dropPartImpl( /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. - String out_reason; + PreformattedMessage out_reason; if (!merge_pred.canMergeSinglePart(part, out_reason)) { if (throw_if_noop) - throw Exception::createDeprecated(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + throw Exception(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); return false; } if (merge_pred.partParticipatesInReplaceRange(part, out_reason)) { if (throw_if_noop) - throw Exception::createDeprecated(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + throw Exception(out_reason, ErrorCodes::PART_IS_TEMPORARILY_LOCKED); return false; } From e32497345013da3ccaf04467d5e52318c82d837f Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 12:36:11 +0100 Subject: [PATCH 012/243] Standardize function formatting for MultiSearchAllPositionsXYZ and MultiSearchAnyXYZ functions --- .../functions/string-search-functions.md | 221 +++++++++++++++++- 1 file changed, 219 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 22f879c62ae..c4b5fb7c1f5 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -178,11 +178,106 @@ Result: │ [0,13,0] │ └───────────────────────────────────────────────────────────────────┘ ``` +## multiSearchAllPositionsCaseInsensitive + +Like [multiSearchAllPositions](#multisearchallpositions) but ignores case. + +**Syntax** + +```sql +multiSearchAllPositionsCaseInsensitive(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Array of the starting position in bytes and counting from 1 (if the substring was found). +- 0 if the substring was not found. + +**Example** + +Query: + +```sql +SELECT multiSearchAllPositionsCaseInsensitive('ClickHouse',['c','h']); +``` + +```response +["1","6"] +``` ## multiSearchAllPositionsUTF8 Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings. +**Syntax** + +```sql +multiSearchAllPositionsUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Array of the starting position in bytes and counting from 1 (if the substring was found). +- 0 if the substring was not found. + +**Example** + +Given `ClickHouse` as a UTF-8 string, find the positions of `C` ('\x43') and `H` ('\x48'). + +Query: + +```sql +SELECT multiSearchAllPositionsUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']); +``` + +```response +["1","6"] +``` + +## multiSearchAllPositionsCaseInsensitiveUTF8 + +Like [multiSearchAllPositionsUTF8](#multisearchallpositionsutf8) but ignores case. + +**Syntax** + +```sql +multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Array of the starting position in bytes and counting from 1 (if the substring was found). +- 0 if the substring was not found. + +**Example** + +Given `ClickHouse` as a UTF-8 string, find the positions of `c` (`\x63`) and `h` (`\x68`). + +Query: + +```sql +SELECT multiSearchAllPositionsCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x63','\x68']); +``` + +```response +["1","6"] +``` + ## multiSearchFirstPosition Like `position` but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. @@ -211,12 +306,134 @@ multiSearchFirstIndex(haystack, \[needle1, needle2, …, n Returns 1, if at least one string needlei matches the string `haystack` and 0 otherwise. -Functions `multiSearchAnyCaseInsensitive`, `multiSearchAnyUTF8` and `multiSearchAnyCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchAnyCaseInsensitive`](#multiSearchAnyCaseInsensitive), [`multiSearchAnyUTF8`](#multiSearchAnyUTF8) and []`multiSearchAnyCaseInsensitiveUTF8`](#multiSearchAnyCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** ```sql -multiSearchAny(haystack, \[needle1, needle2, …, needlen\]) +multiSearchAny(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- 1, if there was at least one match. +- 0, if there was not at least one match. + +**Example** + +Query: + +```sql +SELECT multiSearchAny('ClickHouse',['C','H']); +``` + +```response +1 +``` + +## multiSearchAnyCaseInsensitive {#multiSearchAnyCaseInsensitive} + +Like [multiSearchAny](#multisearchany) but ignores case. + +**Syntax** + +```sql +multiSearchAnyCaseInsensitive(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- 1, if there was at least one case-insensitive match. +- 0, if there was not at least one case-insensitive match. + +**Example** + +Query: + +```sql +SELECT multiSearchAnyCaseInsensitive('ClickHouse',['c','h']); +``` + +```response +1 +``` + +## multiSearchAnyUTF8 {#multiSearchAnyUTF8} + +Like [multiSearchAny](#multisearchany) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings. + +*Syntax** + +```sql +multiSearchAnyUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- 1, if there was at least one match. +- 0, if there was not at least one match. + +**Example** + +Given `ClickHouse` as a UTF-8 string, check if there are any `C` ('\x43') or `H` ('\x48') letters in the word. + +Query: + +```sql +SELECT multiSearchAnyUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']); +``` + +```response +1 +``` + +## multiSearchAnyCaseInsensitiveUTF8 {#multiSearchAnyCaseInsensitiveUTF8} + +Like [multiSearchAnyUTF8](#multiSearchAnyUTF8) but ignores case. + +*Syntax** + +```sql +multiSearchAnyCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- 1, if there was at least one case-insensitive match. +- 0, if there was not at least one case-insensitive match. + +**Example** + +Given `ClickHouse` as a UTF-8 string, check if there is any letter `h`(`\x68`) in the word, ignoring case. + +Query: + +```sql +SELECT multiSearchAnyCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x68']); +``` + +```response +1 ``` ## match {#match} From 19b90d8348aec4dc4fc038c89155a169ba4d6224 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 16:28:59 +0100 Subject: [PATCH 013/243] Resolve conflicts after merge, add system.filesystem_cache_settings --- src/Interpreters/Cache/EvictionCandidates.cpp | 23 +++++- src/Interpreters/Cache/EvictionCandidates.h | 14 ++++ src/Interpreters/Cache/FileCache.cpp | 23 ++++-- .../Cache/LRUFileCachePriority.cpp | 71 +++++++++--------- src/Interpreters/Cache/Metadata.cpp | 2 +- .../Cache/SLRUFileCachePriority.cpp | 2 +- .../StorageSystemFilesystemCacheSettings.cpp | 72 +++++++++++++++++++ .../StorageSystemFilesystemCacheSettings.h | 22 ++++++ src/Storages/System/attachSystemTables.cpp | 2 + ...resize_filesystem_cache_hardcore.reference | 7 ++ ...ically_resize_filesystem_cache_hardcore.sh | 46 ++++++++++++ 11 files changed, 237 insertions(+), 47 deletions(-) create mode 100644 src/Storages/System/StorageSystemFilesystemCacheSettings.cpp create mode 100644 src/Storages/System/StorageSystemFilesystemCacheSettings.h create mode 100644 tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference create mode 100755 tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index cb15af72704..080ef675918 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -57,13 +57,25 @@ void EvictionCandidates::add( ++candidates_size; } +void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & lock) +{ + for (const auto & [key, key_candidates] : candidates) + { + for (const auto & candidate : key_candidates.candidates) + candidate->getQueueIterator()->remove(lock); + } + invalidated_queue_entries = true; +} + void EvictionCandidates::evict() { if (candidates.empty()) return; auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); - queue_entries_to_invalidate.reserve(candidates_size); + + if (!invalidated_queue_entries) + queue_entries_to_invalidate.reserve(candidates_size); for (auto & [key, key_candidates] : candidates) { @@ -111,7 +123,9 @@ void EvictionCandidates::evict() /// it was freed in favour of some reserver, so we can make it visibly /// free only for that particular reserver. - queue_entries_to_invalidate.push_back(iterator); + if (!invalidated_queue_entries) + queue_entries_to_invalidate.push_back(iterator); + key_candidates.candidates.pop_back(); } } @@ -165,4 +179,9 @@ void EvictionCandidates::setSpaceHolder( hold_space = std::make_unique(size, elements, priority, lock); } +void EvictionCandidates::insert(EvictionCandidates && other, const CachePriorityGuard::Lock &) +{ + candidates.insert(make_move_iterator(other.candidates.begin()), make_move_iterator(other.candidates.end())); +} + } diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 2745d508a5d..571010a14bc 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -9,6 +9,13 @@ class EvictionCandidates public: using FinalizeEvictionFunc = std::function; + EvictionCandidates() = default; + EvictionCandidates(EvictionCandidates && other) noexcept + { + candidates = std::move(other.candidates); + candidates_size = std::move(other.candidates_size); + queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate); + } ~EvictionCandidates(); void add( @@ -16,8 +23,12 @@ public: LockedKey & locked_key, const CachePriorityGuard::Lock &); + void insert(EvictionCandidates && other, const CachePriorityGuard::Lock &); + void evict(); + void removeQueueEntries(const CachePriorityGuard::Lock &); + void onFinalize(FinalizeEvictionFunc && func) { on_finalize.emplace_back(std::move(func)); } void finalize( @@ -47,7 +58,10 @@ private: size_t candidates_size = 0; std::vector on_finalize; + std::vector queue_entries_to_invalidate; + bool invalidated_queue_entries = false; + IFileCachePriority::HoldSpacePtr hold_space; }; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e845559e8a7..75e199c544b 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1383,7 +1383,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { - std::vector evicted_paths; + std::optional eviction_candidates; { cache_is_being_resized.store(true, std::memory_order_relaxed); SCOPE_EXIT({ @@ -1391,18 +1391,27 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, }); auto cache_lock = lockCache(); - FileCacheReserveStat stat; - auto eviction_candidates = main_priority->collectCandidatesForEviction( - new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock); - evicted_paths = eviction_candidates.evictFromMemory(nullptr, cache_lock); + FileCacheReserveStat stat; + eviction_candidates.emplace(main_priority->collectCandidatesForEviction( + new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock)); + + eviction_candidates->removeQueueEntries(cache_lock); main_priority->modifySizeLimits( new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); } - for (const auto & path : evicted_paths) - fs::remove(path); + try + { + eviction_candidates->evict(); + } + catch (...) + { + auto cache_lock = lockCache(); + eviction_candidates->finalize(nullptr, cache_lock); + throw; + } LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", actual_settings.max_size, new_settings.max_size, diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 4a80bce7658..d1c46691c37 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -279,10 +279,42 @@ bool LRUFileCachePriority::collectCandidatesForEviction( auto can_fit = [&] { - return canFit(size, 1, stat.stat.releasable_size, stat.stat.releasable_count, lock); + return canFit(size, 1, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock); }; + iterateForEviction(res, stat, can_fit, lock); - return can_fit(); + if (can_fit()) + { + /// As eviction is done without a cache priority lock, + /// then if some space was partially available and some needed + /// to be freed via eviction, we need to make sure that this + /// partially available space is still available + /// after we finish with eviction for non-available space. + /// So we create a space holder for the currently available part + /// of the required space for the duration of eviction of the other + /// currently non-available part of the space. + + const size_t hold_size = size > stat.total_stat.releasable_size + ? size - stat.total_stat.releasable_size + : 0; + + const size_t hold_elements = elements > stat.total_stat.releasable_count + ? elements - stat.total_stat.releasable_count + : 0; + + if (hold_size || hold_elements) + res.setSpaceHolder(hold_size, hold_elements, *this, lock); + + // LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). " + // "Took hold of size {} and elements {}", + // res.size(), stat.total_stat.releasable_size, hold_size, hold_elements); + + return true; + } + else + { + return false; + } } EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( @@ -295,7 +327,7 @@ EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( EvictionCandidates res; auto stop_condition = [&, this]() { - return canFit(0, 0, stat.stat.releasable_size, stat.stat.releasable_count, + return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock, &desired_size, &desired_elements_count) || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); }; @@ -334,39 +366,6 @@ void LRUFileCachePriority::iterateForEviction( { return stop_condition() ? IterationResult::BREAK : iterate_func(locked_key, segment_metadata); }, lock); - - if (can_fit()) - { - /// As eviction is done without a cache priority lock, - /// then if some space was partially available and some needed - /// to be freed via eviction, we need to make sure that this - /// partially available space is still available - /// after we finish with eviction for non-available space. - /// So we create a space holder for the currently available part - /// of the required space for the duration of eviction of the other - /// currently non-available part of the space. - - const size_t hold_size = size > stat.total_stat.releasable_size - ? size - stat.total_stat.releasable_size - : 0; - - const size_t hold_elements = elements > stat.total_stat.releasable_count - ? elements - stat.total_stat.releasable_count - : 0; - - if (hold_size || hold_elements) - res.setSpaceHolder(hold_size, hold_elements, *this, lock); - - // LOG_TEST(log, "Collected {} candidates for eviction (total size: {}). " - // "Took hold of size {} and elements {}", - // res.size(), stat.total_stat.releasable_size, hold_size, hold_elements); - - return true; - } - else - { - return false; - } } LRUFileCachePriority::LRUIterator LRUFileCachePriority::move( diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 082c33032f2..26611f02379 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -941,7 +941,7 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( file_segment->detach(segment_lock, *this); - if (!remove_only_metadata) + // if (!remove_only_metadata) { try { diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index df4d4276e89..79ca489cea2 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -263,7 +263,7 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); - chassert(res.size() == stat.stat.releasable_count); + chassert(res.size() == stat.total_stat.releasable_count); if (max_candidates_to_evict && res.size() == max_candidates_to_evict) return res; diff --git a/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp b/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp new file mode 100644 index 00000000000..8915032baf7 --- /dev/null +++ b/src/Storages/System/StorageSystemFilesystemCacheSettings.cpp @@ -0,0 +1,72 @@ +#include "StorageSystemFilesystemCacheSettings.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemFilesystemCacheSettings::getColumnsDescription() +{ + return ColumnsDescription + { + {"cache_name", std::make_shared(), "Name of the cache object"}, + {"path", std::make_shared(), "Cache directory"}, + {"max_size", std::make_shared(), "Cache size limit by the number of bytes"}, + {"max_elements", std::make_shared(), "Cache size limit by the number of elements"}, + {"current_size", std::make_shared(), "Current cache size by the number of bytes"}, + {"current_elements", std::make_shared(), "Current cache size by the number of elements"}, + {"max_file_segment_size", std::make_shared(), "Maximum allowed file segment size"}, + {"boundary_alignment", std::make_shared(), "Boundary alignment of file segments"}, + {"cache_on_write_operations", std::make_shared(), "Write-through cache enablemenet setting"}, + {"cache_hits_threshold", std::make_shared(), "Cache hits threshold enablemenet setting"}, + {"background_download_threads", std::make_shared(), "Number of background download threads"}, + {"background_download_queue_size_limit", std::make_shared(), "Queue size limit for background download"}, + {"load_metadata_threads", std::make_shared(), "Number of load metadata threads"}, + {"enable_bypass_cache_threshold", std::make_shared(), "Bypass cache threshold limit enablement setting"}, + }; +} + +StorageSystemFilesystemCacheSettings::StorageSystemFilesystemCacheSettings(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_, getColumnsDescription()) +{ +} + +void StorageSystemFilesystemCacheSettings::fillData( + MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + context->checkAccess(AccessType::SHOW_FILESYSTEM_CACHES); + + auto caches = FileCacheFactory::instance().getAll(); + + for (const auto & [cache_name, cache_data] : caches) + { + const auto & settings = cache_data->getSettings(); + const auto & cache = cache_data->cache; + + size_t i = 0; + res_columns[i++]->insert(cache_name); + res_columns[i++]->insert(settings.base_path); + res_columns[i++]->insert(settings.max_size); + res_columns[i++]->insert(settings.max_elements); + res_columns[i++]->insert(cache->getUsedCacheSize()); + res_columns[i++]->insert(cache->getFileSegmentsNum()); + res_columns[i++]->insert(settings.max_file_segment_size); + res_columns[i++]->insert(settings.boundary_alignment); + res_columns[i++]->insert(settings.cache_on_write_operations); + res_columns[i++]->insert(settings.cache_hits_threshold); + res_columns[i++]->insert(settings.background_download_threads); + res_columns[i++]->insert(settings.background_download_queue_size_limit); + res_columns[i++]->insert(settings.load_metadata_threads); + res_columns[i++]->insert(settings.enable_bypass_cache_with_threshold); + } +} + +} diff --git a/src/Storages/System/StorageSystemFilesystemCacheSettings.h b/src/Storages/System/StorageSystemFilesystemCacheSettings.h new file mode 100644 index 00000000000..59a123c32c1 --- /dev/null +++ b/src/Storages/System/StorageSystemFilesystemCacheSettings.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageSystemFilesystemCacheSettings final : public IStorageSystemOneBlock +{ +public: + explicit StorageSystemFilesystemCacheSettings(const StorageID & table_id_); + + std::string getName() const override { return "SystemFilesystemCacheSettings"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + void fillData(MutableColumns & res_columns, ContextPtr, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index cd8be60e342..6ff86b26ca9 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -78,6 +78,7 @@ #include #include #include +#include #include #include #include @@ -213,6 +214,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "part_moves_between_shards", "Contains information about parts which are currently in a process of moving between shards and their progress."); attach(context, system_database, "asynchronous_inserts", "Contains information about pending asynchronous inserts in queue in server's memory."); attachNoDescription(context, system_database, "filesystem_cache", "Contains information about all entries inside filesystem cache for remote objects."); + attachNoDescription(context, system_database, "filesystem_cache_settings", "Contains information about all filesystem cache settings"); attachNoDescription(context, system_database, "query_cache", "Contains information about all entries inside query cache in server's memory."); attachNoDescription(context, system_database, "remote_data_paths", "Contains a mapping from a filename on local filesystem to a blob name inside object storage."); attach(context, system_database, "certificates", "Contains information about available certificates and their sources."); diff --git a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference new file mode 100644 index 00000000000..fcb49fa9945 --- /dev/null +++ b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference @@ -0,0 +1,7 @@ +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh new file mode 100755 index 00000000000..79c43048b89 --- /dev/null +++ b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +disk_name="s3_cache" + +$CLICKHOUSE_CLIENT -nm --query " +DROP TABLE IF EXISTS test; +CREATE TABLE test (a String) engine=MergeTree() ORDER BY tuple() SETTINGS disk = '$disk_name'; +INSERT INTO test SELECT randomString(1000); +" + +$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + +prev_max_size=$($CLICKHOUSE_CLIENT --query "SELECT max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'") +$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" + +config_path=/etc/clickhouse-server/config.d/storage_conf.xml +config_path_tmp=$config_path.tmp + +new_max_size=$($CLICKHOUSE_CLIENT --query "SELECT divide(max_size, 2) FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name'") +sed -i "s|$prev_max_size<\/max_size>|$new_max_size<\/max_size>|" $config_path + +# echo $prev_max_size +# echo $new_max_size + +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" + +$CLICKHOUSE_CLIENT --query "SELECT max_size == $new_max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" +$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" +$CLICKHOUSE_CLIENT --query "SELECT current_size <= max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" + +sed -i "s|$new_max_size<\/max_size>|$prev_max_size<\/max_size>|" $config_path + +$CLICKHOUSE_CLIENT -nm --query " +set send_logs_level='fatal'; +SYSTEM RELOAD CONFIG" + +$CLICKHOUSE_CLIENT --query "SELECT max_size == $prev_max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" +$CLICKHOUSE_CLIENT --query "SELECT current_size > 0 FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" +$CLICKHOUSE_CLIENT --query "SELECT current_size <= max_size FROM system.filesystem_cache_settings WHERE cache_name = '$disk_name' FORMAT TabSeparated" From 1d453af6ff2f49fb2dc3beabbcdc2e00fe21757f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 16:47:22 +0100 Subject: [PATCH 014/243] Fix style check --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index d1c46691c37..78ece5a3124 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -16,9 +16,6 @@ namespace ProfileEvents { extern const Event FilesystemCacheEvictionSkippedFileSegments; extern const Event FilesystemCacheEvictionTries; - extern const Event FilesystemCacheEvictMicroseconds; - extern const Event FilesystemCacheEvictedBytes; - extern const Event FilesystemCacheEvictedFileSegments; extern const Event FilesystemCacheEvictionSkippedEvictingFileSegments; } From ef826ee881b2c260999845f86820389c4378e2ab Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 19:47:50 +0100 Subject: [PATCH 015/243] Standardize format of multiSearchFirstIndexXYZ functions --- .../functions/string-search-functions.md | 125 +++++++++++++++++- 1 file changed, 123 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index c4b5fb7c1f5..c067f9010fa 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -294,12 +294,133 @@ multiSearchFirstPosition(haystack, \[needle1, needle2, … Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. -Functions `multiSearchFirstIndexCaseInsensitive`, `multiSearchFirstIndexUTF8` and `multiSearchFirstIndexCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstIndexCaseInsensitive`](#multiSearchFirstIndexCaseInsensitive), [`multiSearchFirstIndexUTF8`](#multiSearchFirstIndexUTF8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multiSearchFirstIndexCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** ```sql -multiSearchFirstIndex(haystack, \[needle1, needle2, …, needlen\]) +multiSearchFirstIndex(haystack, [needle1, needle2, ..., needleN]) +``` +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- index (starting from 1) of the leftmost found needle. +- 0, if there was no match. + +**Example** + +Query: + +```sql +SELECT multiSearchFirstIndex('Hello World',['World','Hello']); +``` + +```response +1 +``` + +## multiSearchFirstIndexCaseInsensitive {#multiSearchFirstIndexCaseInsensitive} + +Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Ignores case. + +**Syntax** + +```sql +multiSearchFirstIndexCaseInsensitive(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- index (starting from 1) of the leftmost found needle. +- 0, if there was no match. + +**Example** + +Query: + +```sql +SELECT multiSearchFirstIndexCaseInsensitive('hElLo WoRlD',['World','Hello']); +``` + +```response +1 +``` + +## multiSearchFirstIndexUTF8 {#multiSearchFirstIndexUTF8} + +Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings. + +**Syntax** + +```sql +multiSearchFirstIndexUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- index (starting from 1) of the leftmost found needle. +- 0, if there was no match. + +**Example** + +Given `Hello World` as a UTF-8 string, find the first index of UTF-8 strings `Hello` and `World`. + +Query: + +```sql +SELECT multiSearchFirstIndexUTF8('\x48\x65\x6c\x6c\x6f\x20\x57\x6f\x72\x6c\x64',['\x57\x6f\x72\x6c\x64','\x48\x65\x6c\x6c\x6f']); +``` + +```response +1 +``` + +## multiSearchFirstIndexCaseInsensitiveUTF8 {#multiSearchFirstIndexCaseInsensitiveUTF8} + +Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings. Ignores case. + +**Syntax** + +```sql +multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- index (starting from 1) of the leftmost found needle. +- 0, if there was no match. + +**Example** + +Given `HELLO WORLD` as a UTF-8 string, find the first index of UTF-8 strings `hello` and `world`. + +Query: + +```sql +SELECT multiSearchFirstIndexCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['\x68\x65\x6c\x6c\x6f','\x77\x6f\x72\x6c\x64']); +``` + +```response +1 ``` ## multiSearchAny {#multisearchany} From a97a3196b45fa5cb86f7d95b4869318864a18562 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 20:52:55 +0100 Subject: [PATCH 016/243] Standardize formatting of multisearchFirstPositionXYZ functions --- .../functions/string-search-functions.md | 128 +++++++++++++++++- 1 file changed, 125 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index c067f9010fa..232397ce592 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -278,16 +278,138 @@ SELECT multiSearchAllPositionsCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\ ["1","6"] ``` -## multiSearchFirstPosition +## multiSearchFirstPosition {#multiSearchFirstPosition} Like `position` but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. -Functions `multiSearchFirstPositionCaseInsensitive`, `multiSearchFirstPositionUTF8` and `multiSearchFirstPositionCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function. +Functions [`multiSearchFirstPositionCaseInsensitive`](#multiSearchFirstPositionCaseInsensitive), [`multiSearchFirstPositionUTF8`](#multiSearchFirstPositionUTF8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multiSearchFirstPositionCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. **Syntax** ```sql -multiSearchFirstPosition(haystack, \[needle1, needle2, …, needlen\]) +multiSearchFirstPosition(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings. +- 0, if there was no match. + +**Example** + +Query: + +```sql +SELECT multiSearchFirstPosition('Hello World',['llo', 'Wor', 'ld']); +``` + +```response +3 +``` + +## multiSearchFirstPositionCaseInsensitive + +Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but ignores case. + +**Syntax** + +```sql +multiSearchFirstPositionCaseInsensitive(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Array of substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings. +- 0, if there was no match. + +**Example** + +Query: + +```sql +SELECT multiSearchFirstPositionCaseInsensitive('HELLO WORLD',['wor', 'ld', 'ello']); +``` + +```response +2 +``` + +## multiSearchFirstPositionUTF8 {#multiSearchFirstPositionUTF8} + +Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings. + +**Syntax** + +```sql +multiSearchFirstPositionUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings. +- 0, if there was no match. + +**Example** + +Find the leftmost offset in UTF-8 string `hello world` which matches any of the given needles. + +Query: + +```sql +SELECT multiSearchFirstPositionUTF8('\x68\x65\x6c\x6c\x6f\x20\x77\x6f\x72\x6c\x64',['wor', 'ld', 'ello']); +``` + +```response +2 +``` + +## multiSearchFirstPositionCaseInsensitiveUTF8 {#multiSearchFirstPositionCaseInsensitiveUTF8} + +Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. + +**Syntax** + +```sql +multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN]) +``` + +**Parameters** + +- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) + +**Returned value** + +- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings, ignoring case. +- 0, if there was no match. + +**Example** + +Find the leftmost offset in UTF-8 string `HELLO WORLD` which matches any of the given needles. + +Query: + +```sql +SELECT multiSearchFirstPositionCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['wor', 'ld', 'ello']); +``` + +```response +2 ``` ## multiSearchFirstIndex From 89afca881f81142481b31682882fdb2e1482f08c Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 21:06:17 +0100 Subject: [PATCH 017/243] Minor formatting changes to multipleSearchAllPositions --- docs/en/sql-reference/functions/string-search-functions.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 232397ce592..1e564eb4741 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -159,11 +159,12 @@ multiSearchAllPositions(haystack, [needle1, needle2, ..., needleN]) **Arguments** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. Array +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned values** -- Array of the starting position in bytes and counting from 1 (if the substring was found) or 0 (if the substring was not found) +- Array of the starting position in bytes and counting from 1 (if the substring was found). +- 0, if the substring was not found. **Example** From cc35db1cef95a57c458d2606f517b0f43b91c8e4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 21:20:33 +0100 Subject: [PATCH 018/243] Finor fixes --- .../functions/string-search-functions.md | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 1e564eb4741..9430fb0f728 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -192,7 +192,7 @@ multiSearchAllPositionsCaseInsensitive(haystack, [needle1, needle2, ..., needleN **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -213,7 +213,7 @@ SELECT multiSearchAllPositionsCaseInsensitive('ClickHouse',['c','h']); ## multiSearchAllPositionsUTF8 -Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings. +Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. **Syntax** @@ -224,7 +224,7 @@ multiSearchAllPositionsUTF8(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -233,7 +233,7 @@ multiSearchAllPositionsUTF8(haystack, [needle1, needle2, ..., needleN]) **Example** -Given `ClickHouse` as a UTF-8 string, find the positions of `C` ('\x43') and `H` ('\x48'). +Given `ClickHouse` as a UTF-8 string, find the positions of `C` (`\x43`) and `H` (`\x48`). Query: @@ -258,7 +258,7 @@ multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., nee **Parameters** - `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -279,9 +279,9 @@ SELECT multiSearchAllPositionsCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\ ["1","6"] ``` -## multiSearchFirstPosition {#multiSearchFirstPosition} +## multiSearchFirstPosition -Like `position` but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. +Like [`position`](#position) but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings. Functions [`multiSearchFirstPositionCaseInsensitive`](#multiSearchFirstPositionCaseInsensitive), [`multiSearchFirstPositionUTF8`](#multiSearchFirstPositionUTF8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multiSearchFirstPositionCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function. @@ -294,7 +294,7 @@ multiSearchFirstPosition(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -326,7 +326,7 @@ multiSearchFirstPositionCaseInsensitive(haystack, [needle1, needle2, ..., needle **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Array of substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Array of substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -345,7 +345,7 @@ SELECT multiSearchFirstPositionCaseInsensitive('HELLO WORLD',['wor', 'ld', 'ello 2 ``` -## multiSearchFirstPositionUTF8 {#multiSearchFirstPositionUTF8} +## multiSearchFirstPositionUTF8 Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings. @@ -358,7 +358,7 @@ multiSearchFirstPositionUTF8(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -379,7 +379,7 @@ SELECT multiSearchFirstPositionUTF8('\x68\x65\x6c\x6c\x6f\x20\x77\x6f\x72\x6c\x6 2 ``` -## multiSearchFirstPositionCaseInsensitiveUTF8 {#multiSearchFirstPositionCaseInsensitiveUTF8} +## multiSearchFirstPositionCaseInsensitiveUTF8 Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case. @@ -427,7 +427,7 @@ multiSearchFirstIndex(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -446,7 +446,7 @@ SELECT multiSearchFirstIndex('Hello World',['World','Hello']); 1 ``` -## multiSearchFirstIndexCaseInsensitive {#multiSearchFirstIndexCaseInsensitive} +## multiSearchFirstIndexCaseInsensitive Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Ignores case. @@ -459,7 +459,7 @@ multiSearchFirstIndexCaseInsensitive(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -478,7 +478,7 @@ SELECT multiSearchFirstIndexCaseInsensitive('hElLo WoRlD',['World','Hello']); 1 ``` -## multiSearchFirstIndexUTF8 {#multiSearchFirstIndexUTF8} +## multiSearchFirstIndexUTF8 Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings. @@ -512,7 +512,7 @@ SELECT multiSearchFirstIndexUTF8('\x48\x65\x6c\x6c\x6f\x20\x57\x6f\x72\x6c\x64', 1 ``` -## multiSearchFirstIndexCaseInsensitiveUTF8 {#multiSearchFirstIndexCaseInsensitiveUTF8} +## multiSearchFirstIndexCaseInsensitiveUTF8 Returns the index `i` (starting from 1) of the leftmost found needlei in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings. Ignores case. @@ -525,7 +525,7 @@ multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needl **Parameters** - `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -546,7 +546,7 @@ SELECT multiSearchFirstIndexCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4 1 ``` -## multiSearchAny {#multisearchany} +## multiSearchAny Returns 1, if at least one string needlei matches the string `haystack` and 0 otherwise. @@ -561,7 +561,7 @@ multiSearchAny(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -580,7 +580,7 @@ SELECT multiSearchAny('ClickHouse',['C','H']); 1 ``` -## multiSearchAnyCaseInsensitive {#multiSearchAnyCaseInsensitive} +## multiSearchAnyCaseInsensitive Like [multiSearchAny](#multisearchany) but ignores case. @@ -612,9 +612,9 @@ SELECT multiSearchAnyCaseInsensitive('ClickHouse',['c','h']); 1 ``` -## multiSearchAnyUTF8 {#multiSearchAnyUTF8} +## multiSearchAnyUTF8 -Like [multiSearchAny](#multisearchany) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings. +Like [multiSearchAny](#multisearchany) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings. *Syntax** @@ -625,7 +625,7 @@ multiSearchAnyUTF8(haystack, [needle1, needle2, ..., needleN]) **Parameters** - `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md) +- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md). **Returned value** @@ -646,7 +646,7 @@ SELECT multiSearchAnyUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x 1 ``` -## multiSearchAnyCaseInsensitiveUTF8 {#multiSearchAnyCaseInsensitiveUTF8} +## multiSearchAnyCaseInsensitiveUTF8 Like [multiSearchAnyUTF8](#multiSearchAnyUTF8) but ignores case. From 7288aeb826ec4ea964bb6e408c7644a2f53cf7c1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 21:45:36 +0100 Subject: [PATCH 019/243] More consistency edits --- .../functions/string-search-functions.md | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 9430fb0f728..7ffaee53f89 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -126,6 +126,8 @@ Like [position](#position) but assumes `haystack` and `needle` are UTF-8 encoded Function `positionUTF8` correctly counts character `ö` (represented by two points) as a single Unicode codepoint: +Query: + ``` sql SELECT positionUTF8('Motörhead', 'r'); ``` @@ -163,11 +165,13 @@ multiSearchAllPositions(haystack, [needle1, needle2, ..., needleN]) **Returned values** -- Array of the starting position in bytes and counting from 1 (if the substring was found). +- Array of the starting position in bytes and counting from 1, if the substring was found. - 0, if the substring was not found. **Example** +Query: + ``` sql SELECT multiSearchAllPositions('Hello, World!', ['hello', '!', 'world']); ``` @@ -207,6 +211,8 @@ Query: SELECT multiSearchAllPositionsCaseInsensitive('ClickHouse',['c','h']); ``` +Result: + ```response ["1","6"] ``` @@ -241,6 +247,8 @@ Query: SELECT multiSearchAllPositionsUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']); ``` +Result: + ```response ["1","6"] ``` @@ -275,6 +283,8 @@ Query: SELECT multiSearchAllPositionsCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x63','\x68']); ``` +Result: + ```response ["1","6"] ``` @@ -309,6 +319,8 @@ Query: SELECT multiSearchFirstPosition('Hello World',['llo', 'Wor', 'ld']); ``` +Result: + ```response 3 ``` @@ -341,6 +353,8 @@ Query: SELECT multiSearchFirstPositionCaseInsensitive('HELLO WORLD',['wor', 'ld', 'ello']); ``` +Result: + ```response 2 ``` @@ -375,6 +389,8 @@ Query: SELECT multiSearchFirstPositionUTF8('\x68\x65\x6c\x6c\x6f\x20\x77\x6f\x72\x6c\x64',['wor', 'ld', 'ello']); ``` +Result: + ```response 2 ``` @@ -409,6 +425,8 @@ Query: SELECT multiSearchFirstPositionCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['wor', 'ld', 'ello']); ``` +Result: + ```response 2 ``` @@ -442,6 +460,8 @@ Query: SELECT multiSearchFirstIndex('Hello World',['World','Hello']); ``` +Result: + ```response 1 ``` @@ -474,6 +494,8 @@ Query: SELECT multiSearchFirstIndexCaseInsensitive('hElLo WoRlD',['World','Hello']); ``` +Result: + ```response 1 ``` @@ -508,6 +530,8 @@ Query: SELECT multiSearchFirstIndexUTF8('\x48\x65\x6c\x6c\x6f\x20\x57\x6f\x72\x6c\x64',['\x57\x6f\x72\x6c\x64','\x48\x65\x6c\x6c\x6f']); ``` +Result: + ```response 1 ``` @@ -542,6 +566,8 @@ Query: SELECT multiSearchFirstIndexCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['\x68\x65\x6c\x6c\x6f','\x77\x6f\x72\x6c\x64']); ``` +Result: + ```response 1 ``` @@ -576,6 +602,8 @@ Query: SELECT multiSearchAny('ClickHouse',['C','H']); ``` +Result: + ```response 1 ``` @@ -608,6 +636,8 @@ Query: SELECT multiSearchAnyCaseInsensitive('ClickHouse',['c','h']); ``` +Result: + ```response 1 ``` @@ -642,6 +672,8 @@ Query: SELECT multiSearchAnyUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']); ``` +Result: + ```response 1 ``` @@ -676,6 +708,8 @@ Query: SELECT multiSearchAnyCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x68']); ``` +Result: + ```response 1 ``` From 1dcba74f66d64bf7d77b7da585cabc16a5d4e29f Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 28 Mar 2024 21:54:26 +0100 Subject: [PATCH 020/243] Small grammar edits to description at top of the page --- .../functions/string-search-functions.md | 43 ++++++++++++++++--- 1 file changed, 37 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 7ffaee53f89..8aff8b7e930 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -6,14 +6,17 @@ sidebar_label: Searching in Strings # Functions for Searching in Strings -All functions in this section search by default case-sensitively. Case-insensitive search is usually provided by separate function variants. -Note that case-insensitive search follows the lowercase-uppercase rules of the English language. E.g. Uppercased `i` in English language is -`I` whereas in Turkish language it is `İ` - results for languages other than English may be unexpected. +All functions in this section search case-sensitively by default. Case-insensitive search is usually provided by separate function variants. -Functions in this section also assume that the searched string and the search string are single-byte encoded text. If this assumption is +:::note +Case-insensitive search follows the lowercase-uppercase rules of the English language. E.g. Uppercased `i` in the English language is +`I` whereas in the Turkish language it is `İ` - results for languages other than English may be unexpected. +::: + +Functions in this section also assume that the searched string (refered to in this section as `haystack`) and the search string (refered to in this section as `needle`) are single-byte encoded text. If this assumption is violated, no exception is thrown and results are undefined. Search with UTF-8 encoded strings is usually provided by separate function variants. Likewise, if a UTF-8 function variant is used and the input strings are not UTF-8 encoded text, no exception is thrown and the -results are undefined. Note that no automatic Unicode normalization is performed, you can use the +results are undefined. Note that no automatic Unicode normalization is performed, however you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that. [General strings functions](string-functions.md) and [functions for replacing in strings](string-replace-functions.md) are described separately. @@ -55,6 +58,8 @@ Type: `Integer`. **Examples** +Query: + ``` sql SELECT position('Hello, world!', '!'); ``` @@ -69,12 +74,16 @@ Result: Example with `start_pos` argument: +Query: + ``` sql SELECT position('Hello, world!', 'o', 1), position('Hello, world!', 'o', 7) ``` +Result: + ``` text ┌─position('Hello, world!', 'o', 1)─┬─position('Hello, world!', 'o', 7)─┐ │ 5 │ 9 │ @@ -83,6 +92,8 @@ SELECT Example for `needle IN haystack` syntax: +Query: + ```sql SELECT 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s); ``` @@ -97,6 +108,8 @@ Result: Examples with empty `needle` substring: +Query: + ``` sql SELECT position('abc', ''), @@ -108,6 +121,8 @@ SELECT position('abc', '', 5) ``` +Result: + ``` text ┌─position('abc', '')─┬─position('abc', '', 0)─┬─position('abc', '', 1)─┬─position('abc', '', 2)─┬─position('abc', '', 3)─┬─position('abc', '', 4)─┬─position('abc', '', 5)─┐ │ 1 │ 1 │ 1 │ 2 │ 3 │ 4 │ 0 │ @@ -116,7 +131,23 @@ SELECT ## positionCaseInsensitive -Like [position](#position) but searches case-insensitively. +A case insensitive invariant of [position](#position). + +**Example** + +Query: + +``` sql +SELECT position('Hello, world!', 'hello'); +``` + +Result: + +``` text +┌─position('Hello, world!', 'hello')─┐ +│ 0 │ +└────────────────────────────────────┘ +``` ## positionUTF8 From 2a183fcbef32152534846bb569013e686284ef76 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 1 Apr 2024 16:29:39 +0000 Subject: [PATCH 021/243] fix build --- src/Databases/DatabasesCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 824e9fd7b4d..c074bf201bd 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -348,7 +348,7 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name backQuote(database_name), backQuote(table_name)); } -std::vector> DatabaseWithOwnTablesBase::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context, bool skip_not_loaded) const +std::vector> DatabaseWithOwnTablesBase::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const { std::vector> res; From aacb65299b21ab07cbf94a9d9b5d5a8c761b8935 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 1 Apr 2024 16:45:45 +0000 Subject: [PATCH 022/243] fix --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index b07b203f786..3f62b9719d2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -97,7 +97,7 @@ bool DatabasePostgreSQL::empty() const } -DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */) const +DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */, bool /* skip_not_loaded */) const { std::lock_guard lock(mutex); Tables tables; From 9c98f47f97c6a6c260a2969cd9ccfaafd22d0740 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 1 Apr 2024 16:53:49 +0000 Subject: [PATCH 023/243] more fixes --- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabasesCommon.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 2688ff2443c..1de7f80f512 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -225,7 +225,7 @@ std::vector> DatabaseHDFS::getTablesForBackup(cons * Returns an empty iterator because the database does not have its own tables * But only caches them for quick access */ -DatabaseTablesIteratorPtr DatabaseHDFS::getTablesIterator(ContextPtr, const FilterByNameFunction &) const +DatabaseTablesIteratorPtr DatabaseHDFS::getTablesIterator(ContextPtr, const FilterByNameFunction &, bool) const { return std::make_unique(Tables{}, getDatabaseName()); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 3859c2fe0ce..e9168d68ef7 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -460,7 +460,7 @@ DatabaseTablesIteratorPtr DatabaseOrdinary::getTablesIterator(ContextPtr local_c return DatabaseWithOwnTablesBase::getTablesIterator(local_context, filter_by_table_name, skip_not_loaded); } -Strings DatabaseOrdinary::getAllTableNames(ContextPtr context) const +Strings DatabaseOrdinary::getAllTableNames(ContextPtr) const { std::set unique_names; { diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 3b1646f1903..7b8f7468e81 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -873,7 +873,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep std::vector replicated_tables_to_rename; size_t total_tables = 0; std::vector replicated_ids; - for (auto existing_tables_it = getTablesIterator(getContext(), {}); existing_tables_it->isValid(); + for (auto existing_tables_it = getTablesIterator(getContext(), {}, /*skip_not_loaded=*/false); existing_tables_it->isValid(); existing_tables_it->next(), ++total_tables) { String name = existing_tables_it->name(); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index c074bf201bd..57385c98c15 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -352,7 +352,7 @@ std::vector> DatabaseWithOwnTablesBase::getTablesF { std::vector> res; - for (auto it = getTablesIterator(local_context, filter); it->isValid(); it->next()) + for (auto it = getTablesIterator(local_context, filter, /*skip_not_loaded=*/false); it->isValid(); it->next()) { auto storage = it->table(); if (!storage) From 2f5c52f7dff0e3a5c9bd82d565dfc368fa2ab4f0 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 2 Apr 2024 05:14:12 +0000 Subject: [PATCH 024/243] fix 0320_long_values_pretty_are_not_cut_if_single Signed-off-by: Duc Canh Le --- .../03020_long_values_pretty_are_not_cut_if_single.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh b/tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh index fa9e9f6d3e1..b66951d93f6 100755 --- a/tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh +++ b/tests/queries/0_stateless/03020_long_values_pretty_are_not_cut_if_single.sh @@ -9,6 +9,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # But cutting it in the result of SHOW CREATE TABLE will be bad for a user. # That's why we control it with the setting `output_format_pretty_max_value_width_apply_for_single_value`. +# Make sure that system.metric_log exists +${CLICKHOUSE_CLIENT} --query "SELECT 1 FORMAT Null" +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + + ${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format Pretty | grep -P '^COMMENT' ${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format PrettyCompact | grep -P '^COMMENT' ${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE system.metric_log" --format PrettySpace | grep -P '^COMMENT' From ed9ee5ab4cfa56cd615024d20f1d7d1c31b88be3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 15:47:48 +0000 Subject: [PATCH 025/243] First portion --- .../03033_with_fill_interpolate.reference | 0 .../03033_with_fill_interpolate.sql | 27 ++++++++++++ .../03034_normalized_ast.reference | 0 .../0_stateless/03034_normalized_ast.sql | 7 +++ ...035_alias_column_bug_distributed.reference | 0 .../03035_alias_column_bug_distributed.sql | 43 +++++++++++++++++++ .../0_stateless/03036_with_numbers.reference | 20 +++++++++ .../0_stateless/03036_with_numbers.sql | 8 ++++ 8 files changed, 105 insertions(+) create mode 100644 tests/queries/0_stateless/03033_with_fill_interpolate.reference create mode 100644 tests/queries/0_stateless/03033_with_fill_interpolate.sql create mode 100644 tests/queries/0_stateless/03034_normalized_ast.reference create mode 100644 tests/queries/0_stateless/03034_normalized_ast.sql create mode 100644 tests/queries/0_stateless/03035_alias_column_bug_distributed.reference create mode 100644 tests/queries/0_stateless/03035_alias_column_bug_distributed.sql create mode 100644 tests/queries/0_stateless/03036_with_numbers.reference create mode 100644 tests/queries/0_stateless/03036_with_numbers.sql diff --git a/tests/queries/0_stateless/03033_with_fill_interpolate.reference b/tests/queries/0_stateless/03033_with_fill_interpolate.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03033_with_fill_interpolate.sql b/tests/queries/0_stateless/03033_with_fill_interpolate.sql new file mode 100644 index 00000000000..816633af757 --- /dev/null +++ b/tests/queries/0_stateless/03033_with_fill_interpolate.sql @@ -0,0 +1,27 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/55794 +DROP TABLE IF EXISTS 03033_example_table; + +CREATE TABLE 03033_example_table +( + ColumnA Int64, + ColumnB Int64, + ColumnC Int64 +) +ENGINE = MergeTree() +ORDER BY ColumnA; + +WITH +helper AS ( + SELECT + * + FROM + 03033_example_table + ORDER BY + ColumnA WITH FILL INTERPOLATE ( + ColumnB AS ColumnC, + ColumnC AS ColumnA + ) +) +SELECT ColumnB FROM helper; + +DROP TABLE IF EXISTS 03033_example_table; diff --git a/tests/queries/0_stateless/03034_normalized_ast.reference b/tests/queries/0_stateless/03034_normalized_ast.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03034_normalized_ast.sql b/tests/queries/0_stateless/03034_normalized_ast.sql new file mode 100644 index 00000000000..ff6f8da118c --- /dev/null +++ b/tests/queries/0_stateless/03034_normalized_ast.sql @@ -0,0 +1,7 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/49472 +SELECT + concat(database, table) AS name, + count() +FROM clusterAllReplicas(default, system.tables) +GROUP BY name +FORMAT Null; diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.reference b/tests/queries/0_stateless/03035_alias_column_bug_distributed.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql new file mode 100644 index 00000000000..fb459b3289b --- /dev/null +++ b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql @@ -0,0 +1,43 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/44414 +DROP TABLE IF EXISTS alias_bug; +DROP TABLE IF EXISTS alias_bug_dist; +CREATE TABLE alias_bug +( + `src` String, + `theAlias` String ALIAS trimBoth(src) +) +ENGINE = MergeTree() +ORDER BY src; + +CREATE TABLE alias_bug_dist +AS alias_bug +ENGINE = Distributed('default', currentDatabase(), 'alias_bug', rand()); + +INSERT INTO alias_bug VALUES ('SOURCE1'); + +-- OK +SELECT theAlias,CAST(NULL, 'Nullable(String)') AS src FROM alias_bug LIMIT 1 FORMAT Null; + +-- Not OK +SELECT theAlias,CAST(NULL, 'Nullable(String)') AS src FROM alias_bug_dist LIMIT 1 FORMAT Null; + +DROP TABLE IF EXISTS alias_bug; +DROP TABLE IF EXISTS alias_bug_dist; +CREATE TABLE alias_bug +( + `s` String, + `src` String, + `theAlias` String ALIAS trimBoth(src) +) +ENGINE = MergeTree() +ORDER BY src; + +CREATE TABLE alias_bug_dist +AS alias_bug +ENGINE = Distributed('default', currentDatabase(), 'alias_bug', rand()); + +-- Unknown identifier +SELECT CAST(123, 'String') AS src,theAlias FROM alias_bug_dist LIMIT 1 FORMAT Null; + +DROP TABLE IF EXISTS alias_bug; +DROP TABLE IF EXISTS alias_bug_dist; diff --git a/tests/queries/0_stateless/03036_with_numbers.reference b/tests/queries/0_stateless/03036_with_numbers.reference new file mode 100644 index 00000000000..7b36cc96f5e --- /dev/null +++ b/tests/queries/0_stateless/03036_with_numbers.reference @@ -0,0 +1,20 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/03036_with_numbers.sql b/tests/queries/0_stateless/03036_with_numbers.sql new file mode 100644 index 00000000000..5e08bb6e065 --- /dev/null +++ b/tests/queries/0_stateless/03036_with_numbers.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/13843 +WITH 10 AS n +SELECT * +FROM numbers(n); + +WITH cast(10, 'UInt64') AS n +SELECT * +FROM numbers(n); From c11aa0122647b39177d381499d08f6102a5e5160 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 17:48:48 +0200 Subject: [PATCH 026/243] Fix race --- src/Interpreters/Cache/EvictionCandidates.cpp | 16 +++++++++++++--- src/Interpreters/Cache/EvictionCandidates.h | 4 ++-- src/Interpreters/Cache/FileSegment.cpp | 5 +++++ src/Interpreters/Cache/FileSegment.h | 1 + src/Interpreters/Cache/LRUFileCachePriority.cpp | 6 ++++++ src/Interpreters/Cache/LRUFileCachePriority.h | 2 +- 6 files changed, 28 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index 080ef675918..4ca6aeea22e 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -30,6 +30,10 @@ EvictionCandidates::~EvictionCandidates() iterator->invalidate(); } + /// We cannot reset evicting flag if we already removed queue entries. + if (removed_queue_entries) + return; + /// Here `candidates` contain only those file segments /// which failed to be removed during evict() /// because there was some exception before evict() @@ -62,9 +66,15 @@ void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & loc for (const auto & [key, key_candidates] : candidates) { for (const auto & candidate : key_candidates.candidates) + { + const auto & file_segment = candidate->file_segment; + auto file_segment_lock = file_segment->lock(); + candidate->getQueueIterator()->remove(lock); + file_segment->setQueueIteratorUnlocked(nullptr, file_segment_lock); + } } - invalidated_queue_entries = true; + removed_queue_entries = true; } void EvictionCandidates::evict() @@ -74,7 +84,7 @@ void EvictionCandidates::evict() auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); - if (!invalidated_queue_entries) + if (!removed_queue_entries) queue_entries_to_invalidate.reserve(candidates_size); for (auto & [key, key_candidates] : candidates) @@ -123,7 +133,7 @@ void EvictionCandidates::evict() /// it was freed in favour of some reserver, so we can make it visibly /// free only for that particular reserver. - if (!invalidated_queue_entries) + if (!removed_queue_entries) queue_entries_to_invalidate.push_back(iterator); key_candidates.candidates.pop_back(); diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 571010a14bc..140728ae704 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -4,7 +4,7 @@ namespace DB { -class EvictionCandidates +class EvictionCandidates : private boost::noncopyable { public: using FinalizeEvictionFunc = std::function; @@ -60,7 +60,7 @@ private: std::vector on_finalize; std::vector queue_entries_to_invalidate; - bool invalidated_queue_entries = false; + bool removed_queue_entries = false; IFileCachePriority::HoldSpacePtr hold_space; }; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 9ec2b090dc7..0d64b602928 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -165,6 +165,11 @@ FileSegment::Priority::IteratorPtr FileSegment::getQueueIterator() const void FileSegment::setQueueIterator(Priority::IteratorPtr iterator) { auto lock = lockFileSegment(); + setQueueIteratorUnlocked(iterator, lock); +} + +void FileSegment::setQueueIteratorUnlocked(Priority::IteratorPtr iterator, const FileSegmentGuard::Lock &) +{ if (queue_iterator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Queue iterator cannot be set twice"); queue_iterator = iterator; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index c34ee064345..12046e59bd6 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -176,6 +176,7 @@ public: Priority::IteratorPtr getQueueIterator() const; void setQueueIterator(Priority::IteratorPtr iterator); + void setQueueIteratorUnlocked(Priority::IteratorPtr iterator, const FileSegmentGuard::Lock &); KeyMetadataPtr tryGetKeyMetadata() const; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 78ece5a3124..4f2b17ea104 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -431,6 +431,12 @@ bool LRUFileCachePriority::modifySizeLimits( return true; } +IFileCachePriority::EntryPtr LRUFileCachePriority::LRUIterator::getEntry() const +{ + assertValid(); + return *iterator; +} + void LRUFileCachePriority::LRUIterator::remove(const CachePriorityGuard::Lock & lock) { assertValid(); diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index ed3a455126a..0d10f22701e 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -146,7 +146,7 @@ public: LRUIterator & operator =(const LRUIterator & other); bool operator ==(const LRUIterator & other) const; - EntryPtr getEntry() const override { return *iterator; } + EntryPtr getEntry() const override; size_t increasePriority(const CachePriorityGuard::Lock &) override; From 4441a1b3f3c2f6844e233e379335d4bdf4922ab5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 15:59:17 +0000 Subject: [PATCH 027/243] Close https://github.com/ClickHouse/ClickHouse/issues/55803 --- .../0_stateless/03037_union_view.reference | 0 .../queries/0_stateless/03037_union_view.sql | 24 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03037_union_view.reference create mode 100644 tests/queries/0_stateless/03037_union_view.sql diff --git a/tests/queries/0_stateless/03037_union_view.reference b/tests/queries/0_stateless/03037_union_view.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03037_union_view.sql b/tests/queries/0_stateless/03037_union_view.sql new file mode 100644 index 00000000000..04f4afab4c4 --- /dev/null +++ b/tests/queries/0_stateless/03037_union_view.sql @@ -0,0 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/55803 +CREATE TABLE broken_table +( + start DateTime64(6), + end DateTime64(6), +) +ENGINE = ReplacingMergeTree(start) +ORDER BY (start); + +CREATE VIEW broken_view as +SELECT + t.start as start, + t.end as end, + cast(datediff('second', t.start, t.end) as float) as total_sec +FROM broken_table t FINAL +UNION ALL +SELECT + null as start, + null as end, + null as total_sec; + +SELECT v.start, v.total_sec +FROM broken_view v FINAL +WHERE v.start IS NOT NULL; From d6504764000b762cea48a2c633286ea77cb388c7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:01:55 +0000 Subject: [PATCH 028/243] Better --- tests/queries/0_stateless/03037_union_view.sql | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/03037_union_view.sql b/tests/queries/0_stateless/03037_union_view.sql index 04f4afab4c4..fb8aa7df954 100644 --- a/tests/queries/0_stateless/03037_union_view.sql +++ b/tests/queries/0_stateless/03037_union_view.sql @@ -1,4 +1,7 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55803 +DROP TABLE IF EXISTS broken_table; +DROP TABLE IF EXISTS broken_view; + CREATE TABLE broken_table ( start DateTime64(6), @@ -22,3 +25,6 @@ SELECT SELECT v.start, v.total_sec FROM broken_view v FINAL WHERE v.start IS NOT NULL; + +DROP TABLE IF EXISTS broken_table; +DROP TABLE IF EXISTS broken_view; \ No newline at end of file From f5c514615301659bd9fad8b6dcc13623a034a620 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Apr 2024 18:04:53 +0200 Subject: [PATCH 029/243] [RFC] Try to add global Real and CPU trace collector --- programs/server/Server.cpp | 6 ++- src/Common/QueryProfiler.cpp | 8 ++++ src/Common/QueryProfiler.h | 2 + src/Common/ThreadPool.cpp | 15 ++++--- src/Common/ThreadPool.h | 32 ++++++++++++--- src/Common/ThreadPool_fwd.h | 7 ++-- src/Common/ThreadStatus.h | 1 + src/Core/ServerSettings.h | 2 + src/Interpreters/ThreadStatusExt.cpp | 40 +++++++++++++++++-- .../config.d/serverwide_trace_collector.xml | 4 ++ .../__init__.py | 1 + .../configs/global_profiler.xml | 4 ++ .../test_trace_collector_serverwide/test.py | 38 ++++++++++++++++++ 13 files changed, 142 insertions(+), 18 deletions(-) create mode 100644 tests/config/config.d/serverwide_trace_collector.xml create mode 100644 tests/integration/test_trace_collector_serverwide/__init__.py create mode 100644 tests/integration/test_trace_collector_serverwide/configs/global_profiler.xml create mode 100644 tests/integration/test_trace_collector_serverwide/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 450e1696c11..a048bebc45b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -734,13 +734,17 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif + bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. GlobalThreadPool::initialize( server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, - server_settings.thread_pool_queue_size); + server_settings.thread_pool_queue_size, + will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 34ffbf6c498..3b7289167e3 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -198,6 +198,7 @@ void Timer::cleanup() { if (timer_id) { + LOG_INFO(log, "CLEANUP TIMER"); int err = timer_delete(*timer_id); if (err) LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString()); @@ -260,6 +261,13 @@ QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_t #endif } + +template +void QueryProfilerBase::setPeriod(UInt32 period_) +{ + timer.set(period_); +} + template QueryProfilerBase::~QueryProfilerBase() { diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 254b11137cc..ea4cc73bca6 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -57,6 +57,8 @@ public: QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_); ~QueryProfilerBase(); + void setPeriod(UInt32 period_); + private: void cleanup(); diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 3c2e6228421..eaee070c44f 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -490,8 +490,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ template class ThreadPoolImpl; -template class ThreadPoolImpl>; -template class ThreadFromGlobalPoolImpl; +template class ThreadPoolImpl>; +template class ThreadFromGlobalPoolImpl; +template class ThreadFromGlobalPoolImpl; std::unique_ptr GlobalThreadPool::the_instance; @@ -500,7 +501,9 @@ GlobalThreadPool::GlobalThreadPool( size_t max_threads_, size_t max_free_threads_, size_t queue_size_, - const bool shutdown_on_exception_) + const bool shutdown_on_exception_, + UInt64 global_profiler_real_time_period_ns_, + UInt64 global_profiler_cpu_time_period_ns_) : FreeThreadPool( CurrentMetrics::GlobalThread, CurrentMetrics::GlobalThreadActive, @@ -509,10 +512,12 @@ GlobalThreadPool::GlobalThreadPool( max_free_threads_, queue_size_, shutdown_on_exception_) + , global_profiler_real_time_period_ns(global_profiler_real_time_period_ns_) + , global_profiler_cpu_time_period_ns(global_profiler_cpu_time_period_ns_) { } -void GlobalThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) +void GlobalThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size, UInt64 global_profiler_real_time_period_ns, UInt64 global_profiler_cpu_time_period_ns) { if (the_instance) { @@ -520,7 +525,7 @@ void GlobalThreadPool::initialize(size_t max_threads, size_t max_free_threads, s "The global thread pool is initialized twice"); } - the_instance.reset(new GlobalThreadPool(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/)); + the_instance.reset(new GlobalThreadPool(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/, global_profiler_real_time_period_ns, global_profiler_cpu_time_period_ns)); } GlobalThreadPool & GlobalThreadPool::instance() diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 31e4eabf63b..528f782caf2 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -172,10 +172,21 @@ class GlobalThreadPool : public FreeThreadPool, private boost::noncopyable size_t max_threads_, size_t max_free_threads_, size_t queue_size_, - bool shutdown_on_exception_); + bool shutdown_on_exception_, + UInt64 global_profiler_real_time_period_ns_, + UInt64 global_profiler_cpu_time_period_ns_); public: - static void initialize(size_t max_threads = 10000, size_t max_free_threads = 1000, size_t queue_size = 10000); + UInt64 global_profiler_real_time_period_ns; + UInt64 global_profiler_cpu_time_period_ns; + + static void initialize( + size_t max_threads = 10000, + size_t max_free_threads = 1000, + size_t queue_size = 10000, + UInt64 global_profiler_real_time_period_ns_ = 0, + UInt64 global_profiler_cpu_time_period_ns_ = 0); + static GlobalThreadPool & instance(); static void shutdown(); }; @@ -187,7 +198,7 @@ public: * NOTE: User code should use 'ThreadFromGlobalPool' declared below instead of directly using this class. * */ -template +template class ThreadFromGlobalPoolImpl : boost::noncopyable { public: @@ -197,11 +208,15 @@ public: explicit ThreadFromGlobalPoolImpl(Function && func, Args &&... args) : state(std::make_shared()) { + UInt64 global_profiler_real_time_period = GlobalThreadPool::instance().global_profiler_real_time_period_ns; + UInt64 global_profiler_cpu_time_period = GlobalThreadPool::instance().global_profiler_cpu_time_period_ns; /// NOTE: /// - If this will throw an exception, the destructor won't be called /// - this pointer cannot be passed in the lambda, since after detach() it will not be valid GlobalThreadPool::instance().scheduleOrThrow([ my_state = state, + global_profiler_real_time_period, + global_profiler_cpu_time_period, my_func = std::forward(func), my_args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture { @@ -220,6 +235,12 @@ public: /// Thread status holds raw pointer on query context, thus it always must be destroyed /// before sending signal that permits to join this thread. DB::ThreadStatus thread_status; + if constexpr (global_trace_collector_allowed) + { + if (unlikely(global_profiler_real_time_period != 0 || global_profiler_cpu_time_period != 0)) + thread_status.initGlobalProfiler(global_profiler_real_time_period, global_profiler_cpu_time_period); + } + std::apply(function, arguments); }, {}, // default priority @@ -305,11 +326,12 @@ protected: /// you need to use class, or you need to use ThreadFromGlobalPool below. /// /// See the comments of ThreadPool below to know how it works. -using ThreadFromGlobalPoolNoTracingContextPropagation = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPoolNoTracingContextPropagation = ThreadFromGlobalPoolImpl; /// An alias of thread that execute jobs/tasks on global thread pool by implicit passing tracing context on current thread to underlying worker as parent tracing context. /// If jobs/tasks are directly scheduled by using APIs of this class, you need to use this class or you need to use class above. -using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPoolWithoutTraceCollector = ThreadFromGlobalPoolImpl; /// Recommended thread pool for the case when multiple thread pools are created and destroyed. /// diff --git a/src/Common/ThreadPool_fwd.h b/src/Common/ThreadPool_fwd.h index 2782acc9c51..fea4e59f087 100644 --- a/src/Common/ThreadPool_fwd.h +++ b/src/Common/ThreadPool_fwd.h @@ -3,11 +3,12 @@ template class ThreadPoolImpl; -template +template class ThreadFromGlobalPoolImpl; -using ThreadFromGlobalPoolNoTracingContextPropagation = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPoolNoTracingContextPropagation = ThreadFromGlobalPoolImpl; -using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPool = ThreadFromGlobalPoolImpl; +using ThreadFromGlobalPoolWithoutTraceCollector = ThreadFromGlobalPoolImpl; using ThreadPool = ThreadPoolImpl; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 48b52f8aa6e..2d33c0ac021 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -307,6 +307,7 @@ public: void flushUntrackedMemory(); + void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); private: void applyGlobalSettings(); void applyQuerySettings(); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 6608a35a5a2..e05b3cf9e31 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -137,6 +137,8 @@ namespace DB M(UInt64, http_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ M(UInt64, http_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ M(UInt64, http_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the http connections which do not belong to any disk or storage.", 0) \ + M(UInt64, global_profiler_real_time_period_ns, 0, "Period for real clock timer of global profiler (in nanoseconds). Set 0 value to turn off the real clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ + M(UInt64, global_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of global profiler (in nanoseconds). Set 0 value to turn off the CPU clock global profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 1c24c4f85c9..4b9bd069bc6 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #if defined(OS_LINUX) # include @@ -457,6 +458,27 @@ void ThreadStatus::resetPerformanceCountersLastUsage() taskstats->reset(); } + +void ThreadStatus::initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period) +{ + + try + { + if (global_profiler_real_time_period > 0) + query_profiler_real = std::make_unique(thread_id, + /* period= */ static_cast(global_profiler_real_time_period)); + + if (global_profiler_cpu_time_period > 0) + query_profiler_cpu = std::make_unique(thread_id, + /* period= */ static_cast(global_profiler_cpu_time_period)); + } + catch (...) + { + tryLogCurrentException("ThreadStatus", "Cannot initialize GlobalProfiler"); + } + +} + void ThreadStatus::initQueryProfiler() { if (internal_thread) @@ -474,12 +496,22 @@ void ThreadStatus::initQueryProfiler() try { if (settings.query_profiler_real_time_period_ns > 0) - query_profiler_real = std::make_unique(thread_id, - /* period= */ static_cast(settings.query_profiler_real_time_period_ns)); + { + if (!query_profiler_real) + query_profiler_real = std::make_unique(thread_id, + /* period= */ static_cast(settings.query_profiler_real_time_period_ns)); + else + query_profiler_real->setPeriod(static_cast(settings.query_profiler_real_time_period_ns)); + } if (settings.query_profiler_cpu_time_period_ns > 0) - query_profiler_cpu = std::make_unique(thread_id, - /* period= */ static_cast(settings.query_profiler_cpu_time_period_ns)); + { + if (!query_profiler_cpu) + query_profiler_cpu = std::make_unique(thread_id, + /* period= */ static_cast(settings.query_profiler_cpu_time_period_ns)); + else + query_profiler_cpu->setPeriod(static_cast(settings.query_profiler_cpu_time_period_ns)); + } } catch (...) { diff --git a/tests/config/config.d/serverwide_trace_collector.xml b/tests/config/config.d/serverwide_trace_collector.xml new file mode 100644 index 00000000000..602e07469f3 --- /dev/null +++ b/tests/config/config.d/serverwide_trace_collector.xml @@ -0,0 +1,4 @@ + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_trace_collector_serverwide/__init__.py b/tests/integration/test_trace_collector_serverwide/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_trace_collector_serverwide/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_trace_collector_serverwide/configs/global_profiler.xml b/tests/integration/test_trace_collector_serverwide/configs/global_profiler.xml new file mode 100644 index 00000000000..5112d267182 --- /dev/null +++ b/tests/integration/test_trace_collector_serverwide/configs/global_profiler.xml @@ -0,0 +1,4 @@ + + 10000000 + 10000000 + diff --git a/tests/integration/test_trace_collector_serverwide/test.py b/tests/integration/test_trace_collector_serverwide/test.py new file mode 100644 index 00000000000..5a7bba15fd7 --- /dev/null +++ b/tests/integration/test_trace_collector_serverwide/test.py @@ -0,0 +1,38 @@ +#!/usr/bin/env python3 + +import pytest +import time + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", main_configs=["configs/global_profiler.xml"]) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def test_global_thread_profiler(start_cluster): + node1.query("CREATE TABLE t (key UInt32, value String) Engine = MergeTree() ORDER BY key") + + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + + time.sleep(5) + + node1.query("SYSTEM FLUSH LOGS") + + assert int(node1.query("SELECT count() FROM system.trace_log where trace_type='Real' and query_id = ''").strip()) > 0 From 3d9a6e9b8e5f8864f9f8a0481439a316c5deaba7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:06:02 +0000 Subject: [PATCH 030/243] Close https://github.com/ClickHouse/ClickHouse/issues/48308 --- .../03038_ambiguous_column.reference | 0 .../0_stateless/03038_ambiguous_column.sql | 41 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03038_ambiguous_column.reference create mode 100644 tests/queries/0_stateless/03038_ambiguous_column.sql diff --git a/tests/queries/0_stateless/03038_ambiguous_column.reference b/tests/queries/0_stateless/03038_ambiguous_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03038_ambiguous_column.sql b/tests/queries/0_stateless/03038_ambiguous_column.sql new file mode 100644 index 00000000000..69c8e52d734 --- /dev/null +++ b/tests/queries/0_stateless/03038_ambiguous_column.sql @@ -0,0 +1,41 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/48308 +DROP TABLE IF EXISTS 03038_table; + +CREATE TABLE 03038_table +( + `time` DateTime +) +ENGINE = MergeTree +ORDER BY time; + +SELECT * +FROM +( + SELECT + toUInt64(time) AS time, + toHour(03038_table.time) + FROM 03038_table +) +ORDER BY time ASC; + +WITH subquery AS ( + SELECT + toUInt64(time) AS time, + toHour(03038_table.time) + FROM 03038_table +) +SELECT * +FROM subquery +ORDER BY subquery.time ASC; + +SELECT * +FROM +( + SELECT + toUInt64(time) AS time, + toHour(03038_table.time) AS hour + FROM 03038_table +) +ORDER BY time ASC, hour; + +DROP TABLE IF EXISTS 03038_table; From 723a733c84a241de56fbf66ffc84a332c995c673 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Apr 2024 18:07:02 +0200 Subject: [PATCH 031/243] Missing change --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 652d25a0a35..06f2f5fe902 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -67,6 +67,7 @@ ln -sf $SRC_PATH/config.d/validate_tcp_client_information.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/config.d/zero_copy_destructive_operations.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/block_number.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/handlers.yaml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/serverwide_trace_collector.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From d11d10050b3315227dbf0019a86bd8fa25d9bf71 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Apr 2024 18:08:04 +0200 Subject: [PATCH 032/243] Remove debug line --- src/Common/QueryProfiler.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 3b7289167e3..f985ec95e88 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -198,7 +198,6 @@ void Timer::cleanup() { if (timer_id) { - LOG_INFO(log, "CLEANUP TIMER"); int err = timer_delete(*timer_id); if (err) LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString()); From a54efe56450ed781e5fb101014cd460b9db6fefb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:11:02 +0000 Subject: [PATCH 033/243] Close https://github.com/ClickHouse/ClickHouse/issues/45535 --- ...known_identifier_window_function.reference | 40 +++++++++++++++++++ ...039_unknown_identifier_window_function.sql | 34 ++++++++++++++++ 2 files changed, 74 insertions(+) create mode 100644 tests/queries/0_stateless/03039_unknown_identifier_window_function.reference create mode 100644 tests/queries/0_stateless/03039_unknown_identifier_window_function.sql diff --git a/tests/queries/0_stateless/03039_unknown_identifier_window_function.reference b/tests/queries/0_stateless/03039_unknown_identifier_window_function.reference new file mode 100644 index 00000000000..405da28a579 --- /dev/null +++ b/tests/queries/0_stateless/03039_unknown_identifier_window_function.reference @@ -0,0 +1,40 @@ +0 10 +1 10 +2 10 +3 10 +4 10 +5 10 +6 10 +7 10 +8 10 +9 10 +0 10 +1 10 +2 10 +3 10 +4 10 +5 10 +6 10 +7 10 +8 10 +9 10 +0 10 0 +1 10 1 +2 10 2 +3 10 3 +4 10 4 +5 10 5 +6 10 6 +7 10 7 +8 10 8 +9 10 9 +0 10 +1 10 +2 10 +3 10 +4 10 +5 10 +6 10 +7 10 +8 10 +9 10 diff --git a/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql new file mode 100644 index 00000000000..ca3bb521eba --- /dev/null +++ b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql @@ -0,0 +1,34 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/45535 + +SELECT + *, + count() OVER () AS c +FROM numbers(10) +ORDER BY toString(number); + + +WITH + toString(number) as str +SELECT + *, + count() OVER () AS c +FROM numbers(10) +ORDER BY str; + +SELECT + *, + count() OVER () AS c, + toString(number) as str +FROM numbers(10) +ORDER BY str; + + +WITH + test AS ( + SELECT + *, + count() OVER () AS c + FROM numbers(10) + ) +SELECT * FROM test +ORDER BY toString(number); From 44847fbb2faf0d82a05c4b247d4540ca446f2269 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 18:16:22 +0200 Subject: [PATCH 034/243] Analyzer: cover new analyzer with old analyzer fails --- .../03040_array_sum_and_join.reference | 5 ++++ .../0_stateless/03040_array_sum_and_join.sql | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+) create mode 100644 tests/queries/0_stateless/03040_array_sum_and_join.reference create mode 100644 tests/queries/0_stateless/03040_array_sum_and_join.sql diff --git a/tests/queries/0_stateless/03040_array_sum_and_join.reference b/tests/queries/0_stateless/03040_array_sum_and_join.reference new file mode 100644 index 00000000000..d81fd9a2f69 --- /dev/null +++ b/tests/queries/0_stateless/03040_array_sum_and_join.reference @@ -0,0 +1,5 @@ +79 name1 42.7027027027027 +62 name2 33.513513513513516 +44 name3 23.783783783783786 +[[1,2],[1,2]] +[(3,[1,2]),(4,[1,2])] diff --git a/tests/queries/0_stateless/03040_array_sum_and_join.sql b/tests/queries/0_stateless/03040_array_sum_and_join.sql new file mode 100644 index 00000000000..0084f0e4c7b --- /dev/null +++ b/tests/queries/0_stateless/03040_array_sum_and_join.sql @@ -0,0 +1,26 @@ + +select t.1 as cnt, + t.2 as name, + t.3 as percent +from ( + select arrayJoin(result) as t + from ( + select [ + (79, 'name1'), + (62, 'name2'), + (44, 'name3') + ] as data, + arraySum(arrayMap(t -> t.1, data)) as total, + arrayMap(t -> + tuple(t.1, t.2, + multiIf(total = 0, 0, t.1 > 0 and t.1 < 10, -1.0, + (toFloat32(t.1) / toFloat32(total)) * 100) + ), + data + ) as result + ) + ); + +SELECT arrayMap(x -> arrayMap(x -> (x.1), [(1, 1), (2, 2)]), [(3, 3), (4, 4)]); + +SELECT arrayMap(x -> (x.1, arrayMap(x -> (x.1), [(1, 1), (2, 2)])), [(3, 3), (4, 4)]); From a3028ed9cd4c88d8ab5bb86ab47a4ec1475df067 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Apr 2024 16:20:09 +0000 Subject: [PATCH 035/243] Automatic style fix --- .../test_trace_collector_serverwide/test.py | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_trace_collector_serverwide/test.py b/tests/integration/test_trace_collector_serverwide/test.py index 5a7bba15fd7..88d235642b9 100644 --- a/tests/integration/test_trace_collector_serverwide/test.py +++ b/tests/integration/test_trace_collector_serverwide/test.py @@ -10,6 +10,7 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", main_configs=["configs/global_profiler.xml"]) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -19,8 +20,11 @@ def start_cluster(): finally: cluster.shutdown() + def test_global_thread_profiler(start_cluster): - node1.query("CREATE TABLE t (key UInt32, value String) Engine = MergeTree() ORDER BY key") + node1.query( + "CREATE TABLE t (key UInt32, value String) Engine = MergeTree() ORDER BY key" + ) node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") node1.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") @@ -35,4 +39,11 @@ def test_global_thread_profiler(start_cluster): node1.query("SYSTEM FLUSH LOGS") - assert int(node1.query("SELECT count() FROM system.trace_log where trace_type='Real' and query_id = ''").strip()) > 0 + assert ( + int( + node1.query( + "SELECT count() FROM system.trace_log where trace_type='Real' and query_id = ''" + ).strip() + ) + > 0 + ) From 18402c6191dfa5bc4ec8a9278253f5bc241b60df Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:22:04 +0000 Subject: [PATCH 036/243] Close https://github.com/ClickHouse/ClickHouse/issues/44365 --- .../0_stateless/03040_alias_column_join.reference | 0 .../queries/0_stateless/03040_alias_column_join.sql | 13 +++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03040_alias_column_join.reference create mode 100644 tests/queries/0_stateless/03040_alias_column_join.sql diff --git a/tests/queries/0_stateless/03040_alias_column_join.reference b/tests/queries/0_stateless/03040_alias_column_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03040_alias_column_join.sql b/tests/queries/0_stateless/03040_alias_column_join.sql new file mode 100644 index 00000000000..f4ea2e5914d --- /dev/null +++ b/tests/queries/0_stateless/03040_alias_column_join.sql @@ -0,0 +1,13 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/44365 +DROP TABLE IF EXISTS 03040_test; + +CREATE TABLE 03040_test +( + id UInt64, + val String alias 'value: '||toString(id) +) ENGINE = MergeTree +ORDER BY tuple(); + +SELECT val FROM 03040_test t GROUP BY val; + +DROP TABLE IF EXISTS 03040_test; From 170f50e095f9d6076c84ba8825c08310ebd55c8e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 1 Apr 2024 09:57:10 +0200 Subject: [PATCH 037/243] More than 255 replicas in ReplicatedTableStatus --- src/Storages/MergeTree/ReplicatedTableStatus.h | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedTableStatus.h b/src/Storages/MergeTree/ReplicatedTableStatus.h index ce9ad3640f4..786a5fdb44d 100644 --- a/src/Storages/MergeTree/ReplicatedTableStatus.h +++ b/src/Storages/MergeTree/ReplicatedTableStatus.h @@ -24,8 +24,8 @@ struct ReplicatedTableStatus UInt64 log_max_index; UInt64 log_pointer; UInt64 absolute_delay; - UInt8 total_replicas; - UInt8 active_replicas; + UInt32 total_replicas; + UInt32 active_replicas; UInt64 lost_part_count; String last_queue_update_exception; /// If the error has happened fetching the info from ZooKeeper, this field will be set. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8ca061db4ec..b0f82e85ac4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7003,7 +7003,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit } res.log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); - res.total_replicas = all_replicas.size(); + res.total_replicas = UInt32(all_replicas.size()); if (get_result[1].error == Coordination::Error::ZNONODE) res.lost_part_count = 0; else From 478cabee22a66008988a0302c8b5111e59f6f70e Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 18:33:56 +0200 Subject: [PATCH 038/243] Close: https://github.com/ClickHouse/ClickHouse/issues/15411 --- .../03041_analyzer_gigachad_join.reference | 1 + .../0_stateless/03041_analyzer_gigachad_join.sql | 14 ++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03041_analyzer_gigachad_join.reference create mode 100644 tests/queries/0_stateless/03041_analyzer_gigachad_join.sql diff --git a/tests/queries/0_stateless/03041_analyzer_gigachad_join.reference b/tests/queries/0_stateless/03041_analyzer_gigachad_join.reference new file mode 100644 index 00000000000..a859a600512 --- /dev/null +++ b/tests/queries/0_stateless/03041_analyzer_gigachad_join.reference @@ -0,0 +1 @@ +123456789 111 222 diff --git a/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql new file mode 100644 index 00000000000..462e63b121b --- /dev/null +++ b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql @@ -0,0 +1,14 @@ +CREATE TABLE IF NOT EXISTS first engine = MergeTree PARTITION BY (inn, toYYYYMM(received)) ORDER BY (inn, sessionId) +AS SELECT now() AS received, '123456789' AS inn, '42' AS sessionId; + +CREATE TABLE IF NOT EXISTS second engine = MergeTree PARTITION BY (inn, toYYYYMM(received)) ORDER BY (inn, sessionId) +AS SELECT now() AS received, '123456789' AS inn, '42' AS sessionId, '111' AS serial, '222' AS reg; + +SELECT alias_first.inn, arrayFirst(t -> isNotNull(t), regInfo.1), arrayFirst(t -> isNotNull(t), regInfo.2) + FROM first AS alias_first + INNER JOIN ( + SELECT alias_second.inn, alias_second.sessionId, groupArray((serial, reg)) AS regInfo + FROM second AS alias_second + GROUP BY inn, sessionId + ) AS resp ON (alias_first.inn = resp.inn) AND (alias_first.sessionId = resp.sessionId) +WHERE if('123456789' IS NOT NULL, alias_first.inn = '123456789', 1) From a4a56a9b6c0ce7ce66006be87d8bf0ef8b660aed Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:38:18 +0000 Subject: [PATCH 039/243] Close https://github.com/ClickHouse/ClickHouse/issues/44153 --- .../03041_select_with_query_result.reference | 0 .../03041_select_with_query_result.sql | 41 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/03041_select_with_query_result.reference create mode 100644 tests/queries/0_stateless/03041_select_with_query_result.sql diff --git a/tests/queries/0_stateless/03041_select_with_query_result.reference b/tests/queries/0_stateless/03041_select_with_query_result.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03041_select_with_query_result.sql b/tests/queries/0_stateless/03041_select_with_query_result.sql new file mode 100644 index 00000000000..3edf51d635e --- /dev/null +++ b/tests/queries/0_stateless/03041_select_with_query_result.sql @@ -0,0 +1,41 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/44153 +DROP TABLE IF EXISTS parent; +DROP TABLE IF EXISTS join_table_1; +DROP TABLE IF EXISTS join_table_2; + +CREATE TABLE parent( + a_id Int64, + b_id Int64, + c_id Int64, + created_at Int64 +) +ENGINE=MergeTree() +ORDER BY (a_id, b_id, c_id, created_at); + +CREATE TABLE join_table_1( + a_id Int64, + b_id Int64 +) +ENGINE=MergeTree() +ORDER BY (a_id, b_id); + +CREATE TABLE join_table_2( + c_id Int64, + created_at Int64 +) +ENGINE=MergeTree() +ORDER BY (c_id, created_at); + +WITH with_table as ( + SELECT p.a_id, p.b_id, p.c_id FROM parent p + LEFT JOIN join_table_1 jt1 ON jt1.a_id = p.a_id AND jt1.b_id = p.b_id + LEFT JOIN join_table_2 jt2 ON jt2.c_id = p.c_id + WHERE + p.a_id = 0 AND (jt2.c_id = 0 OR p.created_at = 0) +) +SELECT p.a_id, p.b_id, COUNT(*) as f_count FROM with_table +GROUP BY p.a_id, p.b_id; + +DROP TABLE IF EXISTS parent; +DROP TABLE IF EXISTS join_table_1; +DROP TABLE IF EXISTS join_table_2; From 0a5747377bb44bee51dcf6223930c16529cb9a83 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 18:43:12 +0200 Subject: [PATCH 040/243] Close: https://github.com/ClickHouse/ClickHouse/issues/14978 --- .../03042_analyzer_alias_join.reference | 0 .../0_stateless/03042_analyzer_alias_join.sql | 20 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03042_analyzer_alias_join.reference create mode 100644 tests/queries/0_stateless/03042_analyzer_alias_join.sql diff --git a/tests/queries/0_stateless/03042_analyzer_alias_join.reference b/tests/queries/0_stateless/03042_analyzer_alias_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03042_analyzer_alias_join.sql b/tests/queries/0_stateless/03042_analyzer_alias_join.sql new file mode 100644 index 00000000000..f3341fd314a --- /dev/null +++ b/tests/queries/0_stateless/03042_analyzer_alias_join.sql @@ -0,0 +1,20 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/14978 +CREATE TABLE test1(id UInt64, t1value UInt64) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test2(id UInt64, t2value String) ENGINE=MergeTree ORDER BY tuple(); + +SELECT NULL AS t2value +FROM test1 t1 +LEFT JOIN ( + SELECT id, t2value FROM test2 +) t2 +ON t1.id=t2.id +WHERE t2.t2value='test'; + +-- workaround should work too +SELECT NULL AS _svalue +FROM test1 t1 +LEFT JOIN ( + SELECT id, t2value FROM test2 +) t2 +ON t1.id=t2.id +WHERE t2.t2value='test'; From 1fb23c64f15b70228f7b0911f4e5358c4a077b61 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:41:42 +0000 Subject: [PATCH 041/243] Close https://github.com/ClickHouse/ClickHouse/issues/42399 --- .../0_stateless/03042_not_found_column_c1.reference | 0 tests/queries/0_stateless/03042_not_found_column_c1.sql | 8 ++++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03042_not_found_column_c1.reference create mode 100644 tests/queries/0_stateless/03042_not_found_column_c1.sql diff --git a/tests/queries/0_stateless/03042_not_found_column_c1.reference b/tests/queries/0_stateless/03042_not_found_column_c1.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03042_not_found_column_c1.sql b/tests/queries/0_stateless/03042_not_found_column_c1.sql new file mode 100644 index 00000000000..8ce7dcd9d4f --- /dev/null +++ b/tests/queries/0_stateless/03042_not_found_column_c1.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/42399 + +CREATE TABLE IF NOT EXISTS t0 (c0 Int32) ENGINE = Memory() ; +CREATE TABLE t1 (c0 Int32, c1 Int32, c2 Int32) ENGINE = Memory() ; +CREATE TABLE t2 (c0 String, c1 String MATERIALIZED (c2), c2 Int32) ENGINE = Memory() ; +CREATE TABLE t3 (c0 String, c1 String, c2 String) ENGINE = Log() ; +CREATE TABLE IF NOT EXISTS t4 (c0 Int32) ENGINE = Log() ; +SELECT t3.c1, t3.c2, t1.c1, t1.c0, t2.c2, t0.c0, t1.c2, t2.c1, t4.c0 FROM t3, t0, t1, t2, t4; From 2d8f07318c06ec330c8d6e87facd387bc2b63341 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:45:21 +0000 Subject: [PATCH 042/243] Close: https://github.com/ClickHouse/ClickHouse/issues/27115 --- ...3_group_array_result_is_expected.reference | 1 + .../03043_group_array_result_is_expected.sql | 44 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/03043_group_array_result_is_expected.reference create mode 100644 tests/queries/0_stateless/03043_group_array_result_is_expected.sql diff --git a/tests/queries/0_stateless/03043_group_array_result_is_expected.reference b/tests/queries/0_stateless/03043_group_array_result_is_expected.reference new file mode 100644 index 00000000000..d43aa556dce --- /dev/null +++ b/tests/queries/0_stateless/03043_group_array_result_is_expected.reference @@ -0,0 +1 @@ +['2021-07-01','2021-07-02','2021-07-03','2021-07-04','2021-07-05','2021-07-06','2021-07-07','2021-07-08','2021-07-09','2021-07-10','2021-07-11','2021-07-12','2021-07-13','2021-07-14','2021-07-15','2021-07-16','2021-07-17','2021-07-18','2021-07-19','2021-07-20','2021-07-21','2021-07-22','2021-07-23','2021-07-24','2021-07-25','2021-07-26','2021-07-27','2021-07-28','2021-07-29'] 29 diff --git a/tests/queries/0_stateless/03043_group_array_result_is_expected.sql b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql new file mode 100644 index 00000000000..df77ca66647 --- /dev/null +++ b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql @@ -0,0 +1,44 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/27115 +drop table if exists fill_ex; + +create table fill_ex ( + eventDate Date , + storeId String +) +engine = ReplacingMergeTree() +partition by toYYYYMM(eventDate) +order by (storeId,eventDate); + +insert into fill_ex (eventDate,storeId) values ('2021-07-16','s') ('2021-07-17','ee'); + +select + groupArray(key) as keys, + count() as c +from + ( + select + *, + eventDate as key + from + ( + select + eventDate + from + ( + select + eventDate + from + fill_ex final + where + eventDate >= toDate('2021-07-01') + and eventDate Date: Tue, 2 Apr 2024 17:48:28 +0100 Subject: [PATCH 043/243] Bump From ed0522ae1a2e96e59386a7ed25ba85a774d429ad Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 18:48:00 +0200 Subject: [PATCH 044/243] Close: https://github.com/ClickHouse/ClickHouse/issues/17319 --- .../03044_analyzer_alias_join.reference | 0 .../0_stateless/03044_analyzer_alias_join.sql | 17 +++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03044_analyzer_alias_join.reference create mode 100644 tests/queries/0_stateless/03044_analyzer_alias_join.sql diff --git a/tests/queries/0_stateless/03044_analyzer_alias_join.reference b/tests/queries/0_stateless/03044_analyzer_alias_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03044_analyzer_alias_join.sql b/tests/queries/0_stateless/03044_analyzer_alias_join.sql new file mode 100644 index 00000000000..5202b57a7b1 --- /dev/null +++ b/tests/queries/0_stateless/03044_analyzer_alias_join.sql @@ -0,0 +1,17 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/17319 +CREATE TEMPORARY TABLE hits (date Date, data Float64) engine=Memory(); + +SELECT + subquery1.period AS period, + if(1=1, 0, subquery1.data1) AS data, + if(1=1, 0, subquery2.data) AS other_data +FROM +( + SELECT date AS period, data AS data1 + FROM hits +) AS subquery1 +LEFT JOIN +( + SELECT date AS period, data AS data + FROM hits +) AS subquery2 ON (subquery1.period = subquery2.period) From f1fb042be3d54d0347568abcc6e5c3358665d075 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 18:55:25 +0200 Subject: [PATCH 045/243] Close: https://github.com/ClickHouse/ClickHouse/issues/13210 --- ...3045_analyzer_alias_join_with_if.reference | 0 .../03045_analyzer_alias_join_with_if.sql | 33 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/03045_analyzer_alias_join_with_if.reference create mode 100644 tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql diff --git a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.reference b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql new file mode 100644 index 00000000000..a0546f57736 --- /dev/null +++ b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql @@ -0,0 +1,33 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/13210 +CREATE TABLE test_a_table ( + name String, + a_col String +) +Engine = MergeTree() +ORDER BY name; + +CREATE TABLE test_b_table ( + name String, + b_col String, + some_val String +) +Engine = MergeTree() +ORDER BY name; + +SELECT + b.name name, + a.a_col a_col, + b.b_col b_col, + 'N' some_val +from test_a_table a +join test_b_table b on a.name = b.name +where b.some_val = 'Y'; + +SELECT + b.name name, + a.a_col a_col, + b.b_col b_col, + if(1,'N',b.some_val) some_val +from test_a_table a +join test_b_table b on a.name = b.name +where b.some_val = 'Y'; From b19b0890becd3cf9f3d7e23744e0e477ca12e4ef Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:56:16 +0000 Subject: [PATCH 046/243] Close: https://github.com/ClickHouse/ClickHouse/issues/11813 --- .../03044_array_join_columns_in_nested_table.reference | 1 + .../0_stateless/03044_array_join_columns_in_nested_table.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/03044_array_join_columns_in_nested_table.reference create mode 100644 tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql diff --git a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.reference b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql new file mode 100644 index 00000000000..f3ec80b8a94 --- /dev/null +++ b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql @@ -0,0 +1,2 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/11813 +select 1 from (select 1 x) l join (select 1 y, [1] a) r on l.x = r.y array join r.a; From 042e612485d7083749342b8658bbb5e580da580a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 16:59:03 +0000 Subject: [PATCH 047/243] Close: https://github.com/ClickHouse/ClickHouse/issues/23053 --- ...wn_identifier_alias_substitution.reference | 0 ..._unknown_identifier_alias_substitution.sql | 20 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.reference create mode 100644 tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql diff --git a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.reference b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql new file mode 100644 index 00000000000..cadcbdc0ce5 --- /dev/null +++ b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql @@ -0,0 +1,20 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/23053 +DROP TABLE IF EXISTS repl_tbl; + +CREATE TEMPORARY TABLE repl_tbl +( + `key` UInt32, + `val_1` UInt32, + `val_2` String, + `val_3` String, + `val_4` String, + `val_5` UUID, + `ts` DateTime +) +ENGINE = ReplacingMergeTree(ts) +ORDER BY `key`; +set prefer_column_name_to_alias = 1; +INSERT INTO repl_tbl (key) SELECT number FROM numbers(10); +WITH 10 as k SELECT k as key, * FROM repl_tbl WHERE key = k; + +DROP TABLE IF EXISTS repl_tbl; From 668aa9bafd25b6c27a8aba02dd0b1d53c782fc65 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:03:40 +0000 Subject: [PATCH 048/243] Close: https://github.com/ClickHouse/ClickHouse/issues/37729 --- ...03046_column_in_block_array_join.reference | 2 + .../03046_column_in_block_array_join.sql | 37 +++++++++++++++++++ 2 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/03046_column_in_block_array_join.reference create mode 100644 tests/queries/0_stateless/03046_column_in_block_array_join.sql diff --git a/tests/queries/0_stateless/03046_column_in_block_array_join.reference b/tests/queries/0_stateless/03046_column_in_block_array_join.reference new file mode 100644 index 00000000000..f9264f7fbd3 --- /dev/null +++ b/tests/queries/0_stateless/03046_column_in_block_array_join.reference @@ -0,0 +1,2 @@ +Hello +World diff --git a/tests/queries/0_stateless/03046_column_in_block_array_join.sql b/tests/queries/0_stateless/03046_column_in_block_array_join.sql new file mode 100644 index 00000000000..9a2bb19d81e --- /dev/null +++ b/tests/queries/0_stateless/03046_column_in_block_array_join.sql @@ -0,0 +1,37 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/37729 + +DROP TABLE IF EXISTS nested_test; +DROP TABLE IF EXISTS join_test; + +CREATE TABLE nested_test +( + s String, + nest Nested + ( + x UInt64, + y UInt64 + ) +) ENGINE = MergeTree +ORDER BY s; + +CREATE TABLE join_test +( + id Int64, + y UInt64 +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO nested_test +VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []); + +INSERT INTO join_test +VALUES (1,1),(2,4),(3,20),(4,40); + +SELECT s +FROM nested_test AS t1 +ARRAY JOIN nest +INNER JOIN join_test AS t2 ON nest.y = t2.y; + +DROP TABLE IF EXISTS nested_test; +DROP TABLE IF EXISTS join_test; From 9b74e246af4eec622ca749aebbbe2985428e51f7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 19:03:54 +0200 Subject: [PATCH 049/243] Rename a method --- src/Interpreters/Cache/FileSegment.cpp | 132 ++++++++++++------------- src/Interpreters/Cache/FileSegment.h | 6 +- 2 files changed, 69 insertions(+), 69 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 0d64b602928..f56f5d3f66c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -113,7 +113,7 @@ FileSegment::Range::Range(size_t left_, size_t right_) : left(left_), right(righ FileSegment::State FileSegment::state() const { - auto lock = lockFileSegment(); + auto lk = lock(); return download_state; } @@ -130,7 +130,7 @@ String FileSegment::tryGetPath() const return metadata->getFileSegmentPath(*this); } -FileSegmentGuard::Lock FileSegment::lockFileSegment() const +FileSegmentGuard::Lock FileSegment::lock() const { ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentLockMicroseconds); return segment_guard.lock(); @@ -152,29 +152,30 @@ void FileSegment::setDownloadState(State state, const FileSegmentGuard::Lock & l size_t FileSegment::getReservedSize() const { - auto lock = lockFileSegment(); + auto lk = lock(); return reserved_size; } FileSegment::Priority::IteratorPtr FileSegment::getQueueIterator() const { - auto lock = lockFileSegment(); + auto lk = lock(); return queue_iterator; } void FileSegment::setQueueIterator(Priority::IteratorPtr iterator) { - auto lock = lockFileSegment(); - setQueueIteratorUnlocked(iterator, lock); -} - -void FileSegment::setQueueIteratorUnlocked(Priority::IteratorPtr iterator, const FileSegmentGuard::Lock &) -{ + auto lk = lock(); if (queue_iterator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Queue iterator cannot be set twice"); queue_iterator = iterator; } +void FileSegment::resetQueueIterator() +{ + auto lk = lock(); + queue_iterator.reset(); +} + size_t FileSegment::getCurrentWriteOffset() const { return range().left + downloaded_size; @@ -187,14 +188,14 @@ size_t FileSegment::getDownloadedSize() const void FileSegment::setDownloadedSize(size_t delta) { - auto lock = lockFileSegment(); + auto lk = lock(); downloaded_size += delta; assert(downloaded_size == std::filesystem::file_size(getPath())); } bool FileSegment::isDownloaded() const { - auto lock = lockFileSegment(); + auto lk = lock(); return download_state == State::DOWNLOADED; } @@ -208,8 +209,7 @@ String FileSegment::getCallerId() String FileSegment::getDownloader() const { - auto lock = lockFileSegment(); - return getDownloaderUnlocked(lock); + return getDownloaderUnlocked(lock()); } String FileSegment::getDownloaderUnlocked(const FileSegmentGuard::Lock &) const @@ -219,11 +219,11 @@ String FileSegment::getDownloaderUnlocked(const FileSegmentGuard::Lock &) const String FileSegment::getOrSetDownloader() { - auto lock = lockFileSegment(); + auto lk = lock(); - assertNotDetachedUnlocked(lock); + assertNotDetachedUnlocked(lk); - auto current_downloader = getDownloaderUnlocked(lock); + auto current_downloader = getDownloaderUnlocked(lk); if (current_downloader.empty()) { @@ -233,7 +233,7 @@ String FileSegment::getOrSetDownloader() return "notAllowed:" + stateToString(download_state); current_downloader = downloader_id = caller_id; - setDownloadState(State::DOWNLOADING, lock); + setDownloadState(State::DOWNLOADING, lk); chassert(key_metadata.lock()); } @@ -257,15 +257,15 @@ void FileSegment::resetDownloadingStateUnlocked(const FileSegmentGuard::Lock & l void FileSegment::resetDownloader() { - auto lock = lockFileSegment(); + auto lk = lock(); SCOPE_EXIT({ cv.notify_all(); }); - assertNotDetachedUnlocked(lock); - assertIsDownloaderUnlocked("resetDownloader", lock); + assertNotDetachedUnlocked(lk); + assertIsDownloaderUnlocked("resetDownloader", lk); - resetDownloadingStateUnlocked(lock); - resetDownloaderUnlocked(lock); + resetDownloadingStateUnlocked(lk); + resetDownloaderUnlocked(lk); } void FileSegment::resetDownloaderUnlocked(const FileSegmentGuard::Lock &) @@ -294,8 +294,8 @@ void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, cons bool FileSegment::isDownloader() const { - auto lock = lockFileSegment(); - return isDownloaderUnlocked(lock); + auto lk = lock(); + return isDownloaderUnlocked(lk); } bool FileSegment::isDownloaderUnlocked(const FileSegmentGuard::Lock & lock) const @@ -305,21 +305,21 @@ bool FileSegment::isDownloaderUnlocked(const FileSegmentGuard::Lock & lock) cons FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() { - auto lock = lockFileSegment(); - assertIsDownloaderUnlocked("getRemoteFileReader", lock); + auto lk = lock(); + assertIsDownloaderUnlocked("getRemoteFileReader", lk); return remote_file_reader; } void FileSegment::resetRemoteFileReader() { - auto lock = lockFileSegment(); - assertIsDownloaderUnlocked("resetRemoteFileReader", lock); + auto lk = lock(); + assertIsDownloaderUnlocked("resetRemoteFileReader", lk); remote_file_reader.reset(); } FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - auto lock = lockFileSegment(); + auto lk = lock(); if (remote_file_reader && (download_state == State::DOWNLOADED || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION)) { @@ -330,8 +330,8 @@ FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) { - auto lock = lockFileSegment(); - assertIsDownloaderUnlocked("setRemoteFileReader", lock); + auto lk = lock(); + assertIsDownloaderUnlocked("setRemoteFileReader", lk); if (remote_file_reader) throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote file reader already exists"); @@ -347,9 +347,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); { - auto lock = lockFileSegment(); - assertIsDownloaderUnlocked("write", lock); - assertNotDetachedUnlocked(lock); + auto lk = lock(); + assertIsDownloaderUnlocked("write", lk); + assertNotDetachedUnlocked(lk); } const auto file_segment_path = getPath(); @@ -408,10 +408,10 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const int code = e.getErrno(); const bool is_no_space_left_error = code == /* No space left on device */28 || code == /* Quota exceeded */122; - auto lock = lockFileSegment(); + auto lk = lock(); - e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lock))); - setDownloadFailedUnlocked(lock); + e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lk))); + setDownloadFailedUnlocked(lk); if (downloaded_size == 0 && fs::exists(file_segment_path)) { @@ -434,9 +434,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset) } catch (Exception & e) { - auto lock = lockFileSegment(); - e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lock))); - setDownloadFailedUnlocked(lock); + auto lk = lock(); + e.addMessage(fmt::format("{}, current cache state: {}", e.what(), getInfoForLogUnlocked(lk))); + setDownloadFailedUnlocked(lk); throw; } @@ -449,7 +449,7 @@ FileSegment::State FileSegment::wait(size_t offset) span.addAttribute("clickhouse.key", key().toString()); span.addAttribute("clickhouse.offset", offset); - auto lock = lockFileSegment(); + auto lk = lock(); if (downloader_id.empty() || offset < getCurrentWriteOffset()) return download_state; @@ -462,10 +462,10 @@ FileSegment::State FileSegment::wait(size_t offset) LOG_TEST(log, "{} waiting on: {}, current downloader: {}", getCallerId(), range().toString(), downloader_id); ProfileEventTimeIncrement watch(ProfileEvents::FileSegmentWaitMicroseconds); - chassert(!getDownloaderUnlocked(lock).empty()); - chassert(!isDownloaderUnlocked(lock)); + chassert(!getDownloaderUnlocked(lk).empty()); + chassert(!isDownloaderUnlocked(lk)); - [[maybe_unused]] const auto ok = cv.wait_for(lock, std::chrono::seconds(60), [&, this]() + [[maybe_unused]] const auto ok = cv.wait_for(lk, std::chrono::seconds(60), [&, this]() { return download_state != State::DOWNLOADING || offset < getCurrentWriteOffset(); }); @@ -511,10 +511,10 @@ bool FileSegment::reserve(size_t size_to_reserve, size_t lock_wait_timeout_milli bool is_file_segment_size_exceeded; { - auto lock = lockFileSegment(); + auto lk = lock(); - assertNotDetachedUnlocked(lock); - assertIsDownloaderUnlocked("reserve", lock); + assertNotDetachedUnlocked(lk); + assertIsDownloaderUnlocked("reserve", lk); expected_downloaded_size = getDownloadedSize(); @@ -557,7 +557,7 @@ bool FileSegment::reserve(size_t size_to_reserve, size_t lock_wait_timeout_milli bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user, lock_wait_timeout_milliseconds); if (!reserved) - setDownloadFailedUnlocked(lockFileSegment()); + setDownloadFailedUnlocked(lock()); return reserved; } @@ -582,8 +582,8 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &) void FileSegment::setDownloadFailed() { - auto lock = lockFileSegment(); - setDownloadFailedUnlocked(lock); + auto lk = lock(); + setDownloadFailedUnlocked(lk); } void FileSegment::setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock) @@ -605,22 +605,22 @@ void FileSegment::setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock) void FileSegment::completePartAndResetDownloader() { - auto lock = lockFileSegment(); + auto lk = lock(); SCOPE_EXIT({ cv.notify_all(); }); - assertNotDetachedUnlocked(lock); - assertIsDownloaderUnlocked("completePartAndResetDownloader", lock); + assertNotDetachedUnlocked(lk); + assertIsDownloaderUnlocked("completePartAndResetDownloader", lk); chassert(download_state == State::DOWNLOADING || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); if (download_state == State::DOWNLOADING) - resetDownloadingStateUnlocked(lock); + resetDownloadingStateUnlocked(lk); - resetDownloaderUnlocked(lock); + resetDownloaderUnlocked(lk); - LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(lock)); + LOG_TEST(log, "Complete batch. ({})", getInfoForLogUnlocked(lk)); } void FileSegment::complete() @@ -640,7 +640,7 @@ void FileSegment::complete() throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot complete file segment: {}", getInfoForLog()); } - auto segment_lock = lockFileSegment(); + auto segment_lock = lock(); if (isCompleted(false)) return; @@ -756,8 +756,8 @@ void FileSegment::complete() String FileSegment::getInfoForLog() const { - auto lock = lockFileSegment(); - return getInfoForLogUnlocked(lock); + auto lk = lock(); + return getInfoForLogUnlocked(lk); } String FileSegment::getInfoForLogUnlocked(const FileSegmentGuard::Lock &) const @@ -799,7 +799,7 @@ String FileSegment::stateToString(FileSegment::State state) bool FileSegment::assertCorrectness() const { - return assertCorrectnessUnlocked(lockFileSegment()); + return assertCorrectnessUnlocked(lock()); } bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) const @@ -869,8 +869,8 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) void FileSegment::assertNotDetached() const { - auto lock = lockFileSegment(); - assertNotDetachedUnlocked(lock); + auto lk = lock(); + assertNotDetachedUnlocked(lk); } void FileSegment::assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock) const @@ -887,7 +887,7 @@ void FileSegment::assertNotDetachedUnlocked(const FileSegmentGuard::Lock & lock) FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment) { - auto lock = file_segment->lockFileSegment(); + auto lock = file_segment->lock(); auto key_metadata = file_segment->tryGetKeyMetadata(); return Info{ .key = file_segment->key(), @@ -910,7 +910,7 @@ FileSegment::Info FileSegment::getInfo(const FileSegmentPtr & file_segment) bool FileSegment::isDetached() const { - auto lock = lockFileSegment(); + auto lk = lock(); return download_state == State::DETACHED; } @@ -926,7 +926,7 @@ bool FileSegment::isCompleted(bool sync) const if (is_completed_state()) return true; - auto lock = lockFileSegment(); + auto lk = lock(); return is_completed_state(); } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 12046e59bd6..7793c50d2d5 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -171,12 +171,13 @@ public: * ========== Methods used by `cache` ======================== */ - FileSegmentGuard::Lock lock() const { return segment_guard.lock(); } + FileSegmentGuard::Lock lock() const; Priority::IteratorPtr getQueueIterator() const; void setQueueIterator(Priority::IteratorPtr iterator); - void setQueueIteratorUnlocked(Priority::IteratorPtr iterator, const FileSegmentGuard::Lock &); + + void resetQueueIterator(); KeyMetadataPtr tryGetKeyMetadata() const; @@ -242,7 +243,6 @@ private: bool assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) const; LockedKeyPtr lockKeyMetadata(bool assert_exists = true) const; - FileSegmentGuard::Lock lockFileSegment() const; String tryGetPath() const; From 607a9d761babcae14c9d4ff64c8753900464e2b8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Apr 2024 19:04:02 +0200 Subject: [PATCH 050/243] Fix --- src/Interpreters/Cache/EvictionCandidates.cpp | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index 4ca6aeea22e..5fa2b337e64 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -67,11 +67,11 @@ void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & loc { for (const auto & candidate : key_candidates.candidates) { - const auto & file_segment = candidate->file_segment; - auto file_segment_lock = file_segment->lock(); + auto queue_iterator = candidate->getQueueIterator(); + queue_iterator->invalidate(); - candidate->getQueueIterator()->remove(lock); - file_segment->setQueueIteratorUnlocked(nullptr, file_segment_lock); + candidate->file_segment->resetQueueIterator(); + queue_iterator->remove(lock); } } removed_queue_entries = true; @@ -101,10 +101,14 @@ void EvictionCandidates::evict() { auto & candidate = key_candidates.candidates.back(); chassert(candidate->releasable()); - const auto segment = candidate->file_segment; - auto iterator = segment->getQueueIterator(); - chassert(iterator); + + IFileCachePriority::IteratorPtr iterator; + if (!removed_queue_entries) + { + iterator = segment->getQueueIterator(); + chassert(iterator); + } ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); @@ -133,7 +137,7 @@ void EvictionCandidates::evict() /// it was freed in favour of some reserver, so we can make it visibly /// free only for that particular reserver. - if (!removed_queue_entries) + if (iterator) queue_entries_to_invalidate.push_back(iterator); key_candidates.candidates.pop_back(); From 2fd23a15653cef89400a914824e44a27c823beeb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 19:05:39 +0200 Subject: [PATCH 051/243] Close: https://github.com/ClickHouse/ClickHouse/issues/9954 --- .../03047_analyzer_alias_join.reference | 2 ++ .../0_stateless/03047_analyzer_alias_join.sql | 30 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/03047_analyzer_alias_join.reference create mode 100644 tests/queries/0_stateless/03047_analyzer_alias_join.sql diff --git a/tests/queries/0_stateless/03047_analyzer_alias_join.reference b/tests/queries/0_stateless/03047_analyzer_alias_join.reference new file mode 100644 index 00000000000..3bb92083849 --- /dev/null +++ b/tests/queries/0_stateless/03047_analyzer_alias_join.reference @@ -0,0 +1,2 @@ +1 1 \N +1 2 1 diff --git a/tests/queries/0_stateless/03047_analyzer_alias_join.sql b/tests/queries/0_stateless/03047_analyzer_alias_join.sql new file mode 100644 index 00000000000..a6f81516430 --- /dev/null +++ b/tests/queries/0_stateless/03047_analyzer_alias_join.sql @@ -0,0 +1,30 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/9954 +SELECT + 1 AS value, + * +FROM +( + SELECT 1 AS key +) AS l +LEFT JOIN +( + SELECT + 2 AS key, + 1 AS value +) AS r USING (key) +SETTINGS join_use_nulls = 1; + +SELECT + 1 AS value, + * +FROM +( + SELECT 2 AS key +) AS l +LEFT JOIN +( + SELECT + 2 AS key, + 1 AS value +) AS r USING (key) +SETTINGS join_use_nulls = 1 From a19bce649aef3798972e8fe4129bbc0ac4dbdd84 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:07:55 +0000 Subject: [PATCH 052/243] Close: https://github.com/ClickHouse/ClickHouse/issues/32639 --- .../03047_group_by_field_identified_aggregation.reference | 1 + .../03047_group_by_field_identified_aggregation.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03047_group_by_field_identified_aggregation.reference create mode 100644 tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql diff --git a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.reference b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.reference new file mode 100644 index 00000000000..556d825db42 --- /dev/null +++ b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.reference @@ -0,0 +1 @@ +2 1 diff --git a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql new file mode 100644 index 00000000000..e1363ea4dda --- /dev/null +++ b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/32639 + +SELECT 0 AND id ? 1 : 2 AS a, sum(id) FROM (SELECT 1 AS id) GROUP BY a; From a3f973662289a7ad218f8bf7994b207b4891e64f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:13:13 +0000 Subject: [PATCH 053/243] Close: https://github.com/ClickHouse/ClickHouse/issues/41964 --- ...48_not_found_column_xxx_in_block.reference | 0 .../03048_not_found_column_xxx_in_block.sql | 66 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 tests/queries/0_stateless/03048_not_found_column_xxx_in_block.reference create mode 100644 tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql diff --git a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.reference b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql new file mode 100644 index 00000000000..25f88050eb1 --- /dev/null +++ b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql @@ -0,0 +1,66 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/41964 + +DROP TABLE IF EXISTS ab_12_aaa; +DROP TABLE IF EXISTS ab_12_bbb; + +CREATE TABLE ab_12_aaa +( + `id` String, + `subid` Int32, + `prodcat` String, + `prodtype` String, + `quality` String, + `m1` Float64, + `m2` Float64, + `r1` Float64, + `r2` Float64, + `d1` Float64, + `d2` Float64, + `pcs` Float64, + `qty` Float64, + `amt` Float64, + `amts` Float64, + `prc` Float64, + `prcs` Float64, + `suqty` Float64, + `suamt` Float64, + `_year` String +) +ENGINE = MergeTree +ORDER BY (_year, prodcat, prodtype, quality, d1, id) +SETTINGS index_granularity = 8192; + +CREATE TABLE ab_12_bbb +( + `id` String, + `sales_type` String, + `date` Date32, + `o1` String, + `o2` String, + `o3` String, + `o4` String, + `o5` String, + `short` String, + `a1` String, + `a2` String, + `a3` String, + `idx` String, + `a4` String, + `ctx` String, + `_year` String, + `theyear` UInt16 MATERIALIZED toYear(`date`), + `themonth` UInt8 MATERIALIZED toMonth(`date`), + `theweek` UInt8 MATERIALIZED toISOWeek(`date`) +) +ENGINE = MergeTree +ORDER BY (theyear, themonth, _year, id, sales_type, date) +SETTINGS index_granularity = 8192; + +SELECT * +FROM ab_12_aaa aa +LEFT JOIN ab_12_bbb bb +ON bb.id = aa.id AND bb.`_year` = aa.`_year` +WHERE bb.theyear >= 2019; + +DROP TABLE IF EXISTS ab_12_aaa; +DROP TABLE IF EXISTS ab_12_bbb; From 14820032799b252091910a3fd999866078976761 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 2 Apr 2024 19:16:49 +0200 Subject: [PATCH 054/243] Close: https://github.com/ClickHouse/ClickHouse/issues/7520 --- .../03049_analyzer_group_by_alias.reference | 7 ++++++ .../03049_analyzer_group_by_alias.sql | 22 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/03049_analyzer_group_by_alias.reference create mode 100644 tests/queries/0_stateless/03049_analyzer_group_by_alias.sql diff --git a/tests/queries/0_stateless/03049_analyzer_group_by_alias.reference b/tests/queries/0_stateless/03049_analyzer_group_by_alias.reference new file mode 100644 index 00000000000..dac5487d445 --- /dev/null +++ b/tests/queries/0_stateless/03049_analyzer_group_by_alias.reference @@ -0,0 +1,7 @@ +1 5 +2 4 +1 +2 +1 +1 +2 diff --git a/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql new file mode 100644 index 00000000000..67df48e0cad --- /dev/null +++ b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql @@ -0,0 +1,22 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/7520 +CREATE TABLE test (`a` UInt32, `b` UInt32) ENGINE = Memory; + +INSERT INTO test VALUES (1,2), (1,3), (2,4); + +-- 1 5 +-- 2 4 + +WITH + a as key +SELECT + a as k1, + sum(b) as k2 +FROM + test +GROUP BY + key +ORDER BY k1, k2; + +WITH a as key SELECT key as k1 FROM test GROUP BY key ORDER BY key; + +WITH a as key SELECT key as k1 FROM test ORDER BY key; From 167c993503f62ea2e9b91816a17e25ae48aaf98d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:17:10 +0000 Subject: [PATCH 055/243] Close: https://github.com/ClickHouse/ClickHouse/issues/54317 --- ...9_unknown_identifier_materialized_column.sql | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql new file mode 100644 index 00000000000..2f12799addb --- /dev/null +++ b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql @@ -0,0 +1,17 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/54317 +DROP DATABASE IF EXISTS 03049_database; +DROP TABLE IF EXISTS 03049_database.l; +DROP TABLE IF EXISTS 03049_database.r; + +CREATE DATABASE 03049_database; +USE 03049_database; + +CREATE TABLE l (y String) Engine Memory; +CREATE TABLE r (d Date, y String, ty UInt16 MATERIALIZED toYear(d)) Engine Memory; +select * from l L left join r R on L.y = R.y where R.ty >= 2019; +select * from l left join r on l.y = r.y where r.ty >= 2019; +select * from 03049_database.l left join 03049_database.r on l.y = r.y where r.ty >= 2019; + +DROP DATABASE IF EXISTS 03049_database; +DROP TABLE IF EXISTS 03049_database.l; +DROP TABLE IF EXISTS 03049_database.r; From e7e20acc5b13d4a84754d42a38356f3b009531c0 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Tue, 2 Apr 2024 10:22:57 -0700 Subject: [PATCH 056/243] Bump From 51b2a8bce250867a9c54408e2d4f8e72887d68ab Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:23:20 +0000 Subject: [PATCH 057/243] Close: 111 --- tests/queries/0_stateless/03050_select_one_one_one.reference | 2 ++ tests/queries/0_stateless/03050_select_one_one_one.sql | 3 +++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03050_select_one_one_one.reference create mode 100644 tests/queries/0_stateless/03050_select_one_one_one.sql diff --git a/tests/queries/0_stateless/03050_select_one_one_one.reference b/tests/queries/0_stateless/03050_select_one_one_one.reference new file mode 100644 index 00000000000..85e6138dc5d --- /dev/null +++ b/tests/queries/0_stateless/03050_select_one_one_one.reference @@ -0,0 +1,2 @@ +1 1 1 +1 1 1 diff --git a/tests/queries/0_stateless/03050_select_one_one_one.sql b/tests/queries/0_stateless/03050_select_one_one_one.sql new file mode 100644 index 00000000000..eee973fe936 --- /dev/null +++ b/tests/queries/0_stateless/03050_select_one_one_one.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/36973 +SELECT 1, 1, 1; +SELECT * FROM (SELECT 1, 1, 1); From d9048766933a11b283e0e2345d6bc6c9d0a57699 Mon Sep 17 00:00:00 2001 From: justindeguzman Date: Tue, 2 Apr 2024 10:23:50 -0700 Subject: [PATCH 058/243] Bump From acf1fb3a6cf3e79899515641848e3214db0c36f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:25:13 +0000 Subject: [PATCH 059/243] Forgotten file --- .../03049_unknown_identifier_materialized_column.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/queries/0_stateless/03049_unknown_identifier_materialized_column.reference diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.reference b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.reference new file mode 100644 index 00000000000..e69de29bb2d From 9ffe6a4640e6197d653c1c5073a98cc659d3f8f6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:26:18 +0000 Subject: [PATCH 060/243] Close: https://github.com/ClickHouse/ClickHouse/issues/40955 --- tests/queries/0_stateless/03051_many_ctes.reference | 4 ++++ tests/queries/0_stateless/03051_many_ctes.sql | 5 +++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03051_many_ctes.reference create mode 100644 tests/queries/0_stateless/03051_many_ctes.sql diff --git a/tests/queries/0_stateless/03051_many_ctes.reference b/tests/queries/0_stateless/03051_many_ctes.reference new file mode 100644 index 00000000000..487b1165348 --- /dev/null +++ b/tests/queries/0_stateless/03051_many_ctes.reference @@ -0,0 +1,4 @@ +2 +2 +2 +2 diff --git a/tests/queries/0_stateless/03051_many_ctes.sql b/tests/queries/0_stateless/03051_many_ctes.sql new file mode 100644 index 00000000000..412a1e6b544 --- /dev/null +++ b/tests/queries/0_stateless/03051_many_ctes.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/40955 +WITH toInt64(2) AS new_x SELECT new_x AS x FROM (SELECT 1 AS x) t; +WITH toInt64(2) AS new_x SELECT * replace(new_x as x) FROM (SELECT 1 AS x) t; +SELECT 2 AS x FROM (SELECT 1 AS x) t; +SELECT * replace(2 as x) FROM (SELECT 1 AS x) t; From 9673e0a9fcf3006f791da2c0c28cfa4d7b5ac9c7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 17:29:01 +0000 Subject: [PATCH 061/243] Just a test --- ...3052_query_hash_includes_aliases.reference | 2 ++ .../03052_query_hash_includes_aliases.sql | 31 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/03052_query_hash_includes_aliases.reference create mode 100644 tests/queries/0_stateless/03052_query_hash_includes_aliases.sql diff --git a/tests/queries/0_stateless/03052_query_hash_includes_aliases.reference b/tests/queries/0_stateless/03052_query_hash_includes_aliases.reference new file mode 100644 index 00000000000..570d7be9c4b --- /dev/null +++ b/tests/queries/0_stateless/03052_query_hash_includes_aliases.reference @@ -0,0 +1,2 @@ +(1,1) (1,0) +(3,4) (3,11) diff --git a/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql new file mode 100644 index 00000000000..e76108c7842 --- /dev/null +++ b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql @@ -0,0 +1,31 @@ +-- https://github.com/ClickHouse/ClickHouse/pull/40065 + +SELECT +( + SELECT + 1 AS number, + number + FROM numbers(1) +) AS s, +( + SELECT + 1, + number + FROM numbers(1) +) AS s2; + +SELECT +( + SELECT + 1 + 2 AS number, + 1 + number AS b + FROM system.numbers + LIMIT 10, 1 +), +( + SELECT + 1 + 2 AS number2, + 1 + number AS b + FROM system.numbers + LIMIT 10, 1 +); From 593abc4037ecc420cd713f62bc166738071ec57b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 19:36:34 +0200 Subject: [PATCH 062/243] Update 03047_analyzer_alias_join.sql --- tests/queries/0_stateless/03047_analyzer_alias_join.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03047_analyzer_alias_join.sql b/tests/queries/0_stateless/03047_analyzer_alias_join.sql index a6f81516430..ef8c067bb72 100644 --- a/tests/queries/0_stateless/03047_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03047_analyzer_alias_join.sql @@ -1,4 +1,3 @@ --- https://github.com/ClickHouse/ClickHouse/issues/9954 SELECT 1 AS value, * From 4fc09fb746ef7902d01d54cf9d2f538d168ec9b9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 20:03:56 +0200 Subject: [PATCH 063/243] Update 03034_normalized_ast.sql --- tests/queries/0_stateless/03034_normalized_ast.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03034_normalized_ast.sql b/tests/queries/0_stateless/03034_normalized_ast.sql index ff6f8da118c..fe801cd8cf4 100644 --- a/tests/queries/0_stateless/03034_normalized_ast.sql +++ b/tests/queries/0_stateless/03034_normalized_ast.sql @@ -3,5 +3,6 @@ SELECT concat(database, table) AS name, count() FROM clusterAllReplicas(default, system.tables) +WHERE database=currentDatabase() GROUP BY name FORMAT Null; From 69b393cf3eef9d22d180af3e36debf5128b4c27c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Apr 2024 20:35:50 +0200 Subject: [PATCH 064/243] Fxi standalone build --- src/Common/ThreadStatus.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 2d33c0ac021..550cb76e736 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -1,5 +1,6 @@ #pragma once +#include "config.h" #include #include #include @@ -307,7 +308,12 @@ public: void flushUntrackedMemory(); +#ifdef CLICKHOUSE_KEEPER_STANDALONE_BUILD + void initGlobalProfiler(UInt64, UInt64) {} +#else void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); +#endif + private: void applyGlobalSettings(); void applyQuerySettings(); From fb2163ed7c2159b3444a2c6ceec459ee77a08322 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 21:24:13 +0200 Subject: [PATCH 065/243] Apply suggestions from code review --- tests/queries/0_stateless/03034_normalized_ast.sql | 2 +- .../queries/0_stateless/03035_alias_column_bug_distributed.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03034_normalized_ast.sql b/tests/queries/0_stateless/03034_normalized_ast.sql index fe801cd8cf4..dd109eb5113 100644 --- a/tests/queries/0_stateless/03034_normalized_ast.sql +++ b/tests/queries/0_stateless/03034_normalized_ast.sql @@ -2,7 +2,7 @@ SELECT concat(database, table) AS name, count() -FROM clusterAllReplicas(default, system.tables) +FROM clusterAllReplicas(test_shard_localhost, system.tables) WHERE database=currentDatabase() GROUP BY name FORMAT Null; diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql index fb459b3289b..5fd1f397425 100644 --- a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql +++ b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql @@ -11,7 +11,7 @@ ORDER BY src; CREATE TABLE alias_bug_dist AS alias_bug -ENGINE = Distributed('default', currentDatabase(), 'alias_bug', rand()); +ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'alias_bug', rand()); INSERT INTO alias_bug VALUES ('SOURCE1'); From 81f210be9328d617e1083283cbd3b654856120fc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 21:53:03 +0200 Subject: [PATCH 066/243] Update tests/queries/0_stateless/03035_alias_column_bug_distributed.sql --- .../queries/0_stateless/03035_alias_column_bug_distributed.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql index 5fd1f397425..3a7b4890bf0 100644 --- a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql +++ b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql @@ -34,7 +34,7 @@ ORDER BY src; CREATE TABLE alias_bug_dist AS alias_bug -ENGINE = Distributed('default', currentDatabase(), 'alias_bug', rand()); +ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'alias_bug', rand()); -- Unknown identifier SELECT CAST(123, 'String') AS src,theAlias FROM alias_bug_dist LIMIT 1 FORMAT Null; From 606058c1ca489f8fcc77ade96d5d1e39573a0628 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 2 Apr 2024 13:29:07 +0000 Subject: [PATCH 067/243] Consolidate SQL compat alias docs into data type docs + improve sidebar order --- .../data-types/aggregatefunction.md | 2 +- docs/en/sql-reference/data-types/array.md | 2 +- docs/en/sql-reference/data-types/boolean.md | 2 +- docs/en/sql-reference/data-types/date.md | 2 +- docs/en/sql-reference/data-types/date32.md | 2 +- docs/en/sql-reference/data-types/datetime.md | 2 +- .../en/sql-reference/data-types/datetime64.md | 2 +- docs/en/sql-reference/data-types/decimal.md | 2 +- docs/en/sql-reference/data-types/enum.md | 2 +- .../sql-reference/data-types/fixedstring.md | 4 +-- docs/en/sql-reference/data-types/float.md | 2 +- docs/en/sql-reference/data-types/geo.md | 4 +-- docs/en/sql-reference/data-types/index.md | 4 +-- docs/en/sql-reference/data-types/int-uint.md | 2 +- docs/en/sql-reference/data-types/ipv4.md | 2 +- docs/en/sql-reference/data-types/ipv6.md | 2 +- docs/en/sql-reference/data-types/json.md | 2 +- .../data-types/lowcardinality.md | 6 ++-- docs/en/sql-reference/data-types/map.md | 8 +++--- .../data-types/multiword-types.md | 27 ------------------ docs/en/sql-reference/data-types/nullable.md | 4 +-- .../data-types/simpleaggregatefunction.md | 2 ++ docs/en/sql-reference/data-types/string.md | 4 +-- docs/en/sql-reference/data-types/tuple.md | 4 +-- docs/en/sql-reference/data-types/uuid.md | 2 +- docs/en/sql-reference/data-types/variant.md | 6 ++-- .../data-types/multiword-types.md | 28 ------------------- .../data-types/multiword-types.mdx | 10 ------- 28 files changed, 39 insertions(+), 102 deletions(-) delete mode 100644 docs/en/sql-reference/data-types/multiword-types.md delete mode 100644 docs/ru/sql-reference/data-types/multiword-types.md delete mode 100644 docs/zh/sql-reference/data-types/multiword-types.mdx diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index fe6d7ebe0dc..87511a505dc 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/aggregatefunction -sidebar_position: 53 +sidebar_position: 46 sidebar_label: AggregateFunction --- diff --git a/docs/en/sql-reference/data-types/array.md b/docs/en/sql-reference/data-types/array.md index 0ee7c8de93c..e5a8ce5d18b 100644 --- a/docs/en/sql-reference/data-types/array.md +++ b/docs/en/sql-reference/data-types/array.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/array -sidebar_position: 52 +sidebar_position: 32 sidebar_label: Array(T) --- diff --git a/docs/en/sql-reference/data-types/boolean.md b/docs/en/sql-reference/data-types/boolean.md index 70abf767a41..4c59bd947de 100644 --- a/docs/en/sql-reference/data-types/boolean.md +++ b/docs/en/sql-reference/data-types/boolean.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/boolean -sidebar_position: 43 +sidebar_position: 22 sidebar_label: Boolean --- diff --git a/docs/en/sql-reference/data-types/date.md b/docs/en/sql-reference/data-types/date.md index 26e4610aec7..7adee3bbf3c 100644 --- a/docs/en/sql-reference/data-types/date.md +++ b/docs/en/sql-reference/data-types/date.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/date -sidebar_position: 47 +sidebar_position: 12 sidebar_label: Date --- diff --git a/docs/en/sql-reference/data-types/date32.md b/docs/en/sql-reference/data-types/date32.md index 38a07cd817d..a08c931b7fc 100644 --- a/docs/en/sql-reference/data-types/date32.md +++ b/docs/en/sql-reference/data-types/date32.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/date32 -sidebar_position: 48 +sidebar_position: 14 sidebar_label: Date32 --- diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 1adff18f598..889bc682d91 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/datetime -sidebar_position: 48 +sidebar_position: 16 sidebar_label: DateTime --- diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 504d0e2b0a6..ef452a723e6 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/datetime64 -sidebar_position: 49 +sidebar_position: 18 sidebar_label: DateTime64 --- diff --git a/docs/en/sql-reference/data-types/decimal.md b/docs/en/sql-reference/data-types/decimal.md index 2b32e72a28f..dfdefdff5a5 100644 --- a/docs/en/sql-reference/data-types/decimal.md +++ b/docs/en/sql-reference/data-types/decimal.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/decimal -sidebar_position: 42 +sidebar_position: 6 sidebar_label: Decimal --- diff --git a/docs/en/sql-reference/data-types/enum.md b/docs/en/sql-reference/data-types/enum.md index 02e73a0360e..ccfeb7f3416 100644 --- a/docs/en/sql-reference/data-types/enum.md +++ b/docs/en/sql-reference/data-types/enum.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/enum -sidebar_position: 50 +sidebar_position: 20 sidebar_label: Enum --- diff --git a/docs/en/sql-reference/data-types/fixedstring.md b/docs/en/sql-reference/data-types/fixedstring.md index a56b3fccbc1..0316df7fe34 100644 --- a/docs/en/sql-reference/data-types/fixedstring.md +++ b/docs/en/sql-reference/data-types/fixedstring.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/data-types/fixedstring -sidebar_position: 45 +sidebar_position: 10 sidebar_label: FixedString(N) --- -# FixedString +# FixedString(N) A fixed-length string of `N` bytes (neither characters nor code points). diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index be7b2a7fcd8..23131d5b4fe 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/float -sidebar_position: 41 +sidebar_position: 4 sidebar_label: Float32, Float64 --- diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 1d37b829dd5..7e3c32b3451 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -1,8 +1,8 @@ --- slug: /en/sql-reference/data-types/geo -sidebar_position: 62 +sidebar_position: 54 sidebar_label: Geo -title: "Geo Data Types" +title: "Geometric" --- ClickHouse supports data types for representing geographical objects — locations, lands, etc. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index ffd063590fa..fcb0b60d022 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/data-types/ sidebar_label: List of data types -sidebar_position: 37 +sidebar_position: 1 --- -# ClickHouse Data Types +# Data Types in ClickHouse ClickHouse can store various kinds of data in table cells. This section describes the supported data types and special considerations for using and/or implementing them if any. diff --git a/docs/en/sql-reference/data-types/int-uint.md b/docs/en/sql-reference/data-types/int-uint.md index 520454a859f..52d2982de19 100644 --- a/docs/en/sql-reference/data-types/int-uint.md +++ b/docs/en/sql-reference/data-types/int-uint.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/int-uint -sidebar_position: 40 +sidebar_position: 2 sidebar_label: UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256 --- diff --git a/docs/en/sql-reference/data-types/ipv4.md b/docs/en/sql-reference/data-types/ipv4.md index 288806f47b3..637ed543e08 100644 --- a/docs/en/sql-reference/data-types/ipv4.md +++ b/docs/en/sql-reference/data-types/ipv4.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/ipv4 -sidebar_position: 59 +sidebar_position: 28 sidebar_label: IPv4 --- diff --git a/docs/en/sql-reference/data-types/ipv6.md b/docs/en/sql-reference/data-types/ipv6.md index 97959308b58..642a7db81fc 100644 --- a/docs/en/sql-reference/data-types/ipv6.md +++ b/docs/en/sql-reference/data-types/ipv6.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/ipv6 -sidebar_position: 60 +sidebar_position: 30 sidebar_label: IPv6 --- diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index fd548a0d5a2..39e37abad82 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/json -sidebar_position: 54 +sidebar_position: 26 sidebar_label: JSON --- diff --git a/docs/en/sql-reference/data-types/lowcardinality.md b/docs/en/sql-reference/data-types/lowcardinality.md index db10103282d..133ac2bd72e 100644 --- a/docs/en/sql-reference/data-types/lowcardinality.md +++ b/docs/en/sql-reference/data-types/lowcardinality.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/data-types/lowcardinality -sidebar_position: 51 -sidebar_label: LowCardinality +sidebar_position: 42 +sidebar_label: LowCardinality(T) --- -# LowCardinality +# LowCardinality(T) Changes the internal representation of other data types to be dictionary-encoded. diff --git a/docs/en/sql-reference/data-types/map.md b/docs/en/sql-reference/data-types/map.md index e0c8b98f9f8..2c734969afc 100644 --- a/docs/en/sql-reference/data-types/map.md +++ b/docs/en/sql-reference/data-types/map.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/data-types/map -sidebar_position: 65 -sidebar_label: Map(key, value) +sidebar_position: 36 +sidebar_label: Map(K, V) --- -# Map(key, value) +# Map(K, V) -`Map(key, value)` data type stores `key:value` pairs. +`Map(K, V)` data type stores `key:value` pairs. **Parameters** diff --git a/docs/en/sql-reference/data-types/multiword-types.md b/docs/en/sql-reference/data-types/multiword-types.md deleted file mode 100644 index ebbe1d84544..00000000000 --- a/docs/en/sql-reference/data-types/multiword-types.md +++ /dev/null @@ -1,27 +0,0 @@ ---- -slug: /en/sql-reference/data-types/multiword-types -sidebar_position: 61 -sidebar_label: Multiword Type Names -title: "Multiword Types" ---- - -When creating tables, you can use data types with a name consisting of several words. This is implemented for better SQL compatibility. - -## Multiword Types Support - -| Multiword types | Simple types | -|----------------------------------|--------------------------------------------------------------| -| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) | -| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| CHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) | -| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) | -| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| BINARY VARYING | [String](../../sql-reference/data-types/string.md) | diff --git a/docs/en/sql-reference/data-types/nullable.md b/docs/en/sql-reference/data-types/nullable.md index 5504765e4a0..abcb87a0c1b 100644 --- a/docs/en/sql-reference/data-types/nullable.md +++ b/docs/en/sql-reference/data-types/nullable.md @@ -1,7 +1,7 @@ --- slug: /en/sql-reference/data-types/nullable -sidebar_position: 55 -sidebar_label: Nullable +sidebar_position: 44 +sidebar_label: Nullable(T) --- # Nullable(T) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 517a28576f0..39f8409c1e1 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -1,5 +1,7 @@ --- slug: /en/sql-reference/data-types/simpleaggregatefunction +sidebar_position: 48 +sidebar_label: SimpleAggregateFunction --- # SimpleAggregateFunction diff --git a/docs/en/sql-reference/data-types/string.md b/docs/en/sql-reference/data-types/string.md index f891a9303e5..8a4f346fdfc 100644 --- a/docs/en/sql-reference/data-types/string.md +++ b/docs/en/sql-reference/data-types/string.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/string -sidebar_position: 44 +sidebar_position: 8 sidebar_label: String --- @@ -13,7 +13,7 @@ When creating tables, numeric parameters for string fields can be set (e.g. `VAR Aliases: -- `String` — `LONGTEXT`, `MEDIUMTEXT`, `TINYTEXT`, `TEXT`, `LONGBLOB`, `MEDIUMBLOB`, `TINYBLOB`, `BLOB`, `VARCHAR`, `CHAR`. +- `String` — `LONGTEXT`, `MEDIUMTEXT`, `TINYTEXT`, `TEXT`, `LONGBLOB`, `MEDIUMBLOB`, `TINYBLOB`, `BLOB`, `VARCHAR`, `CHAR`, `CHAR LARGE OBJECT`, `CHAR VARYING`, `CHARACTER LARGE OBJECT`, `CHARACTER VARYING`, `NCHAR LARGE OBJECT`, `NCHAR VARYING`, `NATIONAL CHARACTER LARGE OBJECT`, `NATIONAL CHARACTER VARYING`, `NATIONAL CHAR VARYING`, `NATIONAL CHARACTER`, `NATIONAL CHAR`, `BINARY LARGE OBJECT`, `BINARY VARYING`, ## Encodings diff --git a/docs/en/sql-reference/data-types/tuple.md b/docs/en/sql-reference/data-types/tuple.md index 8f87eeca075..0525a3b0476 100644 --- a/docs/en/sql-reference/data-types/tuple.md +++ b/docs/en/sql-reference/data-types/tuple.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/data-types/tuple -sidebar_position: 54 +sidebar_position: 34 sidebar_label: Tuple(T1, T2, ...) --- -# Tuple(T1, T2, …) +# Tuple(T1, T2, ...) A tuple of elements, each having an individual [type](../../sql-reference/data-types/index.md#data_types). Tuple must contain at least one element. diff --git a/docs/en/sql-reference/data-types/uuid.md b/docs/en/sql-reference/data-types/uuid.md index 40f756b9588..75e163f5063 100644 --- a/docs/en/sql-reference/data-types/uuid.md +++ b/docs/en/sql-reference/data-types/uuid.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/data-types/uuid -sidebar_position: 46 +sidebar_position: 24 sidebar_label: UUID --- diff --git a/docs/en/sql-reference/data-types/variant.md b/docs/en/sql-reference/data-types/variant.md index 7d10d4b0e97..1a9f1dde8d3 100644 --- a/docs/en/sql-reference/data-types/variant.md +++ b/docs/en/sql-reference/data-types/variant.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/data-types/variant -sidebar_position: 55 -sidebar_label: Variant +sidebar_position: 40 +sidebar_label: Variant(T1, T2, ...) --- -# Variant(T1, T2, T3, ...) +# Variant(T1, T2, ...) This type represents a union of other data types. Type `Variant(T1, T2, ..., TN)` means that each row of this type has a value of either type `T1` or `T2` or ... or `TN` or none of them (`NULL` value). diff --git a/docs/ru/sql-reference/data-types/multiword-types.md b/docs/ru/sql-reference/data-types/multiword-types.md deleted file mode 100644 index cca2d71e480..00000000000 --- a/docs/ru/sql-reference/data-types/multiword-types.md +++ /dev/null @@ -1,28 +0,0 @@ ---- -slug: /ru/sql-reference/data-types/multiword-types -sidebar_position: 61 -sidebar_label: Составные типы ---- - -# Составные типы {#multiword-types} - -При создании таблиц вы можете использовать типы данных с названием, состоящим из нескольких слов. Такие названия поддерживаются для лучшей совместимости с SQL. - -## Поддержка составных типов {#multiword-types-support} - -| Составные типы | Обычные типы | -|-------------------------------------|-----------------------------------------------------------| -| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) | -| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| CHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) | -| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) | -| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) | -| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) | -| BINARY VARYING | [String](../../sql-reference/data-types/string.md) | diff --git a/docs/zh/sql-reference/data-types/multiword-types.mdx b/docs/zh/sql-reference/data-types/multiword-types.mdx deleted file mode 100644 index 85431d47efd..00000000000 --- a/docs/zh/sql-reference/data-types/multiword-types.mdx +++ /dev/null @@ -1,10 +0,0 @@ ---- -slug: /zh/sql-reference/data-types/multiword-types -sidebar_position: 61 -sidebar_label: Multiword Type Names -title: "Multiword Types" ---- - -import Content from '@site/docs/en/sql-reference/data-types/multiword-types.md'; - - From cf8a83bb36a5b4072b100301abf267d74d8a4874 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 2 Apr 2024 21:05:37 +0000 Subject: [PATCH 068/243] Bump From 0ad3ed5ae3deb4e5077a8d18cf9fb16be1495d01 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 2 Apr 2024 18:38:58 -0300 Subject: [PATCH 069/243] Add support for S3 access through aws private link interface --- src/IO/S3/URI.cpp | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 027cb624ed5..b007d9268b7 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -33,12 +33,18 @@ namespace S3 URI::URI(const std::string & uri_) { /// Case when bucket name represented in domain name of S3 URL. - /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) + /// E.g. (https://bucket-name.s3.region.amazonaws.com/key) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); + /// Case when AWS Private Link Interface is being used + /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com) + /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html +// static const RE2 aws_private_link_style_pattern("bucket\\.vpce\\-([a-z0-9\\-.:]+)\\.vpce.amazonaws.com/([^/]*)/(.*)"); + static const RE2 aws_private_link_style_pattern("bucket\\.vpce\\-([a-z0-9\\-.:]+)\\.vpce.amazonaws.com"); + /// Case when bucket name and key represented in path of S3 URL. - /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) + /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#path-style-access static const RE2 path_style_pattern("^/([^/]*)/(.*)"); @@ -103,7 +109,19 @@ URI::URI(const std::string & uri_) String name; String endpoint_authority_from_uri; - if (re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) + if (re2::RE2::FullMatch(uri.getAuthority(), aws_private_link_style_pattern)) + { + if (!re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", + quoteString("ada")); + } + is_virtual_hosted_style = true; + endpoint = uri.getScheme() + "://" + uri.getAuthority(); + validateBucket(bucket, uri); + } + else if (re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) { is_virtual_hosted_style = true; endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri; From c1fd9262a7126fa9a8cbd3e18942eb499ce93965 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 2 Apr 2024 18:42:57 -0300 Subject: [PATCH 070/243] change exception placeholder --- src/IO/S3/URI.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index b007d9268b7..69b539cde8b 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -38,9 +38,8 @@ URI::URI(const std::string & uri_) static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3express[\-a-z0-9]+|s3|cos|obs|oss|eos)([.\-][a-z0-9\-.:]+))"); /// Case when AWS Private Link Interface is being used - /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com) + /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html -// static const RE2 aws_private_link_style_pattern("bucket\\.vpce\\-([a-z0-9\\-.:]+)\\.vpce.amazonaws.com/([^/]*)/(.*)"); static const RE2 aws_private_link_style_pattern("bucket\\.vpce\\-([a-z0-9\\-.:]+)\\.vpce.amazonaws.com"); /// Case when bucket name and key represented in path of S3 URL. @@ -113,10 +112,10 @@ URI::URI(const std::string & uri_) { if (!re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", - quoteString("ada")); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not parse bucket and key from uri {}", uri.toString()); } + + // Default to virtual hosted style is_virtual_hosted_style = true; endpoint = uri.getScheme() + "://" + uri.getAuthority(); validateBucket(bucket, uri); From c7a28b137ad9ca75c44bb531fc79ba034e3e311d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Apr 2024 07:24:20 +0000 Subject: [PATCH 071/243] Update version_date.tsv and changelogs after v24.3.2.23-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.3.2.23-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 33 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.3.2.23-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 3daa62cb212..346868e19c4 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.3.1.2672" +ARG VERSION="24.3.2.23" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index ace01ae9a9f..36f09c092f8 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.3.1.2672" +ARG VERSION="24.3.2.23" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index e92823b686a..531a50efe96 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.3.1.2672" +ARG VERSION="24.3.2.23" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v24.3.2.23-lts.md b/docs/changelogs/v24.3.2.23-lts.md new file mode 100644 index 00000000000..4d59a1cedf6 --- /dev/null +++ b/docs/changelogs/v24.3.2.23-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.2.23-lts (8b7d910960c) FIXME as compared to v24.3.1.2672-lts (2c5c589a882) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix logical error in group_by_use_nulls + grouping set + analyzer + materialize/constant [#61567](https://github.com/ClickHouse/ClickHouse/pull/61567) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix external table cannot parse data type Bool [#62115](https://github.com/ClickHouse/ClickHouse/pull/62115) ([Duc Canh Le](https://github.com/canhld94)). +* Revert "Merge pull request [#61564](https://github.com/ClickHouse/ClickHouse/issues/61564) from liuneng1994/optimize_in_single_value" [#62135](https://github.com/ClickHouse/ClickHouse/pull/62135) ([Raúl Marín](https://github.com/Algunenano)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#62030](https://github.com/ClickHouse/ClickHouse/issues/62030):. [#61869](https://github.com/ClickHouse/ClickHouse/pull/61869) ([Nikita Fomichev](https://github.com/fm4v)). +* Backported in [#62057](https://github.com/ClickHouse/ClickHouse/issues/62057): ... [#62044](https://github.com/ClickHouse/ClickHouse/pull/62044) ([Max K.](https://github.com/maxknv)). +* Backported in [#62204](https://github.com/ClickHouse/ClickHouse/issues/62204):. [#62190](https://github.com/ClickHouse/ClickHouse/pull/62190) ([Konstantin Bogdanov](https://github.com/thevar1able)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix some crashes with analyzer and group_by_use_nulls. [#61933](https://github.com/ClickHouse/ClickHouse/pull/61933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix scalars create as select [#61998](https://github.com/ClickHouse/ClickHouse/pull/61998) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Ignore IfChainToMultiIfPass if returned type changed. [#62059](https://github.com/ClickHouse/ClickHouse/pull/62059) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix type for ConvertInToEqualPass [#62066](https://github.com/ClickHouse/ClickHouse/pull/62066) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Revert output Pretty in tty [#62090](https://github.com/ClickHouse/ClickHouse/pull/62090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index ca1a23a99db..060a0107c1e 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.3.2.23-lts 2024-04-03 v24.3.1.2672-lts 2024-03-27 v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 From 8f40db2fb2c520a8907914f8f5799026c43ed3f2 Mon Sep 17 00:00:00 2001 From: Mark Needham Date: Wed, 3 Apr 2024 08:26:52 +0100 Subject: [PATCH 072/243]
missing closing / --- docs/en/sql-reference/window-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 19821781d0e..32ebc6d028f 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -24,7 +24,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | | `rank()`, `dense_rank()`, `row_number()` | ✅ | | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | -| ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | +| ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions From d7827eaf57974caf9229d0bc3dcb80470a3e3fd9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 09:48:36 +0200 Subject: [PATCH 073/243] Close: https://github.com/ClickHouse/ClickHouse/issues/23104 --- .../03053_analyzer_join_alias.reference | 0 .../0_stateless/03053_analyzer_join_alias.sql | 43 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/03053_analyzer_join_alias.reference create mode 100644 tests/queries/0_stateless/03053_analyzer_join_alias.sql diff --git a/tests/queries/0_stateless/03053_analyzer_join_alias.reference b/tests/queries/0_stateless/03053_analyzer_join_alias.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03053_analyzer_join_alias.sql b/tests/queries/0_stateless/03053_analyzer_join_alias.sql new file mode 100644 index 00000000000..7e11cc7c810 --- /dev/null +++ b/tests/queries/0_stateless/03053_analyzer_join_alias.sql @@ -0,0 +1,43 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/23104 +DROP DATABASE IF EXISTS test_03053; +CREATE DATABASE test_03053; + +CREATE TABLE test_03053.base +( +`id` UInt64, +`id2` UInt64, +`d` UInt64, +`value` UInt64 +) +ENGINE=MergeTree() +PARTITION BY d +ORDER BY (id,id2,d); + +CREATE TABLE test_03053.derived1 +( + `id1` UInt64, + `d1` UInt64, + `value1` UInt64 +) +ENGINE = MergeTree() +PARTITION BY d1 +ORDER BY (id1, d1); + +CREATE TABLE test_03053.derived2 +( + `id2` UInt64, + `d2` UInt64, + `value2` UInt64 +) +ENGINE = MergeTree() +PARTITION BY d2 +ORDER BY (id2, d2); + +SELECT + base.id AS `base.id`, + derived2.id2 AS `derived2.id2`, + derived2.value2 AS `derived2.value2`, + derived1.value1 AS `derived1.value1` +FROM test_03053.base AS base +LEFT JOIN test_03053.derived2 AS derived2 ON base.id2 = derived2.id2 +LEFT JOIN test_03053.derived1 AS derived1 ON base.id = derived1.id1; From d78ab39b4bccdd18120408ab0e050515c5fbe465 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 09:52:38 +0200 Subject: [PATCH 074/243] Close: https://github.com/ClickHouse/ClickHouse/issues/21584 --- .../0_stateless/03054_analyzer_join_alias.reference | 1 + .../0_stateless/03054_analyzer_join_alias.sql | 12 ++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03054_analyzer_join_alias.reference create mode 100644 tests/queries/0_stateless/03054_analyzer_join_alias.sql diff --git a/tests/queries/0_stateless/03054_analyzer_join_alias.reference b/tests/queries/0_stateless/03054_analyzer_join_alias.reference new file mode 100644 index 00000000000..f599e28b8ab --- /dev/null +++ b/tests/queries/0_stateless/03054_analyzer_join_alias.reference @@ -0,0 +1 @@ +10 diff --git a/tests/queries/0_stateless/03054_analyzer_join_alias.sql b/tests/queries/0_stateless/03054_analyzer_join_alias.sql new file mode 100644 index 00000000000..0bf93258aa6 --- /dev/null +++ b/tests/queries/0_stateless/03054_analyzer_join_alias.sql @@ -0,0 +1,12 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/21584 +SELECT count() +FROM +( + SELECT number AS key_1 + FROM numbers(15) +) AS x +ALL INNER JOIN +( + SELECT number AS key_1 + FROM numbers(10) +) AS z ON key_1 = z.key_1; From 6805103ce3545d3e3d94d18a2ae26d5976b2c9c6 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 09:58:11 +0200 Subject: [PATCH 075/243] Close: https://github.com/ClickHouse/ClickHouse/issues/23344 --- .../03055_analyzer_subquery_group_array.reference | 1 + .../0_stateless/03055_analyzer_subquery_group_array.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03055_analyzer_subquery_group_array.reference create mode 100644 tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql diff --git a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.reference b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.reference new file mode 100644 index 00000000000..d05b1f927f4 --- /dev/null +++ b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.reference @@ -0,0 +1 @@ +0 0 diff --git a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql new file mode 100644 index 00000000000..071d8f8e1c8 --- /dev/null +++ b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/23344 +SELECT logTrace(repeat('Hello', 100)), ignore(*) +FROM ( + SELECT ignore((SELECT groupArrayState(([number], [number])) FROM numbers(19000))) +) From 809461ccdae2896e7b91dba36667645864a9aaae Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 10:03:47 +0200 Subject: [PATCH 076/243] Close: https://github.com/ClickHouse/ClickHouse/issues/22627 --- ...3056_analyzer_double_subquery_alias.reference | 1 + .../03056_analyzer_double_subquery_alias.sql | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03056_analyzer_double_subquery_alias.reference create mode 100644 tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql diff --git a/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.reference b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.reference new file mode 100644 index 00000000000..72749c905a3 --- /dev/null +++ b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.reference @@ -0,0 +1 @@ +1 1 1 diff --git a/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql new file mode 100644 index 00000000000..b86ae97c8bf --- /dev/null +++ b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql @@ -0,0 +1,16 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/22627 +WITH + x AS + ( + SELECT 1 AS a + ), + xx AS + ( + SELECT * + FROM x + , x AS x1 + , x AS x2 + ) +SELECT * +FROM xx +WHERE a = 1; From a401d328fd35a65973306571e67539fcd8ec4909 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 10:06:11 +0200 Subject: [PATCH 077/243] Close: https://github.com/ClickHouse/ClickHouse/issues/10276 --- .../03057_analyzer_subquery_alias_join.reference | 1 + .../03057_analyzer_subquery_alias_join.sql | 12 ++++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03057_analyzer_subquery_alias_join.reference create mode 100644 tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql diff --git a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.reference b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.reference new file mode 100644 index 00000000000..3e43e6addc0 --- /dev/null +++ b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.reference @@ -0,0 +1 @@ +1000 100000 diff --git a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql new file mode 100644 index 00000000000..13852471dca --- /dev/null +++ b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql @@ -0,0 +1,12 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/10276 +SELECT + sum(x.n) as n, + sum(z.n) as n2 +FROM +( + SELECT 1000 AS n,1 as id +) AS x +join (select 10000 as n,1 as id) as y +on x.id = y.id +left join (select 100000 as n,1 as id) as z +on x.id = z.id; From 651d382695b9399d9f74e6369772975c3b26746a Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 3 Apr 2024 11:17:57 +0200 Subject: [PATCH 078/243] Minor changes to CramersV functions --- .../aggregate-functions/reference/cramersv.md | 11 ++++++++--- .../reference/cramersvbiascorrected.md | 10 ++++------ 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md index e9e2c367610..77b6043dc13 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md @@ -7,21 +7,26 @@ sidebar_position: 351 [Cramer's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramer's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. +:::note +For a bias corrected version of Cramer's V see: [cramersVBiasCorrected](./cramersvbiascorrected.md) +::: + **Syntax** ``` sql cramersV(column1, column2) ``` -**Arguments** +**Parameters** -- `column1` and `column2` are the columns to be compared +- `column1`: first column to be compared. +- `column2`: second column to be compared. **Returned value** - a value between 0 (corresponding to no association between the columns' values) to 1 (complete association). -**Return type** is always [Float64](../../../sql-reference/data-types/float.md). +Type: always [Float64](../../../sql-reference/data-types/float.md). **Example** diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index f5ad3a8a937..53ec5c0a985 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -5,26 +5,24 @@ sidebar_position: 352 # cramersVBiasCorrected - Cramer's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramer's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). - - **Syntax** ``` sql cramersVBiasCorrected(column1, column2) ``` -**Arguments** +**Parameters** -- `column1` and `column2` are the columns to be compared +- `column1`: first column to be compared. +- `column2`: second column to be compared. **Returned value** - a value between 0 (corresponding to no association between the columns' values) to 1 (complete association). -**Return type** is always [Float64](../../../sql-reference/data-types/float.md). +Type: always [Float64](../../../sql-reference/data-types/float.md). **Example** From 3e2616d921ec5c128138b57a2f20b7e07bedbf7b Mon Sep 17 00:00:00 2001 From: Blargian Date: Wed, 3 Apr 2024 11:23:42 +0200 Subject: [PATCH 079/243] Add 'Query:' before query for consistency --- docs/en/sql-reference/aggregate-functions/reference/cramersv.md | 2 ++ .../aggregate-functions/reference/cramersvbiascorrected.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md index 77b6043dc13..2424ff95237 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md @@ -32,6 +32,8 @@ Type: always [Float64](../../../sql-reference/data-types/float.md). The following two columns being compared below have no association with each other, so the result of `cramersV` is 0: +Query: + ``` sql SELECT cramersV(a, b) diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index 53ec5c0a985..939c04e3fdc 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -28,6 +28,8 @@ Type: always [Float64](../../../sql-reference/data-types/float.md). The following two columns being compared below have a small association with each other. Notice the result of `cramersVBiasCorrected` is smaller than the result of `cramersV`: +Query: + ``` sql SELECT cramersV(a, b), From 2f45d98c970740c2263812fe3044616787f49d96 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Apr 2024 10:03:04 +0000 Subject: [PATCH 080/243] Docs: Improve wording of DROP TABLE docs --- docs/en/sql-reference/statements/drop.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index 159ab09ab94..833ff756449 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -20,11 +20,10 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC] ## DROP TABLE -Deletes the table. -In case when `IF EMPTY` clause is specified server will check if table is empty only on replica that received initial query. +Deletes one or more tables. :::tip -Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) +To undo the deletion of a table, please see see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) ::: Syntax: @@ -33,7 +32,9 @@ Syntax: DROP [TEMPORARY] TABLE [IF EXISTS] [IF EMPTY] [db1.]name_1[, [db2.]name_2, ...] [ON CLUSTER cluster] [SYNC] ``` -Note that deleting multiple tables at the same time is a non-atomic deletion. If a table fails to be deleted, subsequent tables will not be deleted. +Limitations: +- If the clause `IF EMPTY` is specified, the server checks the emptiness of the table only on the replica which received the query. +- Deleting multiple tables at once is not an atomic operation, i.e. if the deletion of a table fails, subsequent tables will not be deleted. ## DROP DICTIONARY From 9d9ee6ce06447bf35d2591976b43bd1e6beeed68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Apr 2024 12:16:50 +0200 Subject: [PATCH 081/243] Remove assertion --- src/Interpreters/Cache/FileSegment.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f56f5d3f66c..ac42c4d75bb 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -845,7 +845,6 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock & lock) chassert(downloaded_size == range().size()); chassert(downloaded_size > 0); chassert(std::filesystem::file_size(getPath()) > 0); - chassert(queue_iterator); check_iterator(queue_iterator); } else From e19c635edd3b511adf1decffaa70829e77f59b6a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 10:12:55 +0200 Subject: [PATCH 082/243] Close: https://github.com/ClickHouse/ClickHouse/issues/4567 --- ...03058_analyzer_ambiguous_columns.reference | 0 .../03058_analyzer_ambiguous_columns.sql | 25 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03058_analyzer_ambiguous_columns.reference create mode 100644 tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql diff --git a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.reference b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql new file mode 100644 index 00000000000..47df6e76a38 --- /dev/null +++ b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql @@ -0,0 +1,25 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/4567 +DROP TABLE IF EXISTS fact; +DROP TABLE IF EXISTS animals; +DROP TABLE IF EXISTS colors; + +create table fact(id Int64, animal_key Int64, color_key Int64) Engine = MergeTree order by tuple(); +insert into fact values (1,1,1),(2,2,2); + +create table animals(animal_key UInt64, animal_name String) Engine = MergeTree order by tuple(); +insert into animals values (0, 'unknown'); + +create table colors(color_key UInt64, color_name String) Engine = MergeTree order by tuple(); +insert into colors values (0, 'unknown'); + + +select id, animal_name, a.animal_key, color_name, color_key +from fact a + left join (select toInt64(animal_key) animal_key, animal_name from animals) b on (a.animal_key = b.animal_key) + left join (select toInt64(color_key) color_key, color_name from colors) c on (a.color_key = c.color_key); -- { serverError AMBIGUOUS_IDENTIFIER } + +select id, animal_name, animal_key, color_name, color_key +from fact a + left join (select toInt64(animal_key) animal_key, animal_name from animals) b on (a.animal_key = b.animal_key) + left join (select toInt64(color_key) color_key, color_name from colors) c on (a.color_key = c.color_key); -- { serverError AMBIGUOUS_IDENTIFIER } + From 5e30984d8d7780182dda35c827b4c4626c06210d Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 10:19:14 +0200 Subject: [PATCH 083/243] Close: https://github.com/ClickHouse/ClickHouse/issues/17710 --- .../03059_analyzer_join_engine_missing_column.reference | 0 .../03059_analyzer_join_engine_missing_column.sql | 8 ++++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.reference create mode 100644 tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql diff --git a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.reference b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql new file mode 100644 index 00000000000..76150335f7e --- /dev/null +++ b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/17710 +CREATE TABLE id_val(id UInt32, val UInt32) ENGINE = Memory; +CREATE TABLE id_val_join0(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 0; +CREATE TABLE id_val_join1(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1; + +SELECT * FROM id_val ANY LEFT JOIN id_val_join0 USING (id) SETTINGS join_use_nulls = 0; + +SELECT * FROM id_val ANY LEFT JOIN id_val_join1 USING (id) SETTINGS join_use_nulls = 1; From 197e6008ea54d158b73cbc55c657f48801f08b48 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 10:24:58 +0200 Subject: [PATCH 084/243] Close: https://github.com/ClickHouse/ClickHouse/issues/11068 --- .../03060_analyzer_regular_view_alias.reference | 0 .../03060_analyzer_regular_view_alias.sql | 15 +++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03060_analyzer_regular_view_alias.reference create mode 100644 tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql diff --git a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.reference b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql new file mode 100644 index 00000000000..ba0257d7b3b --- /dev/null +++ b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql @@ -0,0 +1,15 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/11068 +create table vt(datetime_value DateTime, value Float64) Engine=Memory; + +create view computed_datum_hours as +SELECT + toStartOfHour(b.datetime_value) AS datetime_desc, + sum(b.value) AS value +FROM vt AS b +GROUP BY toStartOfHour(b.datetime_value); + +SELECT + toStartOfHour(b.datetime_value) AS datetime_desc, + sum(b.value) AS value +FROM vt AS b +GROUP BY toStartOfHour(b.datetime_value); From bd90cd532fdc1dff06db3a8f10dfd25c76eb4234 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:10:37 +0200 Subject: [PATCH 085/243] Close: https://github.com/ClickHouse/ClickHouse/issues/24395 --- ...03061_analyzer_alias_as_right_key_in_join.reference | 10 ++++++++++ .../03061_analyzer_alias_as_right_key_in_join.sql | 7 +++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.reference create mode 100644 tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql diff --git a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.reference b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.reference new file mode 100644 index 00000000000..af98bcd6397 --- /dev/null +++ b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.reference @@ -0,0 +1,10 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 diff --git a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql new file mode 100644 index 00000000000..e223909a5a8 --- /dev/null +++ b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql @@ -0,0 +1,7 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/24395 +CREATE TABLE xxxx_yyy (key UInt32, key_b ALIAS key) ENGINE=MergeTree() ORDER BY key; +INSERT INTO xxxx_yyy SELECT number FROM numbers(10); + +SELECT * +FROM xxxx_yyy AS a +INNER JOIN xxxx_yyy AS b ON a.key = b.key_b; From b66cfb4c823eae628ed8e5639d494f71768c93ea Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:14:18 +0200 Subject: [PATCH 086/243] Close: https://github.com/ClickHouse/ClickHouse/issues/23416 --- ...062_analyzer_join_engine_missing_column.reference | 2 ++ .../03062_analyzer_join_engine_missing_column.sql | 12 ++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.reference create mode 100644 tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql diff --git a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.reference b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.reference new file mode 100644 index 00000000000..d496ccad6b6 --- /dev/null +++ b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.reference @@ -0,0 +1,2 @@ +abc 0 0 0 1 +abc 0 0 0 1 diff --git a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql new file mode 100644 index 00000000000..6c24ef6f66d --- /dev/null +++ b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql @@ -0,0 +1,12 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/23416 +create table test (TOPIC String, PARTITION UInt64, OFFSET UInt64, ID UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03062', 'r2') ORDER BY (TOPIC, PARTITION, OFFSET); + +create table test_join (TOPIC String, PARTITION UInt64, OFFSET UInt64) ENGINE = Join(ANY, LEFT, `TOPIC`, `PARTITION`) SETTINGS join_any_take_last_row = 1; + +insert into test values('abc',0,0,0); + +insert into test_join values('abc',0,1); + +select *, joinGet('test_join', 'OFFSET', TOPIC, PARTITION) from test; + +select * from test any left join test_join using (TOPIC, PARTITION); From cc6bcb606eac18334823354500a79d2203be8c44 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:22:32 +0200 Subject: [PATCH 087/243] Close: https://github.com/ClickHouse/ClickHouse/issues/23162 --- ...er_multi_join_wrong_table_specifier.reference | 0 ...analyzer_multi_join_wrong_table_specifier.sql | 16 ++++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.reference create mode 100644 tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql diff --git a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.reference b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql new file mode 100644 index 00000000000..c2c29b688cd --- /dev/null +++ b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql @@ -0,0 +1,16 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/23162 +CREATE TABLE t1 ( k Int64, x Int64) ENGINE = Memory; + +CREATE TABLE t2( x Int64 ) ENGINE = Memory; + +create table s (k Int64, d DateTime) Engine=Memory; + +SELECT * FROM t1 +INNER JOIN s ON t1.k = s.k +INNER JOIN t2 ON t2.x = t1.x +WHERE (t1.d >= now()); -- { serverError UNKNOWN_IDENTIFIER } + +SELECT * FROM t1 +INNER JOIN s ON t1.k = s.k +WHERE (t1.d >= now()); -- { serverError UNKNOWN_IDENTIFIER } + From e09187ea2e9473cd5bad59fd5eaf61a4ddc91e30 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:27:43 +0200 Subject: [PATCH 088/243] Close: https://github.com/ClickHouse/ClickHouse/issues/25655 --- .../0_stateless/03064_analyzer_named_subqueries.reference | 1 + .../queries/0_stateless/03064_analyzer_named_subqueries.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03064_analyzer_named_subqueries.reference create mode 100644 tests/queries/0_stateless/03064_analyzer_named_subqueries.sql diff --git a/tests/queries/0_stateless/03064_analyzer_named_subqueries.reference b/tests/queries/0_stateless/03064_analyzer_named_subqueries.reference new file mode 100644 index 00000000000..556d825db42 --- /dev/null +++ b/tests/queries/0_stateless/03064_analyzer_named_subqueries.reference @@ -0,0 +1 @@ +2 1 diff --git a/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql new file mode 100644 index 00000000000..ef8aca2fefa --- /dev/null +++ b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/25655 +SELECT + sum(t.b) / 1 a, + sum(t.a) +FROM ( SELECT 1 a, 2 b ) t; From 71d85653e9a44c340520a63374af172610a989db Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:29:24 +0200 Subject: [PATCH 089/243] Close: https://github.com/ClickHouse/ClickHouse/issues/11757 --- .../03065_analyzer_cross_join_and_array_join.reference | 2 ++ .../0_stateless/03065_analyzer_cross_join_and_array_join.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.reference create mode 100644 tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql diff --git a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.reference b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.reference new file mode 100644 index 00000000000..594a6a2deeb --- /dev/null +++ b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.reference @@ -0,0 +1,2 @@ +1 3 +2 4 diff --git a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql new file mode 100644 index 00000000000..c270a0f4504 --- /dev/null +++ b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql @@ -0,0 +1,2 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/11757 +select * from (select [1, 2] a) aa cross join (select [3, 4] b) bb array join aa.a, bb.b; From 0758faa039edf0a1b61d05c17b2568e0a53693ea Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:32:26 +0200 Subject: [PATCH 090/243] Add test for analyzer and enable_global_with_statement=1 --- .../03066_analyzer_global_with_statement.reference | 1 + .../0_stateless/03066_analyzer_global_with_statement.sql | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03066_analyzer_global_with_statement.reference create mode 100644 tests/queries/0_stateless/03066_analyzer_global_with_statement.sql diff --git a/tests/queries/0_stateless/03066_analyzer_global_with_statement.reference b/tests/queries/0_stateless/03066_analyzer_global_with_statement.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03066_analyzer_global_with_statement.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql new file mode 100644 index 00000000000..338eb30e6ff --- /dev/null +++ b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql @@ -0,0 +1,7 @@ +WITH 0 AS test +SELECT * +FROM +( + SELECT 1 AS test +) +SETTINGS enable_global_with_statement = 1 From 7872d920586e7843a8dd1db027ecf7c9fa75b07c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 11:35:08 +0200 Subject: [PATCH 091/243] Add test for analyzer and complex alias join and with --- .../03067_analyzer_complex_alias_join.reference | 1 + .../0_stateless/03067_analyzer_complex_alias_join.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03067_analyzer_complex_alias_join.reference create mode 100644 tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql diff --git a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.reference b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.reference new file mode 100644 index 00000000000..6192a595f1e --- /dev/null +++ b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.reference @@ -0,0 +1 @@ +key \N diff --git a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql new file mode 100644 index 00000000000..7d1264a6116 --- /dev/null +++ b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql @@ -0,0 +1,9 @@ +with d as (select 'key'::Varchar(255) c, 'x'::Varchar(255) s) +SELECT r1, c as r2 +FROM ( + SELECT t as s, c as r1 + FROM ( SELECT 'y'::Varchar(255) as t, 'x'::Varchar(255) as s) t1 + LEFT JOIN d USING (s) + ) t2 +LEFT JOIN d using (s) +SETTINGS join_use_nulls=1; From ece7099017285db81637c5f65fad9678301a22f9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 12:04:36 +0200 Subject: [PATCH 092/243] Close: https://github.com/ClickHouse/ClickHouse/issues/6571 --- .../03068_analyzer_distributed_join.reference | 2 + .../03068_analyzer_distributed_join.sql | 55 +++++++++++++++++++ 2 files changed, 57 insertions(+) create mode 100644 tests/queries/0_stateless/03068_analyzer_distributed_join.reference create mode 100644 tests/queries/0_stateless/03068_analyzer_distributed_join.sql diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.reference b/tests/queries/0_stateless/03068_analyzer_distributed_join.reference new file mode 100644 index 00000000000..1444d39d957 --- /dev/null +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.reference @@ -0,0 +1,2 @@ +localhost 9000 0 0 0 +localhost 9000 0 0 0 diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql new file mode 100644 index 00000000000..714a64a01b6 --- /dev/null +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -0,0 +1,55 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/6571 +CREATE TABLE LINEITEM_shard ON CLUSTER test_shard_localhost +( + L_ORDERKEY UInt64, + L_COMMITDATE UInt32, + L_RECEIPTDATE UInt32 +) +ENGINE = MergeTree() +ORDER BY L_ORDERKEY; + +CREATE TABLE LINEITEM AS LINEITEM_shard +ENGINE = Distributed('test_shard_localhost', currentDatabase(), LINEITEM_shard, rand()); + +CREATE TABLE ORDERS_shard ON CLUSTER test_shard_localhost +( + O_ORDERKEY UInt64, + O_ORDERPRIORITY UInt32 +) +ENGINE = MergeTree() +ORDER BY O_ORDERKEY; + +CREATE TABLE ORDERS AS ORDERS_shard +ENGINE = Distributed('test_shard_localhost', currentDatabase(), ORDERS_shard, rand()); + +SET joined_subquery_requires_alias=0; + +select + O_ORDERPRIORITY, + count(*) as order_count +from ORDERS JOIN ( + select L_ORDERKEY + from + LINEITEM_shard + group by L_ORDERKEY + having any(L_COMMITDATE < L_RECEIPTDATE) +) on O_ORDERKEY=L_ORDERKEY +group by O_ORDERPRIORITY +order by O_ORDERPRIORITY +limit 1; + +SET joined_subquery_requires_alias=1; + +select + O_ORDERPRIORITY, + count(*) as order_count +from ORDERS JOIN ( + select L_ORDERKEY + from + LINEITEM_shard + group by L_ORDERKEY + having any(L_COMMITDATE < L_RECEIPTDATE) +) AS x on O_ORDERKEY=L_ORDERKEY +group by O_ORDERPRIORITY +order by O_ORDERPRIORITY +limit 1; From fdb752de6f56a4cfdc2b48b342c873b861d2ddb9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 12:52:28 +0200 Subject: [PATCH 093/243] Close: https://github.com/ClickHouse/ClickHouse/issues/4432 --- .../03069_analyzer_with_alias_in_array_join.reference | 2 ++ .../0_stateless/03069_analyzer_with_alias_in_array_join.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.reference create mode 100644 tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql diff --git a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.reference b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.reference new file mode 100644 index 00000000000..1191247b6d9 --- /dev/null +++ b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.reference @@ -0,0 +1,2 @@ +1 +2 diff --git a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql new file mode 100644 index 00000000000..84ad0b4e199 --- /dev/null +++ b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/4432 +WITH [1, 2] AS zz +SELECT x +FROM system.one +ARRAY JOIN zz AS x From 0782ccaa91fa4a850cea00f52a660ee818e8e3c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 3 Apr 2024 12:53:09 +0200 Subject: [PATCH 094/243] Update docs/en/sql-reference/statements/drop.md Co-authored-by: Han Fei --- docs/en/sql-reference/statements/drop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index 833ff756449..98b849ecf3b 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -23,7 +23,7 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC] Deletes one or more tables. :::tip -To undo the deletion of a table, please see see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) +To undo the deletion of a table, please see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md) ::: Syntax: From 0b831ab87a1d6cc510042d021639548b2600bfdb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 13:09:43 +0200 Subject: [PATCH 095/243] Close: https://github.com/ClickHouse/ClickHouse/issues/8259 --- ...0_analyzer_CTE_scalar_as_numbers.reference | 25 +++++++++++++++++++ .../03070_analyzer_CTE_scalar_as_numbers.sql | 5 ++++ 2 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.reference create mode 100644 tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql diff --git a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.reference b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.reference new file mode 100644 index 00000000000..595b4d6b5b8 --- /dev/null +++ b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.reference @@ -0,0 +1,25 @@ +0 25 +1 25 +2 25 +3 25 +4 25 +5 25 +6 25 +7 25 +8 25 +9 25 +10 25 +11 25 +12 25 +13 25 +14 25 +15 25 +16 25 +17 25 +18 25 +19 25 +20 25 +21 25 +22 25 +23 25 +24 25 diff --git a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql new file mode 100644 index 00000000000..672c4f53e5f --- /dev/null +++ b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql @@ -0,0 +1,5 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/8259 +with + (select 25) as something +select *, something +from numbers(toUInt64(assumeNotNull(something))); From f822791ee397d6def5bd64adc47b077b484d0058 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 13:17:19 +0200 Subject: [PATCH 096/243] Close: https://github.com/ClickHouse/ClickHouse/issues/9233 --- ...r_array_join_forbid_non_existing_columns.reference | 0 ...nalyzer_array_join_forbid_non_existing_columns.sql | 11 +++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.reference create mode 100644 tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql diff --git a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.reference b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql new file mode 100644 index 00000000000..af81e3c2819 --- /dev/null +++ b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql @@ -0,0 +1,11 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/9233 +SELECT * +FROM +( + SELECT + [1, 2, 3] AS x, + [4, 5, 6] AS y +) +ARRAY JOIN + x, + Y; -- { serverError UNKNOWN_IDENTIFIER } From 15dd5ce5f665467d33bb02e7d7ba2decfade3e87 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Apr 2024 08:39:16 -0300 Subject: [PATCH 097/243] use raw literal string --- src/IO/S3/URI.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 69b539cde8b..7f628d27f34 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -40,7 +40,7 @@ URI::URI(const std::string & uri_) /// Case when AWS Private Link Interface is being used /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html - static const RE2 aws_private_link_style_pattern("bucket\\.vpce\\-([a-z0-9\\-.:]+)\\.vpce.amazonaws.com"); + static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.:]+)\.vpce.amazonaws.com)"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) From 40817de19a674763af7f7b83bd7712053f0ae18d Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 13:44:31 +0200 Subject: [PATCH 098/243] Close: https://github.com/ClickHouse/ClickHouse/issues/14699 --- .../03072_analyzer_missing_columns_from_subquery.reference | 1 + .../03072_analyzer_missing_columns_from_subquery.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.reference create mode 100644 tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql diff --git a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.reference b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql new file mode 100644 index 00000000000..68ff81413b7 --- /dev/null +++ b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql @@ -0,0 +1,2 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/14699 +select * from (select number from numbers(1)) where not ignore(*); From c576884a0b40d4870a7cbb08e62ebffec24ee328 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 13:51:06 +0200 Subject: [PATCH 099/243] Close: https://github.com/ClickHouse/ClickHouse/issues/27068 --- .../03073_analyzer_alias_as_column_name.reference | 2 ++ .../0_stateless/03073_analyzer_alias_as_column_name.sql | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03073_analyzer_alias_as_column_name.reference create mode 100644 tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql diff --git a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.reference b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.reference new file mode 100644 index 00000000000..ca6280d2dbf --- /dev/null +++ b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.reference @@ -0,0 +1,2 @@ +1 1997-02-01 +2 1997-02-01 diff --git a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql new file mode 100644 index 00000000000..0e7d2eb95c7 --- /dev/null +++ b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/27068 +CREATE TABLE test ( id String, create_time DateTime ) ENGINE = MergeTree ORDER BY id; + +insert into test values(1,'1970-02-01 00:00:00'); +insert into test values(2,'1970-02-01 00:00:00'); +insert into test values(3,'1970-03-01 00:00:00'); + +select id,'1997-02-01' as create_time from test where test.create_time='1970-02-01 00:00:00' ORDER BY id From 1c98c4752e916e077a1938c663324709fe5ec1c0 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 13:55:38 +0200 Subject: [PATCH 100/243] Close: https://github.com/ClickHouse/ClickHouse/issues/28687 --- .../03074_analyzer_alias_column_in_view.reference | 2 ++ .../0_stateless/03074_analyzer_alias_column_in_view.sql | 6 ++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03074_analyzer_alias_column_in_view.reference create mode 100644 tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql diff --git a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.reference b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql new file mode 100644 index 00000000000..6f970421788 --- /dev/null +++ b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/28687 +create view alias (dummy int, n alias dummy) as select * from system.one; + +select n from alias; + +select * from alias where n=0; From f2d78f650dbb0c2145f16cd0337835bfbdc879df Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 3 Apr 2024 14:32:19 +0200 Subject: [PATCH 101/243] Fix global trace collector --- src/Common/ThreadStatus.h | 4 ---- src/Coordination/Standalone/ThreadStatusExt.cpp | 4 ++++ 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 550cb76e736..02bf82e2da3 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -308,11 +308,7 @@ public: void flushUntrackedMemory(); -#ifdef CLICKHOUSE_KEEPER_STANDALONE_BUILD - void initGlobalProfiler(UInt64, UInt64) {} -#else void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); -#endif private: void applyGlobalSettings(); diff --git a/src/Coordination/Standalone/ThreadStatusExt.cpp b/src/Coordination/Standalone/ThreadStatusExt.cpp index 97f7287be8c..2b89e2f024d 100644 --- a/src/Coordination/Standalone/ThreadStatusExt.cpp +++ b/src/Coordination/Standalone/ThreadStatusExt.cpp @@ -11,4 +11,8 @@ void CurrentThread::attachToGroup(const ThreadGroupPtr &) { } +void ThreadStatus::initGlobalProfiler(UInt64, UInt64) +{ +} + } From ef0a9e889f2c9c3c9a1952e10b29fb1e94613ca3 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 14:35:34 +0200 Subject: [PATCH 102/243] Close: https://github.com/ClickHouse/ClickHouse/issues/28777 --- .../03075_analyzer_subquery_alias.reference | 1 + .../0_stateless/03075_analyzer_subquery_alias.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03075_analyzer_subquery_alias.reference create mode 100644 tests/queries/0_stateless/03075_analyzer_subquery_alias.sql diff --git a/tests/queries/0_stateless/03075_analyzer_subquery_alias.reference b/tests/queries/0_stateless/03075_analyzer_subquery_alias.reference new file mode 100644 index 00000000000..556d825db42 --- /dev/null +++ b/tests/queries/0_stateless/03075_analyzer_subquery_alias.reference @@ -0,0 +1 @@ +2 1 diff --git a/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql new file mode 100644 index 00000000000..897b189b133 --- /dev/null +++ b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql @@ -0,0 +1,10 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/28777 +SELECT + sum(q0.a2) AS a1, + sum(q0.a1) AS a9 +FROM +( + SELECT + 1 AS a1, + 2 AS a2 +) AS q0; From 8ae1f4c4b316e6f81e87ab70e7f1d11ad7e0e771 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 14:37:58 +0200 Subject: [PATCH 103/243] Close: https://github.com/ClickHouse/ClickHouse/issues/29734 --- ...76_analyzer_multiple_joins_alias.reference | 0 .../03076_analyzer_multiple_joins_alias.sql | 51 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.reference create mode 100644 tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql diff --git a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.reference b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql new file mode 100644 index 00000000000..8b8b76a5be1 --- /dev/null +++ b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql @@ -0,0 +1,51 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/29734 +SELECT * +FROM +( + SELECT 1 AS x +) AS a +INNER JOIN +( + SELECT + 1 AS x, + 2 AS y +) AS b ON (a.x = b.x) AND (a.y = b.y); -- { serverError UNKNOWN_IDENTIFIER } + + + +SELECT * +FROM +( + SELECT 1 AS x +) AS a +INNER JOIN +( + SELECT + 1 AS x, + 2 AS y +) AS b ON (a.x = b.x) AND (a.y = b.y) +INNER JOIN +( + SELECT 3 AS x +) AS c ON a.x = c.x; -- { serverError UNKNOWN_IDENTIFIER } + + +SELECT * +FROM +( + SELECT number AS x + FROM numbers(10) +) AS a +INNER JOIN +( + SELECT + number AS x, + number AS y + FROM numbers(10) +) AS b ON (a.x = b.x) AND (a.y = b.y) +INNER JOIN +( + SELECT number AS x + FROM numbers(10) +) AS c ON a.x = c.x; -- { serverError UNKNOWN_IDENTIFIER } + From c3f1130de86a94decff8521dbfb30c20c6a984e5 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 15:10:27 +0200 Subject: [PATCH 104/243] Close: https://github.com/ClickHouse/ClickHouse/issues/33825 --- ...er_multi_scalar_subquery_aliases.reference | 21 ++++++++++++++++++ ...analyzer_multi_scalar_subquery_aliases.sql | 22 +++++++++++++++++++ ...er_multi_scalar_subquery_aliases.reference | 21 ++++++++++++++++++ ...analyzer_multi_scalar_subquery_aliases.sql | 16 ++++++++++++++ 4 files changed, 80 insertions(+) create mode 100644 tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.reference create mode 100644 tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql create mode 100644 tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.reference create mode 100644 tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql diff --git a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.reference b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.reference new file mode 100644 index 00000000000..6b134f711d4 --- /dev/null +++ b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.reference @@ -0,0 +1,21 @@ +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +1 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 +9 9 9 9 diff --git a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql new file mode 100644 index 00000000000..3d558bdd602 --- /dev/null +++ b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql @@ -0,0 +1,22 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/33825 +CREATE TABLE t1 (i Int64, j Int64) ENGINE = Memory; +INSERT INTO t1 SELECT number, number FROM system.numbers LIMIT 10; +SELECT + (SELECT max(i) FROM t1) as i, + (SELECT max(i) FROM t1) as j, + (SELECT max(i) FROM t1) as k, + (SELECT max(i) FROM t1) as l +FROM t1; + +SELECT 1; + +WITH ( + SELECT max(i) + FROM t1 + ) AS value +SELECT + value AS i, + value AS j, + value AS k, + value AS l +FROM t1; diff --git a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.reference b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.reference new file mode 100644 index 00000000000..b2c49b655d4 --- /dev/null +++ b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.reference @@ -0,0 +1,21 @@ +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +111111111111 +1 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 +2222222222 diff --git a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql new file mode 100644 index 00000000000..ded6bfbe4e3 --- /dev/null +++ b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql @@ -0,0 +1,16 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/33825 +CREATE TABLE t2 (first_column Int64, second_column Int64) ENGINE = Memory; +INSERT INTO t2 SELECT number, number FROM system.numbers LIMIT 10; + + +SELECT ( + SELECT 111111111111 + ) AS first_column +FROM t2; + +SELECT 1; + +SELECT ( + SELECT 2222222222 + ) AS second_column +FROM t2; From f6a240b7be2982b4625ffdc21e1ceeee4cd68859 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Apr 2024 10:11:56 -0300 Subject: [PATCH 105/243] add some unit tests --- src/IO/S3/URI.h | 1 + src/IO/tests/gtest_s3_uri.cpp | 20 ++++++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index 2873728bc78..79f3da3fbbb 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -17,6 +17,7 @@ namespace DB::S3 * The following patterns are allowed: * s3://bucket/key * http(s)://endpoint/bucket/key + * TODO specify aws private link */ struct URI { diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 5bf0dfb962d..b3ceb875362 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -74,6 +74,26 @@ const TestCase TestCases[] = { "data", "", true}, + {S3::URI("https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/root/nested/file.txt"), + "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com", + "root", + "nested/file.txt", + "", + true}, + // Test with a file with no extension + {S3::URI("https://bucket.vpce-03b2c987f1bd55c5f-j3b4vg7w.s3.ap-southeast-2.vpce.amazonaws.com/some_bucket/document"), + "https://bucket.vpce-03b2c987f1bd55c5f-j3b4vg7w.s3.ap-southeast-2.vpce.amazonaws.com", + "some_bucket", + "document", + "", + true}, + // Test with a deeply nested file path + {S3::URI("https://bucket.vpce-0242cd56f1bd55c5f-l5b7vg8x.s3.sa-east-1.vpce.amazonaws.com/some_bucket/b/c/d/e/f/g/h/i/j/data.json"), + "https://bucket.vpce-0242cd56f1bd55c5f-l5b7vg8x.s3.sa-east-1.vpce.amazonaws.com", + "some_bucket", + "b/c/d/e/f/g/h/i/j/data.json", + "", + true}, }; class S3UriTest : public testing::TestWithParam From 8e6cbc8b31c93e3825219cc47463c0e854b0a26d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 3 Apr 2024 15:13:59 +0200 Subject: [PATCH 106/243] several fixes for client's keep alive connections --- .../Net/include/Poco/Net/HTTPClientSession.h | 13 + base/poco/Net/include/Poco/Net/HTTPMessage.h | 3 + .../Net/include/Poco/Net/HTTPServerParams.h | 2 +- .../Net/include/Poco/Net/HTTPServerSession.h | 2 + base/poco/Net/src/HTTPClientSession.cpp | 39 +- base/poco/Net/src/HTTPMessage.cpp | 41 ++ base/poco/Net/src/HTTPServerConnection.cpp | 13 +- base/poco/Net/src/HTTPServerSession.cpp | 6 + src/Common/HTTPConnectionPool.cpp | 94 ++-- src/Common/tests/gtest_connection_pool.cpp | 423 ++++++++++++++---- src/Core/ServerSettings.h | 4 +- src/IO/ConnectionTimeouts.cpp | 6 +- 12 files changed, 502 insertions(+), 144 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index 1cef988566c..b418937c4d5 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -213,6 +213,13 @@ namespace Net Poco::Timespan getKeepAliveTimeout() const; /// Returns the connection timeout for HTTP connections. + bool isKeepAliveExpired(double reliability = 1.0) const; + /// Returns if the connection is expired with some margin as fraction of timeout as reliability + + double getKeepAliveReliability() const; + /// Returns the current fraction of keep alive timeout when connection is considered safe to use + /// It helps to avoid situation when a client uses nearly expired connection and receives NoMessageException + virtual std::ostream & sendRequest(HTTPRequest & request); /// Sends the header for the given HTTP request to /// the server. @@ -361,6 +368,7 @@ namespace Net Poco::SharedPtr _pRequestStream; Poco::SharedPtr _pResponseStream; + static const double _defaultKeepAliveReliabilityLevel; static ProxyConfig _globalProxyConfig; HTTPClientSession(const HTTPClientSession &); @@ -455,6 +463,11 @@ namespace Net _lastRequest = time; } + inline double HTTPClientSession::getKeepAliveReliability() const + { + return _defaultKeepAliveReliabilityLevel; + } + } } // namespace Poco::Net diff --git a/base/poco/Net/include/Poco/Net/HTTPMessage.h b/base/poco/Net/include/Poco/Net/HTTPMessage.h index 0bef50803a8..994807ffbff 100644 --- a/base/poco/Net/include/Poco/Net/HTTPMessage.h +++ b/base/poco/Net/include/Poco/Net/HTTPMessage.h @@ -120,6 +120,9 @@ namespace Net /// The value is set to "Keep-Alive" if keepAlive is /// true, or to "Close" otherwise. + void setKeepAliveTimeout(int timeout); + int getKeepAliveTimeout() const; + bool getKeepAlive() const; /// Returns true if /// * the message has a Connection header field and its value is "Keep-Alive" diff --git a/base/poco/Net/include/Poco/Net/HTTPServerParams.h b/base/poco/Net/include/Poco/Net/HTTPServerParams.h index 3c836a630a0..d614c62d57a 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerParams.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerParams.h @@ -44,7 +44,7 @@ namespace Net /// - timeout: 60 seconds /// - keepAlive: true /// - maxKeepAliveRequests: 0 - /// - keepAliveTimeout: 10 seconds + /// - keepAliveTimeout: 15 seconds void setServerName(const std::string & serverName); /// Sets the name and port (name:port) that the server uses to identify itself. diff --git a/base/poco/Net/include/Poco/Net/HTTPServerSession.h b/base/poco/Net/include/Poco/Net/HTTPServerSession.h index ec928af304f..3df7995509a 100644 --- a/base/poco/Net/include/Poco/Net/HTTPServerSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPServerSession.h @@ -56,6 +56,8 @@ namespace Net SocketAddress serverAddress(); /// Returns the server's address. + void setKeepAliveTimeout(Poco::Timespan keepAliveTimeout); + private: bool _firstRequest; Poco::Timespan _keepAliveTimeout; diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index 33a3dcc4901..59800232ba9 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -37,6 +37,7 @@ namespace Net { HTTPClientSession::ProxyConfig HTTPClientSession::_globalProxyConfig; +const double HTTPClientSession::_defaultKeepAliveReliabilityLevel = 0.9; HTTPClientSession::HTTPClientSession(): @@ -220,7 +221,11 @@ void HTTPClientSession::setGlobalProxyConfig(const ProxyConfig& config) void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout) { - _keepAliveTimeout = timeout; + if (connected()) + { + throw Poco::IllegalStateException("cannot change keep alive timeout on initiated connection"); + } + _keepAliveTimeout = timeout; } @@ -243,6 +248,8 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) reconnect(); if (!keepAlive) request.setKeepAlive(false); + if (keepAlive && !request.has(HTTPMessage::CONNECTION_KEEP_ALIVE) && _keepAliveTimeout.totalSeconds() > 0) + request.setKeepAliveTimeout(_keepAliveTimeout.totalSeconds()); if (!request.has(HTTPRequest::HOST) && !_host.empty()) request.setHost(_host, _port); if (!_proxyConfig.host.empty() && !bypassProxy()) @@ -324,6 +331,14 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response) _mustReconnect = getKeepAlive() && !response.getKeepAlive(); + if (!_mustReconnect) + { + /// when server sends its keep alive timeout, client has to follow that value + auto timeout = response.getKeepAliveTimeout(); + if (timeout > 0) + _keepAliveTimeout = Poco::Timespan(timeout, 0); + } + if (!_expectResponseBody || response.getStatus() < 200 || response.getStatus() == HTTPResponse::HTTP_NO_CONTENT || response.getStatus() == HTTPResponse::HTTP_NOT_MODIFIED) _pResponseStream = new HTTPFixedLengthInputStream(*this, 0); else if (response.getChunkedTransferEncoding()) @@ -430,15 +445,17 @@ std::string HTTPClientSession::proxyRequestPrefix() const return result; } +bool HTTPClientSession::isKeepAliveExpired(double reliability) const +{ + Poco::Timestamp now; + return Timespan(Timestamp::TimeDiff(reliability *_keepAliveTimeout.totalMicroseconds())) <= now - _lastRequest; +} bool HTTPClientSession::mustReconnect() const { if (!_mustReconnect) - { - Poco::Timestamp now; - return _keepAliveTimeout <= now - _lastRequest; - } - else return true; + return isKeepAliveExpired(_defaultKeepAliveReliabilityLevel); + return true; } @@ -511,14 +528,16 @@ void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session) if (buffered()) throw Poco::LogicException("assign to a session with not empty buffered data"); - attachSocket(session.detachSocket()); setLastRequest(session.getLastRequest()); setResolvedHost(session.getResolvedHost()); - setKeepAlive(session.getKeepAlive()); + setProxyConfig(session.getProxyConfig()); setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout()); - setKeepAliveTimeout(session.getKeepAliveTimeout()); - setProxyConfig(session.getProxyConfig()); + setKeepAlive(session.getKeepAlive()); + if (!connected()) + setKeepAliveTimeout(session.getKeepAliveTimeout()); + + attachSocket(session.detachSocket()); session.reset(); } diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index 0cd234ee9cb..2f974b8bf0b 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -17,6 +17,7 @@ #include "Poco/NumberFormatter.h" #include "Poco/NumberParser.h" #include "Poco/String.h" +#include using Poco::NumberFormatter; @@ -179,4 +180,44 @@ bool HTTPMessage::getKeepAlive() const } +void HTTPMessage::setKeepAliveTimeout(int timeout) +{ + add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}", timeout)); +} + + +int parseTimeoutFromHeaderValue(const std::string_view header_value) +{ + static const std::string_view timeout_param = "timeout="; + + auto timeout_pos = header_value.find(timeout_param); + if (timeout_pos == std::string::npos) + timeout_pos = header_value.size(); + if (timeout_pos != header_value.size()) + timeout_pos += timeout_param.size(); + + auto timeout_end = header_value.find(',', timeout_pos); + if (timeout_end == std::string::npos) + timeout_end = header_value.size(); + + auto timeout_value_substr = header_value.substr(timeout_pos, timeout_end - timeout_pos); + if (timeout_value_substr.empty()) + return -1; + + int value = 0; + auto [ptr, ec] = std::from_chars(timeout_value_substr.begin(), timeout_value_substr.end(), value); + + if (ec == std::errc()) + return value; + + return -1; +} + + +int HTTPMessage::getKeepAliveTimeout() const +{ + const std::string& ka_header = get(HTTPMessage::CONNECTION_KEEP_ALIVE, HTTPMessage::EMPTY); + return parseTimeoutFromHeaderValue(ka_header); +} + } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPServerConnection.cpp b/base/poco/Net/src/HTTPServerConnection.cpp index c57984b0162..d5eb29d3134 100644 --- a/base/poco/Net/src/HTTPServerConnection.cpp +++ b/base/poco/Net/src/HTTPServerConnection.cpp @@ -88,7 +88,18 @@ void HTTPServerConnection::run() pHandler->handleRequest(request, response); session.setKeepAlive(_pParams->getKeepAlive() && response.getKeepAlive() && session.canKeepAlive()); - } + + /// all that fuzz is all about to make session close with less timeout than 15s (set in HTTPServerParams c-tor) + if (_pParams->getKeepAlive() && response.getKeepAlive() && session.canKeepAlive()) + { + int value = response.getKeepAliveTimeout(); + if (value < 0) + value = request.getKeepAliveTimeout(); + if (value > 0) + session.setKeepAliveTimeout(Poco::Timespan(value, 0)); + } + + } else sendErrorResponse(session, HTTPResponse::HTTP_NOT_IMPLEMENTED); } catch (Poco::Exception&) diff --git a/base/poco/Net/src/HTTPServerSession.cpp b/base/poco/Net/src/HTTPServerSession.cpp index d4f2b24879e..f67a63a9e0e 100644 --- a/base/poco/Net/src/HTTPServerSession.cpp +++ b/base/poco/Net/src/HTTPServerSession.cpp @@ -33,6 +33,12 @@ HTTPServerSession::~HTTPServerSession() { } +void HTTPServerSession::setKeepAliveTimeout(Poco::Timespan keepAliveTimeout) +{ + _keepAliveTimeout = keepAliveTimeout; +} + + bool HTTPServerSession::hasMoreRequests() { diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index cd2505df7f3..21165bbc62d 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -83,17 +84,15 @@ namespace } - size_t roundUp(size_t x, size_t rounding) + constexpr size_t roundUp(size_t x, size_t rounding) { chassert(rounding > 0); - return (x + (rounding - 1)) / rounding * rounding; - } - - - Poco::Timespan divide(const Poco::Timespan span, int divisor) - { - return Poco::Timespan(Poco::Timestamp::TimeDiff(span.totalMicroseconds() / divisor)); + return (x + rounding) / rounding * rounding; } + static_assert(roundUp(10000, 100) == 10100); + static_assert(roundUp(10001, 100) == 10100); + static_assert(roundUp(10099, 100) == 10100); + static_assert(roundUp(10100, 100) == 10200); } namespace DB @@ -202,8 +201,9 @@ public: if (total_connections_in_group >= limits.warning_limit && total_connections_in_group >= mute_warning_until) { - LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}", type, total_connections_in_group, limits.warning_limit); mute_warning_until = roundUp(total_connections_in_group, limits.warning_step); + LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}, next warning at {}", + type, total_connections_in_group, limits.warning_limit, mute_warning_until); } } @@ -213,7 +213,7 @@ public: --total_connections_in_group; - const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 10 : 1; + const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 20 : 1; if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit) { LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group); @@ -221,6 +221,12 @@ public: } } + void atPoolDestroy(size_t connections) + { + std::lock_guard lock(mutex); + total_connections_in_group -= connections; + } + HTTPConnectionGroupType getType() const { return type; } const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const { return metrics; } @@ -273,9 +279,15 @@ private: public: using Ptr = std::shared_ptr; + using Session::mustReconnect; + + void markAsExpired() + { + isExpired = true; + } + void reconnect() override { - ProfileEvents::increment(metrics.reset); Session::close(); if (auto lock = pool.lock()) @@ -352,6 +364,11 @@ private: std::istream & result = Session::receiveResponse(response); result.exceptions(std::ios::badbit); + // that line is for temporary debug, will be removed + if (response.has(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE)) + LOG_WARNING(log, "received keep alive header: {}", + response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY)); + response_stream = &result; response_stream_completed = false; @@ -392,10 +409,11 @@ private: } response_stream = nullptr; - if (auto lock = pool.lock()) - lock->atConnectionDestroy(*this); - else - ProfileEvents::increment(metrics.reset); + group->atConnectionDestroy(); + + if (!isExpired) + if (auto lock = pool.lock()) + lock->atConnectionDestroy(*this); CurrentMetrics::sub(metrics.active_count); } @@ -404,10 +422,11 @@ private: friend class EndpointConnectionPool; template - explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) - : Session(args...), pool(std::move(pool_)), metrics(std::move(metrics_)) + explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, ConnectionGroup::Ptr group_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) + : Session(args...), pool(std::move(pool_)), group(group_), metrics(std::move(metrics_)) { CurrentMetrics::add(metrics.active_count); + group->atConnectionCreate(); } template @@ -433,10 +452,12 @@ private: return request_stream_completed && response_stream_completed; } - WeakPtr pool; + EndpointConnectionPool::WeakPtr pool; + ConnectionGroup::Ptr group; IHTTPConnectionPoolForEndpoint::Metrics metrics; + bool isExpired = false; - Poco::Logger * log = &Poco::Logger::get("PooledConnection"); + LoggerPtr log = getLogger("PooledConnection"); std::ostream * request_stream = nullptr; std::istream * response_stream = nullptr; @@ -484,7 +505,6 @@ public: IHTTPConnectionPoolForEndpoint::ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) override { - Poco::Timestamp now; std::vector expired_connections; SCOPE_EXIT({ @@ -494,8 +514,9 @@ public: { std::lock_guard lock(mutex); + expired_connections.reserve(stored_connections.size()); - wipeExpiredImpl(expired_connections, now); + wipeExpiredImpl(expired_connections); if (!stored_connections.empty()) { @@ -526,7 +547,6 @@ public: size_t wipeExpired() override { - Poco::Timestamp now; std::vector expired_connections; SCOPE_EXIT({ @@ -535,19 +555,21 @@ public: }); std::lock_guard lock(mutex); - return wipeExpiredImpl(expired_connections, now); + return wipeExpiredImpl(expired_connections); } - size_t wipeExpiredImpl(std::vector & expired_connections, Poco::Timestamp now) TSA_REQUIRES(mutex) + size_t wipeExpiredImpl(std::vector & expired_connections) TSA_REQUIRES(mutex) { + auto isSoftLimitReached = group->isSoftLimitReached(); while (!stored_connections.empty()) { auto connection = stored_connections.top(); - if (!isExpired(now, connection)) + if (!isExpired(connection, isSoftLimitReached)) return stored_connections.size(); stored_connections.pop(); + connection->markAsExpired(); expired_connections.push_back(connection); } @@ -569,16 +591,16 @@ private: WeakPtr getWeakFromThis() { return EndpointConnectionPool::weak_from_this(); } - bool isExpired(Poco::Timestamp & now, ConnectionPtr connection) + bool isExpired(ConnectionPtr connection, bool isSoftLimitReached) TSA_REQUIRES(mutex) { - if (group->isSoftLimitReached()) - return now > (connection->getLastRequest() + divide(connection->getKeepAliveTimeout(), 10)); - return now > connection->getLastRequest() + connection->getKeepAliveTimeout(); + if (isSoftLimitReached) + return connection->isKeepAliveExpired(0.1); + return connection->isKeepAliveExpired(0.8); } ConnectionPtr allocateNewConnection() { - ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), getMetrics(), host, port); + ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); connection->setKeepAlive(true); if (!proxy_configuration.isEmpty()) @@ -586,8 +608,6 @@ private: connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); } - group->atConnectionCreate(); - return connection; } @@ -619,8 +639,6 @@ private: void atConnectionDestroy(PooledConnection & connection) { - group->atConnectionDestroy(); - if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() || group->isStoreLimitReached()) { @@ -631,14 +649,14 @@ private: auto connection_to_store = allocateNewConnection(); connection_to_store->assign(connection); - CurrentMetrics::add(getMetrics().stored_count, 1); - ProfileEvents::increment(getMetrics().preserved, 1); - { MemoryTrackerSwitcher switcher{&total_memory_tracker}; std::lock_guard lock(mutex); stored_connections.push(connection_to_store); } + + CurrentMetrics::add(getMetrics().stored_count, 1); + ProfileEvents::increment(getMetrics().preserved, 1); } @@ -726,7 +744,7 @@ createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port, class HTTPConnectionPools::Impl { private: - const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 5 * 60; + const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 10 * 60; const Poco::Timespan wipe_timeout = Poco::Timespan(DEFAULT_WIPE_TIMEOUT_SECONDS, 0); ConnectionGroup::Ptr disk_group = std::make_shared(HTTPConnectionGroupType::DISK); diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp index dcc3c11fd52..36bf8bc7dae 100644 --- a/src/Common/tests/gtest_connection_pool.cpp +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -17,6 +16,39 @@ namespace { +template +class SafeHandler +{ +public: + using Ptr = std::shared_ptr>; + + SafeHandler() = default; + SafeHandler(SafeHandler&) = delete; + SafeHandler& operator=(SafeHandler&) = delete; + + T get() + { + std::lock_guard lock(mutex); + return obj; + } + + void set(T && options_) + { + std::lock_guard lock(mutex); + obj = std::move(options_); + } + +protected: + std::mutex mutex; + T obj = {}; +}; + +struct RequestOptions +{ + size_t slowdown_receive = 0; + int overwrite_keep_alive_timeout = 0; +}; + size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits::max()) { const size_t buffer_size = 4096; @@ -47,13 +79,19 @@ size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = class MockRequestHandler : public Poco::Net::HTTPRequestHandler { public: - explicit MockRequestHandler(std::shared_ptr> slowdown_) - : slowdown(std::move(slowdown_)) + explicit MockRequestHandler(SafeHandler::Ptr options_) + : options(options_) { } void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override { + int value = request.getKeepAliveTimeout(); + ASSERT_GT(value, 0); + + if (options->get().overwrite_keep_alive_timeout > 0) + response.setKeepAliveTimeout(options->get().overwrite_keep_alive_timeout); + response.setStatus(Poco::Net::HTTPResponse::HTTP_OK); auto size = request.getContentLength(); if (size > 0) @@ -61,28 +99,29 @@ public: else response.setChunkedTransferEncoding(true); // or chunk encoding - sleepForSeconds(*slowdown); + if (options->get().slowdown_receive > 0) + sleepForSeconds(options->get().slowdown_receive); stream_copy_n(request.stream(), response.send(), size); } - std::shared_ptr> slowdown; + SafeHandler::Ptr options; }; class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory { public: - explicit HTTPRequestHandlerFactory(std::shared_ptr> slowdown_) - : slowdown(std::move(slowdown_)) + explicit HTTPRequestHandlerFactory(SafeHandler::Ptr options_) + : options(options_) { } Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override { - return new MockRequestHandler(slowdown); + return new MockRequestHandler(options); } - std::shared_ptr> slowdown; + SafeHandler::Ptr options; }; } @@ -94,6 +133,8 @@ class ConnectionPoolTest : public testing::Test { protected: ConnectionPoolTest() { + options = std::make_shared>(); + startServer(); } @@ -102,7 +143,7 @@ protected: DB::HTTPConnectionPools::Limits def_limits{}; DB::HTTPConnectionPools::instance().setLimits(def_limits, def_limits, def_limits); - setSlowDown(0); + options->set(RequestOptions()); DB::HTTPConnectionPools::instance().dropCache(); DB::CurrentThread::getProfileEvents().reset(); @@ -129,7 +170,7 @@ protected: void startServer() { server_data.reset(); - server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive); + server_data.handler_factory = new HTTPRequestHandlerFactory(options); server_data.server = std::make_unique( server_data.handler_factory, server_data.port); @@ -143,11 +184,20 @@ protected: void setSlowDown(size_t seconds) { - *slowdown_receive = seconds; + auto opt = options->get(); + opt.slowdown_receive = seconds; + options->set(std::move(opt)); + } + + void setOverWriteTimeout(size_t seconds) + { + auto opt = options->get(); + opt.overwrite_keep_alive_timeout = int(seconds); + options->set(std::move(opt)); } DB::ConnectionTimeouts timeouts; - std::shared_ptr> slowdown_receive = std::make_shared>(0); + SafeHandler::Ptr options; struct ServerData { @@ -182,7 +232,7 @@ protected: void wait_until(std::function pred) { while (!pred()) - sleepForMilliseconds(250); + sleepForMilliseconds(10); } void echoRequest(String data, HTTPSession & session) @@ -245,45 +295,52 @@ TEST_F(ConnectionPoolTest, CanRequest) ASSERT_EQ(0, getServer().currentConnections()); ASSERT_EQ(1, getServer().totalConnections()); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); + auto metrics = pool->getMetrics(); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, CanPreserve) { auto pool = getPool(); + auto metrics = pool->getMetrics(); { auto connection = pool->getConnection(timeouts); } - ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); wait_until([&] () { return getServer().currentConnections() == 1; }); ASSERT_EQ(1, getServer().currentConnections()); - - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); } TEST_F(ConnectionPoolTest, CanReuse) { auto pool = getPool(); + auto metrics = pool->getMetrics(); { auto connection = pool->getConnection(timeouts); - // DB::setReuseTag(*connection); } - ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count)); - { auto connection = pool->getConnection(timeouts); - ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); wait_until([&] () { return getServer().currentConnections() == 1; }); ASSERT_EQ(1, getServer().currentConnections()); @@ -293,6 +350,11 @@ TEST_F(ConnectionPoolTest, CanReuse) ASSERT_EQ(1, getServer().totalConnections()); ASSERT_EQ(1, getServer().currentConnections()); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + connection->reset(); } @@ -303,15 +365,16 @@ TEST_F(ConnectionPoolTest, CanReuse) ASSERT_EQ(0, getServer().currentConnections()); ASSERT_EQ(1, getServer().totalConnections()); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); } TEST_F(ConnectionPoolTest, CanReuse10) { auto pool = getPool(); - + auto metrics = pool->getMetrics(); for (int i = 0; i < 10; ++i) { @@ -328,16 +391,23 @@ TEST_F(ConnectionPoolTest, CanReuse10) ASSERT_EQ(0, getServer().currentConnections()); ASSERT_EQ(1, getServer().totalConnections()); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, CanReuse5) { - timeouts.withHTTPKeepAliveTimeout(1); + auto ka = Poco::Timespan(1, 0); // 1 seconds + timeouts.withHTTPKeepAliveTimeout(ka); auto pool = getPool(); + auto metrics = pool->getMetrics(); std::vector connections; connections.reserve(5); @@ -347,11 +417,14 @@ TEST_F(ConnectionPoolTest, CanReuse5) } connections.clear(); - ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); - ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(5, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(5, CurrentMetrics::get(metrics.stored_count)); wait_until([&] () { return getServer().currentConnections() == 5; }); ASSERT_EQ(5, getServer().currentConnections()); @@ -363,35 +436,56 @@ TEST_F(ConnectionPoolTest, CanReuse5) echoRequest("Hello", *connection); } - ASSERT_EQ(5, getServer().totalConnections()); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); - ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); - ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(5, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(5, CurrentMetrics::get(metrics.stored_count)); + + /// wait until all connections are timeouted + wait_until([&] () { return getServer().currentConnections() == 0; }); + + { + // just to trigger pool->wipeExpired(); + auto connection = pool->getConnection(timeouts); + connection->reset(); + } + + ASSERT_EQ(6, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, CanReconnectAndCreate) { auto pool = getPool(); + auto metrics = pool->getMetrics(); std::vector in_use; - const size_t count = 2; + const size_t count = 3; for (int i = 0; i < count; ++i) { auto connection = pool->getConnection(timeouts); - // DB::setReuseTag(*connection); in_use.push_back(connection); } - ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); - ASSERT_EQ(count, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(count, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); auto connection = std::move(in_use.back()); in_use.pop_back(); @@ -402,28 +496,39 @@ TEST_F(ConnectionPoolTest, CanReconnectAndCreate) echoRequest("Hello", *connection); - connection->reset(); + ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); - wait_until([&] () { return getServer().currentConnections() == 1; }); - ASSERT_EQ(1, getServer().currentConnections()); - ASSERT_EQ(count+1, getServer().totalConnections()); - - ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(count, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, CanReconnectAndReuse) { + auto ka = Poco::Timespan(1, 0); // 1 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + auto pool = getPool(); + auto metrics = pool->getMetrics(); std::vector in_use; - const size_t count = 2; + const size_t count = 3; + for (int i = 0; i < count; ++i) + { + auto connection = pool->getConnection(timeouts); + /// make some request in order to show to the server the keep alive headers + echoRequest("Hello", *connection); + in_use.push_back(std::move(connection)); + } + in_use.clear(); + for (int i = 0; i < count; ++i) { auto connection = pool->getConnection(timeouts); - // DB::setReuseTag(*connection); in_use.push_back(std::move(connection)); } @@ -441,11 +546,16 @@ TEST_F(ConnectionPoolTest, CanReconnectAndReuse) wait_until([&] () { return getServer().currentConnections() == 0; }); ASSERT_EQ(0, getServer().currentConnections()); - ASSERT_EQ(2, getServer().totalConnections()); + ASSERT_EQ(count, getServer().totalConnections()); - ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); + ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(count + count - 1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(count + 1, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(count-1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(count-2, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, ReceiveTimeout) @@ -454,6 +564,7 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout) timeouts.withReceiveTimeout(1); auto pool = getPool(); + auto metrics = pool->getMetrics(); { auto connection = pool->getConnection(timeouts); @@ -462,10 +573,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout) ); } - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); { timeouts.withReceiveTimeout(3); @@ -475,10 +590,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout) ); } - ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); { /// timeouts have effect for reused session @@ -489,10 +608,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout) ); } - ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]); - ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP) @@ -500,6 +623,7 @@ TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP) std::string_view message = "Hello ReadWriteBufferFromHTTP"; auto uri = Poco::URI(getServerUrl()); auto metrics = DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{})->getMetrics(); + Poco::Net::HTTPBasicCredentials empty_creds; auto buf_from_http = DB::BuilderRWBufferFromHTTP(uri) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) @@ -527,6 +651,7 @@ TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP) ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); @@ -538,23 +663,26 @@ TEST_F(ConnectionPoolTest, HardLimit) DB::HTTPConnectionPools::instance().setLimits(zero_limits, zero_limits, zero_limits); auto pool = getPool(); + auto metrics = pool->getMetrics(); { auto connection = pool->getConnection(timeouts); } - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); - - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); } TEST_F(ConnectionPoolTest, NoReceiveCall) { auto pool = getPool(); + auto metrics = pool->getMetrics(); { auto connection = pool->getConnection(timeouts); @@ -570,11 +698,124 @@ TEST_F(ConnectionPoolTest, NoReceiveCall) connection->flushRequest(); } - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count)); - ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count)); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); - - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]); - ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]); - ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]); + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); +} + +TEST_F(ConnectionPoolTest, ReconnectedWhenConnectionIsHoldTooLong) +{ + auto ka = Poco::Timespan(1, 0); // 1 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + + auto pool = getPool(); + auto metrics = pool->getMetrics(); + + { + auto connection = pool->getConnection(timeouts); + + echoRequest("Hello", *connection); + + auto fake_ka = Poco::Timespan(30 * 1000 * 1000); // 30 seconds + timeouts.withHTTPKeepAliveTimeout(fake_ka); + DB::setTimeouts(*connection, timeouts); // new keep alive timeout has no effect + + wait_until([&] () { return getServer().currentConnections() == 0; }); + + ASSERT_EQ(1, connection->connected()); + ASSERT_EQ(1, connection->getKeepAlive()); + ASSERT_EQ(1000, connection->getKeepAliveTimeout().totalMilliseconds()); + ASSERT_EQ(1, connection->isKeepAliveExpired(connection->getKeepAliveReliability())); + + echoRequest("Hello", *connection); + } + + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); +} + +TEST_F(ConnectionPoolTest, ReconnectedWhenConnectionIsNearlyExpired) +{ + auto ka = Poco::Timespan(1, 0); // 1 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + + auto pool = getPool(); + auto metrics = pool->getMetrics(); + + { + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + + sleepForMilliseconds(900); + + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + } + } + + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); +} + +TEST_F(ConnectionPoolTest, ServerOverwriteKeepAlive) +{ + auto ka = Poco::Timespan(30, 0); // 30 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + + auto pool = getPool(); + auto metrics = pool->getMetrics(); + + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds()); + ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds()); + } + + { + setOverWriteTimeout(1); + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds()); + ASSERT_EQ(1, connection->getKeepAliveTimeout().totalSeconds()); + } + + { + // server do not overwrite it in the following requests but client has to remember last agreed value + setOverWriteTimeout(0); + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds()); + ASSERT_EQ(1, connection->getKeepAliveTimeout().totalSeconds()); + } + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(3, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); } diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 6608a35a5a2..8d6d8ebc1a2 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -128,9 +128,9 @@ namespace DB M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \ M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \ M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \ - M(UInt64, disk_connections_soft_limit, 1000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_soft_limit, 5000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \ M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \ - M(UInt64, disk_connections_store_limit, 12000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \ + M(UInt64, disk_connections_store_limit, 30000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \ M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \ M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \ M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \ diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index c4b636103fe..8813c958185 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -144,7 +144,11 @@ ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) { session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); - session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); + /// we can not change keep alive timeout for already initiated connections + if (!session.connected()) + { + session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); + } } ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session) From 0bc743b4e2144d6de39500aa753d9d02675caa18 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 15:24:16 +0200 Subject: [PATCH 107/243] Close: https://github.com/ClickHouse/ClickHouse/issues/35608 --- ...alyzer_numeric_literals_as_column_names.reference | 2 ++ ...079_analyzer_numeric_literals_as_column_names.sql | 12 ++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.reference create mode 100644 tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql diff --git a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.reference b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.reference new file mode 100644 index 00000000000..faff07c519f --- /dev/null +++ b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.reference @@ -0,0 +1,2 @@ +\N 1 +str diff --git a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql new file mode 100644 index 00000000000..e6bcad34506 --- /dev/null +++ b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql @@ -0,0 +1,12 @@ +CREATE TABLE testdata (`1` String) ENGINE=MergeTree ORDER BY tuple(); +INSERT INTO testdata VALUES ('testdata'); + +SELECT * +FROM ( + SELECT if(isValidUTF8(`1`), NULL, 'error!') AS error_message, + if(error_message IS NULL, 1, 0) AS valid + FROM testdata +) +WHERE valid; + +select * from (select 'str' as `1`) where 1; From 6c14f6ecf2aba16e407fe887ef38e59d9931510d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Apr 2024 13:25:04 +0000 Subject: [PATCH 108/243] Close: https://github.com/ClickHouse/ClickHouse/pull/62185 --- .../03033_cte_numbers_memory.reference | 10 ++++++++++ .../0_stateless/03033_cte_numbers_memory.sql | 15 +++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03033_cte_numbers_memory.reference create mode 100644 tests/queries/0_stateless/03033_cte_numbers_memory.sql diff --git a/tests/queries/0_stateless/03033_cte_numbers_memory.reference b/tests/queries/0_stateless/03033_cte_numbers_memory.reference new file mode 100644 index 00000000000..8d2470dea44 --- /dev/null +++ b/tests/queries/0_stateless/03033_cte_numbers_memory.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/03033_cte_numbers_memory.sql b/tests/queries/0_stateless/03033_cte_numbers_memory.sql new file mode 100644 index 00000000000..0e3ee9abd65 --- /dev/null +++ b/tests/queries/0_stateless/03033_cte_numbers_memory.sql @@ -0,0 +1,15 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61238 + +WITH +(SELECT number FROM system.numbers LIMIT 1) as w1, +(SELECT number FROM system.numbers LIMIT 1) as w2, +(SELECT number FROM system.numbers LIMIT 1) as w3, +(SELECT number FROM system.numbers LIMIT 1) as w4, +(SELECT number FROM system.numbers LIMIT 1) as w5, +(SELECT number FROM system.numbers LIMIT 1) as w6 +SELECT number FROM ( + SELECT number FROM system.numbers LIMIT 10 + UNION ALL + SELECT number FROM system.numbers LIMIT 10 +) +WHERE number < 5; From 922a14eaf1fd22d1a364ec285851c50cbb2ad54f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 3 Apr 2024 15:33:35 +0200 Subject: [PATCH 109/243] fix stored_count metric --- src/Common/HTTPConnectionPool.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 21165bbc62d..ab0ea1571d3 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -560,6 +560,11 @@ public: size_t wipeExpiredImpl(std::vector & expired_connections) TSA_REQUIRES(mutex) { + SCOPE_EXIT({ + CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size()); + ProfileEvents::increment(getMetrics().expired, expired_connections.size()); + }); + auto isSoftLimitReached = group->isSoftLimitReached(); while (!stored_connections.empty()) { @@ -573,9 +578,6 @@ public: expired_connections.push_back(connection); } - CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size()); - ProfileEvents::increment(getMetrics().expired, expired_connections.size()); - return stored_connections.size(); } From 394a4f39ecf12867344f68ac4730ea34abf8b0dc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Apr 2024 10:37:39 -0300 Subject: [PATCH 110/243] remove unnecessary matching character --- src/IO/S3/URI.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 7f628d27f34..8e364337db5 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -40,7 +40,7 @@ URI::URI(const std::string & uri_) /// Case when AWS Private Link Interface is being used /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html - static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.:]+)\.vpce.amazonaws.com)"); + static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce.amazonaws.com)"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) From ee1c177d10a3c333662ba774af7fb2618193fac1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Apr 2024 13:49:53 +0000 Subject: [PATCH 111/243] Closes: https://github.com/ClickHouse/ClickHouse/issues/29838 --- .../03080_incorrect_join_with_merge.reference | 2 + .../03080_incorrect_join_with_merge.sql | 66 +++++++++++++++++++ 2 files changed, 68 insertions(+) create mode 100644 tests/queries/0_stateless/03080_incorrect_join_with_merge.reference create mode 100644 tests/queries/0_stateless/03080_incorrect_join_with_merge.sql diff --git a/tests/queries/0_stateless/03080_incorrect_join_with_merge.reference b/tests/queries/0_stateless/03080_incorrect_join_with_merge.reference new file mode 100644 index 00000000000..51993f072d5 --- /dev/null +++ b/tests/queries/0_stateless/03080_incorrect_join_with_merge.reference @@ -0,0 +1,2 @@ +2 +2 diff --git a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql new file mode 100644 index 00000000000..ae8e40f6d56 --- /dev/null +++ b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql @@ -0,0 +1,66 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/29838 + +CREATE TABLE first_table_lr +( + id String, + id2 String +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_03080/alter', 'r1') +ORDER BY id; + + +CREATE TABLE first_table +( + id String, + id2 String +) +ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'first_table_lr'); + + +CREATE TABLE second_table_lr +( + id String, + id2 String +) ENGINE = MergeTree() +ORDER BY id; + +CREATE TABLE second_table +( + id String, + id2 String +) +ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'second_table_lr'); + +INSERT INTO first_table VALUES ('1', '2'), ('3', '4'); +INSERT INTO second_table VALUES ('1', '2'), ('3', '4'); + +CREATE TABLE two_tables +( + id String, + id2 String +) +ENGINE = Merge(currentDatabase(), '^(first_table)$'); + +SELECT + count() +FROM first_table as s +GLOBAL ANY JOIN second_table as f USING (id) +WHERE + f.id2 GLOBAL IN ( + SELECT + id2 + FROM second_table + GROUP BY id2 + ); + +SELECT + count() +FROM two_tables as s +GLOBAL ANY JOIN second_table as f USING (id) +WHERE + f.id2 GLOBAL IN ( + SELECT + id2 + FROM second_table + GROUP BY id2 + ); From 4bb8f12374dc58fc87ed76fb7abe04b926500ec8 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 15:32:12 +0200 Subject: [PATCH 112/243] Close: https://github.com/ClickHouse/ClickHouse/issues/35652 --- ...n_name_to_alias__virtual_columns.reference | 3 +++ ..._column_name_to_alias__virtual_columns.sql | 27 +++++++++++++++++++ 2 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.reference create mode 100644 tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql diff --git a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.reference b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.reference new file mode 100644 index 00000000000..ccf161abe8d --- /dev/null +++ b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.reference @@ -0,0 +1,3 @@ +0 0 +0 0 + 0 0 diff --git a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql new file mode 100644 index 00000000000..1fe19cdad2a --- /dev/null +++ b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql @@ -0,0 +1,27 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/35652 +CREATE TABLE test ( + id UInt64 +) +ENGINE = MergeTree() +SAMPLE BY intHash32(id) +ORDER BY intHash32(id); + +SELECT + any(id), + any(id) AS id +FROM test +SETTINGS prefer_column_name_to_alias = 1; + +SELECT + any(_sample_factor), + any(_sample_factor) AS _sample_factor +FROM test +SETTINGS prefer_column_name_to_alias = 1; + +SELECT + any(_partition_id), + any(_sample_factor), + any(_partition_id) AS _partition_id, + any(_sample_factor) AS _sample_factor +FROM test +SETTINGS prefer_column_name_to_alias = 1; From c539a6ba8d4fc5b6adc8e2ef9a240db37dda3550 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 15:41:40 +0200 Subject: [PATCH 113/243] Close: https://github.com/ClickHouse/ClickHouse/issues/36189 --- .../03081_analyzer_agg_func_CTE.reference | 2 ++ .../03081_analyzer_agg_func_CTE.sql | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03081_analyzer_agg_func_CTE.reference create mode 100644 tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql diff --git a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.reference b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.reference new file mode 100644 index 00000000000..bf1b8926236 --- /dev/null +++ b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.reference @@ -0,0 +1,2 @@ +2020-01-01 +9 diff --git a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql new file mode 100644 index 00000000000..81dbbb3b62d --- /dev/null +++ b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql @@ -0,0 +1,18 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/36189 +CREATE TABLE test +( + `dt` Date, + `text` String +) +ENGINE = MergeTree +ORDER BY dt; + +insert into test values ('2020-01-01', 'text1'), ('2019-01-01', 'text2'), ('1900-01-01', 'text3'); + +WITH max(dt) AS maxDt +SELECT maxDt +FROM test; + +WITH max(number) AS maxDt +SELECT maxDt +FROM numbers(10); From 0cda6cf5233a1d24e3c02ea402ee3be4bfa61625 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 15:51:16 +0200 Subject: [PATCH 114/243] Close: https://github.com/ClickHouse/ClickHouse/issues/39634 --- ...nalyzer_left_join_correct_column.reference | 2 ++ ...3082_analyzer_left_join_correct_column.sql | 30 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/03082_analyzer_left_join_correct_column.reference create mode 100644 tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql diff --git a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.reference b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.reference new file mode 100644 index 00000000000..ad3f3d53ab5 --- /dev/null +++ b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.reference @@ -0,0 +1,2 @@ +pk1 2 +pk1 2 diff --git a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql new file mode 100644 index 00000000000..93702fee550 --- /dev/null +++ b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql @@ -0,0 +1,30 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/39634 +CREATE TABLE test1 +( + `pk` String, + `x.y` Decimal(18, 4) +) +ENGINE = MergeTree() +ORDER BY (pk); + +CREATE TABLE test2 +( + `pk` String, + `x.y` Decimal(18, 4) +) +ENGINE = MergeTree() +ORDER BY (pk); + +INSERT INTO test1 SELECT 'pk1', 1; + +INSERT INTO test2 SELECT 'pk1', 2; + +SELECT t1.pk, t2.x.y +FROM test1 t1 +LEFT JOIN test2 t2 + on t1.pk = t2.pk; + +SELECT t1.pk, t2.`x.y` +FROM test1 t1 +LEFT JOIN test2 t2 + on t1.pk = t2.pk; From 532d80e20b60987947ac11eb8c4991916742157f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 3 Apr 2024 16:02:07 +0200 Subject: [PATCH 115/243] fix log level in debug code --- src/Common/HTTPConnectionPool.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index ab0ea1571d3..2d3a87dda6b 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -361,13 +361,16 @@ private: std::istream & receiveResponse(Poco::Net::HTTPResponse & response) override { + int originKA = Session::getKeepAliveTimeout().totalSeconds(); + std::istream & result = Session::receiveResponse(response); result.exceptions(std::ios::badbit); // that line is for temporary debug, will be removed if (response.has(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE)) - LOG_WARNING(log, "received keep alive header: {}", - response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY)); + LOG_INFO(log, "received keep alive header: {}, original was {}", + response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), + originKA); response_stream = &result; response_stream_completed = false; From 0b53980221d419fc1d0227c851502cdaeafc7892 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 16:45:56 +0200 Subject: [PATCH 116/243] Close: https://github.com/ClickHouse/ClickHouse/issues/47432 --- ...03084_analyzer_join_column_alias.reference | 1 + .../03084_analyzer_join_column_alias.sql | 23 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03084_analyzer_join_column_alias.reference create mode 100644 tests/queries/0_stateless/03084_analyzer_join_column_alias.sql diff --git a/tests/queries/0_stateless/03084_analyzer_join_column_alias.reference b/tests/queries/0_stateless/03084_analyzer_join_column_alias.reference new file mode 100644 index 00000000000..acbb8406347 --- /dev/null +++ b/tests/queries/0_stateless/03084_analyzer_join_column_alias.reference @@ -0,0 +1 @@ +2023-01-01 diff --git a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql new file mode 100644 index 00000000000..caf65823532 --- /dev/null +++ b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql @@ -0,0 +1,23 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/47432 +create or replace table t1 +engine = MergeTree() +order by tuple() +as +select 1 as user_id, 2 as level; + + +create or replace table t2 +engine = MergeTree() +order by tuple() +as +select 1 as user_id, 'website' as event_source, '2023-01-01 00:00:00'::DateTime as timestamp; + + +alter table t2 +add column date Date alias toDate(timestamp); + +SELECT + any(t2.date) as any_val +FROM t1 AS t1 +LEFT JOIN t2 as t2 + ON (t1.user_id = t2.user_id); From bc9d380e5a88be17cda135c9f649103404204bb2 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 16:46:45 +0200 Subject: [PATCH 117/243] Close: https://github.com/ClickHouse/ClickHouse/issues/54910 --- .../03085_analyzer_alias_column_group_by.reference | 1 + .../0_stateless/03085_analyzer_alias_column_group_by.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/03085_analyzer_alias_column_group_by.reference create mode 100644 tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql diff --git a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.reference b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.reference new file mode 100644 index 00000000000..804f12662b8 --- /dev/null +++ b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.reference @@ -0,0 +1 @@ +String 1 diff --git a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql new file mode 100644 index 00000000000..f4eaa5d9710 --- /dev/null +++ b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql @@ -0,0 +1,4 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/54910 +SELECT toTypeName(stat_standard_id) AS stat_standard_id_1, count(1) AS value +FROM ( SELECT 'string value' AS stat_standard_id ) +GROUP BY stat_standard_id_1 LIMIT 1 From a5107417449477f1dddb0fd76502c56652a4dd4a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 16:49:35 +0200 Subject: [PATCH 118/243] Close: https://github.com/ClickHouse/ClickHouse/issues/57321 --- ...86_analyzer_window_func_part_of_group_by.reference | 2 ++ .../03086_analyzer_window_func_part_of_group_by.sql | 11 +++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.reference create mode 100644 tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql diff --git a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.reference b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.reference new file mode 100644 index 00000000000..c8b2d51ae53 --- /dev/null +++ b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.reference @@ -0,0 +1,2 @@ +1 2 +2 2 diff --git a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql new file mode 100644 index 00000000000..55a60873a5a --- /dev/null +++ b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql @@ -0,0 +1,11 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/57321 +SELECT + ver, + max(ver) OVER () AS ver_max +FROM +( + SELECT 1 AS ver + UNION ALL + SELECT 2 AS ver +) +GROUP BY ver From 346a0ec53b1dd2984ee4fe55785d61cec3a5b11b Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 17:05:48 +0200 Subject: [PATCH 119/243] Close: https://github.com/ClickHouse/ClickHouse/issues/59154 --- .../03087_analyzer_subquery_with_alias.reference | 1 + .../03087_analyzer_subquery_with_alias.sql | 15 +++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03087_analyzer_subquery_with_alias.reference create mode 100644 tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql diff --git a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.reference b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.reference new file mode 100644 index 00000000000..7660873d103 --- /dev/null +++ b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.reference @@ -0,0 +1 @@ +[1] diff --git a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql new file mode 100644 index 00000000000..98aca76fe49 --- /dev/null +++ b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql @@ -0,0 +1,15 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/59154 +SELECT * +FROM +( + WITH + assumeNotNull(( + SELECT 0.9 + )) AS TUNING, + ELEMENT_QUERY AS + ( + SELECT quantiles(TUNING)(1) + ) + SELECT * + FROM ELEMENT_QUERY +); From 39205fe676b97923765a7fe8791f68a7f915de35 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 17:10:10 +0200 Subject: [PATCH 120/243] Close: https://github.com/ClickHouse/ClickHouse/issues/61014 --- ...3088_analyzer_ambiguous_column_multi_call.reference | 0 .../03088_analyzer_ambiguous_column_multi_call.sql | 10 ++++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.reference create mode 100644 tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.reference b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql new file mode 100644 index 00000000000..84afdb295c2 --- /dev/null +++ b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql @@ -0,0 +1,10 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61014 +create database test_03088; + +create table test_03088.a (i int) engine = Log(); + +select + test_03088.a.i +from + test_03088.a, + test_03088.a as x; From ccf0953d8013284815102643cb735d6151352c04 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 3 Apr 2024 17:14:05 +0200 Subject: [PATCH 121/243] Close: https://github.com/ClickHouse/ClickHouse/issues/61950 --- .../03089_analyzer_alias_replacement.reference | 2 ++ .../0_stateless/03089_analyzer_alias_replacement.sql | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03089_analyzer_alias_replacement.reference create mode 100644 tests/queries/0_stateless/03089_analyzer_alias_replacement.sql diff --git a/tests/queries/0_stateless/03089_analyzer_alias_replacement.reference b/tests/queries/0_stateless/03089_analyzer_alias_replacement.reference new file mode 100644 index 00000000000..2f1b638ff54 --- /dev/null +++ b/tests/queries/0_stateless/03089_analyzer_alias_replacement.reference @@ -0,0 +1,2 @@ +1 +4 diff --git a/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql new file mode 100644 index 00000000000..00a3795eab8 --- /dev/null +++ b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql @@ -0,0 +1,8 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61950 + +with dummy + 1 as dummy select dummy from system.one; + +WITH dummy + 3 AS dummy +SELECT dummy + 1 AS y +FROM system.one +SETTINGS enable_global_with_statement = 1; From 925148f928c3d6165a5f8b217ca8a597b0962ab1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Apr 2024 20:32:48 +0200 Subject: [PATCH 122/243] Add logging --- src/Interpreters/Cache/FileCache.cpp | 1 - src/Interpreters/Cache/FileCacheFactory.cpp | 10 +++++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 75e199c544b..90671629e64 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1379,7 +1379,6 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, } } - if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index e05e1935d95..e1f14406058 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -142,8 +142,10 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig caches_by_name_copy = caches_by_name; } + auto * log = &Poco::Logger::get("FileCacheFactory"); + std::unordered_set checked_paths; - for (const auto & [_, cache_info] : caches_by_name_copy) + for (const auto & [cache_name, cache_info] : caches_by_name_copy) { if (cache_info->config_path.empty() || checked_paths.contains(cache_info->config_path)) continue; @@ -155,7 +157,12 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig FileCacheSettings old_settings = cache_info->getSettings(); if (old_settings == new_settings) + { + LOG_TRACE(log, "No settings changes for cache: {}", cache_name); continue; + } + + LOG_TRACE(log, "Will apply settings changes for cache: {}", cache_name); try { @@ -166,6 +173,7 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig /// Settings changes could be partially applied in case of exception, /// make sure cache_info->settings show correct state of applied settings. cache_info->setSettings(old_settings); + tryLogCurrentException(__PRETTY_FUNCTION__); throw; } From be55c3533c9b3b4293bbb052dc8ed585af7c20b7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Apr 2024 15:48:30 -0300 Subject: [PATCH 123/243] default to path style --- src/IO/S3/URI.cpp | 15 +++------------ src/IO/tests/gtest_s3_uri.cpp | 6 +++--- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 8e364337db5..02c77518ab6 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -108,19 +108,10 @@ URI::URI(const std::string & uri_) String name; String endpoint_authority_from_uri; - if (re2::RE2::FullMatch(uri.getAuthority(), aws_private_link_style_pattern)) - { - if (!re2::RE2::PartialMatch(uri.getPath(), path_style_pattern, &bucket, &key)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not parse bucket and key from uri {}", uri.toString()); - } + bool is_using_aws_private_link_interface = re2::RE2::FullMatch(uri.getAuthority(), aws_private_link_style_pattern); - // Default to virtual hosted style - is_virtual_hosted_style = true; - endpoint = uri.getScheme() + "://" + uri.getAuthority(); - validateBucket(bucket, uri); - } - else if (re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) + if (!is_using_aws_private_link_interface + && re2::RE2::FullMatch(uri.getAuthority(), virtual_hosted_style_pattern, &bucket, &name, &endpoint_authority_from_uri)) { is_virtual_hosted_style = true; endpoint = uri.getScheme() + "://" + name + endpoint_authority_from_uri; diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index b3ceb875362..0a164b0dd61 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -79,21 +79,21 @@ const TestCase TestCases[] = { "root", "nested/file.txt", "", - true}, + false}, // Test with a file with no extension {S3::URI("https://bucket.vpce-03b2c987f1bd55c5f-j3b4vg7w.s3.ap-southeast-2.vpce.amazonaws.com/some_bucket/document"), "https://bucket.vpce-03b2c987f1bd55c5f-j3b4vg7w.s3.ap-southeast-2.vpce.amazonaws.com", "some_bucket", "document", "", - true}, + false}, // Test with a deeply nested file path {S3::URI("https://bucket.vpce-0242cd56f1bd55c5f-l5b7vg8x.s3.sa-east-1.vpce.amazonaws.com/some_bucket/b/c/d/e/f/g/h/i/j/data.json"), "https://bucket.vpce-0242cd56f1bd55c5f-l5b7vg8x.s3.sa-east-1.vpce.amazonaws.com", "some_bucket", "b/c/d/e/f/g/h/i/j/data.json", "", - true}, + false}, }; class S3UriTest : public testing::TestWithParam From c6aed8b7938ee78030d5a23740a3bc191e85ca7f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Apr 2024 19:19:54 -0300 Subject: [PATCH 124/243] add optional port to regex --- src/IO/S3/URI.cpp | 2 +- src/IO/S3/URI.h | 2 +- src/IO/tests/gtest_s3_uri.cpp | 14 ++++++++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index 02c77518ab6..0d498c1d743 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -40,7 +40,7 @@ URI::URI(const std::string & uri_) /// Case when AWS Private Link Interface is being used /// E.g. (bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com/bucket-name/key) /// https://docs.aws.amazon.com/AmazonS3/latest/userguide/privatelink-interface-endpoints.html - static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce.amazonaws.com)"); + static const RE2 aws_private_link_style_pattern(R"(bucket\.vpce\-([a-z0-9\-.]+)\.vpce.amazonaws.com(:\d{1,5})?)"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.region.amazonaws.com/bucket-name/key) diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index 79f3da3fbbb..06b7d03aa8c 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -17,7 +17,7 @@ namespace DB::S3 * The following patterns are allowed: * s3://bucket/key * http(s)://endpoint/bucket/key - * TODO specify aws private link + * http(s)://bucket..s3..vpce.amazonaws.com<:port_number>/bucket_name/key */ struct URI { diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 0a164b0dd61..9c1f7bd5219 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -94,6 +94,20 @@ const TestCase TestCases[] = { "b/c/d/e/f/g/h/i/j/data.json", "", false}, + // Zonal + {S3::URI("https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com/root/nested/file.txt"), + "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com", + "root", + "nested/file.txt", + "", + false}, + // Non standard port + {S3::URI("https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com:65535/root/nested/file.txt"), + "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com:65535", + "root", + "nested/file.txt", + "", + false}, }; class S3UriTest : public testing::TestWithParam From 5ce8ab2bd986d0c34abd7d6cf3cf028cad369cbb Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 11:32:32 +0200 Subject: [PATCH 125/243] More logging --- src/Interpreters/Cache/FileCache.cpp | 10 ++++ src/Interpreters/Cache/FileCacheFactory.cpp | 5 +- src/Interpreters/Cache/FileCacheSettings.cpp | 58 ++++++++++++++++++++ src/Interpreters/Cache/FileCacheSettings.h | 3 + 4 files changed, 75 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 90671629e64..8ab46e66a76 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1379,6 +1379,16 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, } } + { + auto cache_lock = lockCache(); + LOG_TRACE(log, "new max size: {}, old max size: {}, " + "new elements count: {}, old_elements_count: {}, " + "current max size: {}, current max elements: {}", + new_settings.max_size, actual_settings.max_size, + new_settings.max_elements, actual_settings.max_elements, + main_priority->getSizeLimit(cache_lock), main_priority->getElementsLimit(cache_lock)); + } + if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index e1f14406058..a046c4c2b6a 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -162,7 +162,10 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig continue; } - LOG_TRACE(log, "Will apply settings changes for cache: {}", cache_name); + LOG_TRACE(log, "Will apply settings changes for cache {}. " + "Settings changes: {} (new settings: {}, old_settings: {})", + cache_name, fmt::join(new_settings.getSettingsDiff(old_settings), ", "), + new_settings.toString(), old_settings.toString()); try { diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 8a48a2de68f..fef17d0df28 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -98,4 +99,61 @@ void FileCacheSettings::loadFromCollection(const NamedCollection & collection) loadImpl(std::move(collection_has), std::move(collection_get_uint), std::move(collection_get_string), std::move(collection_get_double)); } +std::string FileCacheSettings::toString() const +{ + WriteBufferFromOwnString res; + res << "base_path: " << base_path << "\n"; + res << "max_size: " << max_size << "\n"; + res << "max_elements: " << max_elements << "\n"; + res << "max_file_segment_size: " << max_file_segment_size << "\n"; + res << "cache_on_write_operations: " << cache_on_write_operations << "\n"; + res << "cache_hits_threshold: " << cache_hits_threshold << "\n"; + res << "enable_filesystem_query_cache_limit: " << enable_filesystem_query_cache_limit << "\n"; + res << "bypass_cache_threshold: " << bypass_cache_threshold << "\n"; + res << "boundary_alignment: " << boundary_alignment << "\n"; + res << "background_download_threads: " << background_download_threads << "\n"; + res << "background_download_queue_size_limit: " << background_download_queue_size_limit << "\n"; + res << "load_metadata_threads: " << load_metadata_threads << "\n"; + res << "write_cache_per_user_id_directory: " << write_cache_per_user_id_directory << "\n"; + res << "cache_policy: " << cache_policy << "\n"; + res << "slru_size_ratio: " << slru_size_ratio << "\n"; + return res.str(); +} + +std::vector FileCacheSettings::getSettingsDiff(const FileCacheSettings & other) const +{ + std::vector res; + if (base_path != other.base_path) + res.push_back("base_path"); + if (max_size != other.max_size) + res.push_back("max_size"); + if (max_elements != other.max_elements) + res.push_back("max_elements"); + if (max_file_segment_size != other.max_file_segment_size) + res.push_back("max_file_segment_size"); + if (cache_on_write_operations != other.cache_on_write_operations) + res.push_back("cache_on_write_operations"); + if (cache_hits_threshold != other.cache_hits_threshold) + res.push_back("cache_hits_threshold"); + if (enable_filesystem_query_cache_limit != other.enable_filesystem_query_cache_limit) + res.push_back("enable_filesystem_query_cache_limit"); + if (bypass_cache_threshold != other.bypass_cache_threshold) + res.push_back("bypass_cache_threshold"); + if (boundary_alignment != other.boundary_alignment) + res.push_back("boundary_alignment"); + if (background_download_threads != other.background_download_threads) + res.push_back("background_download_threads"); + if (background_download_queue_size_limit != other.background_download_queue_size_limit) + res.push_back("background_download_queue_size_limit"); + if (load_metadata_threads != other.load_metadata_threads) + res.push_back("load_metadata_threads"); + if (write_cache_per_user_id_directory != other.write_cache_per_user_id_directory) + res.push_back("write_cache_per_user_directory"); + if (cache_policy != other.cache_policy) + res.push_back("cache_policy"); + if (slru_size_ratio != other.slru_size_ratio) + res.push_back("slru_size_ratio"); + return res; +} + } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index 14770b3f005..7dab14ac896 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -41,6 +41,9 @@ struct FileCacheSettings void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); void loadFromCollection(const NamedCollection & collection); + std::string toString() const; + std::vector getSettingsDiff(const FileCacheSettings & other) const; + bool operator ==(const FileCacheSettings &) const = default; private: From e516bef844b397a8f9c041b914e85939b225446c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Apr 2024 09:52:58 +0000 Subject: [PATCH 126/243] Annalyzer: limit maximal size of column in constant folding --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index f5474ddb662..8f6c461d92d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6083,7 +6083,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi * Example: SELECT toTypeName(sum(number)) FROM numbers(10); */ if (column && isColumnConst(*column) && !typeid_cast(column.get())->getDataColumn().isDummy() && - (!hasAggregateFunctionNodes(node) && !hasFunctionNode(node, "arrayJoin"))) + !hasAggregateFunctionNodes(node) && !hasFunctionNode(node, "arrayJoin") && + /// Sanity check: do not convert large columns to constants + column->byteSize() < 1_MiB) { /// Replace function node with result constant node Field column_constant_value; From 58c79af80b38c1c3b3f64718690edafdad2156af Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Apr 2024 12:22:11 +0200 Subject: [PATCH 127/243] Simpler --- src/Common/QueryProfiler.cpp | 9 ++++++++ src/Common/ThreadStatus.cpp | 23 +++++++++++++++++++ .../Standalone/ThreadStatusExt.cpp | 4 ---- src/Interpreters/ThreadStatusExt.cpp | 21 ----------------- .../test_trace_collector_serverwide/test.py | 3 +++ 5 files changed, 35 insertions(+), 25 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index f985ec95e88..b616b795405 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -264,7 +264,16 @@ QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_t template void QueryProfilerBase::setPeriod(UInt32 period_) { +#if defined(SANITIZER) + UNUSED(period); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); +#elif defined(__APPLE__) + UNUSED(period); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler cannot work on OSX"); +#else timer.set(period_); +#endif + } template diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index cf50d305e95..a3a7257b472 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -124,6 +124,29 @@ ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) #endif } +void ThreadStatus::initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period) +{ +#if !defined(SANITIZER) && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) && !defined(__APPLE__) + try + { + if (global_profiler_real_time_period > 0) + query_profiler_real = std::make_unique(thread_id, + /* period= */ static_cast(global_profiler_real_time_period)); + + if (global_profiler_cpu_time_period > 0) + query_profiler_cpu = std::make_unique(thread_id, + /* period= */ static_cast(global_profiler_cpu_time_period)); + } + catch (...) + { + tryLogCurrentException("ThreadStatus", "Cannot initialize GlobalProfiler"); + } +#else + UNUSED(global_profiler_real_time_period); + UNUSED(global_profiler_cpu_time_period); +#endif +} + ThreadGroupPtr ThreadStatus::getThreadGroup() const { chassert(current_thread == this); diff --git a/src/Coordination/Standalone/ThreadStatusExt.cpp b/src/Coordination/Standalone/ThreadStatusExt.cpp index 2b89e2f024d..97f7287be8c 100644 --- a/src/Coordination/Standalone/ThreadStatusExt.cpp +++ b/src/Coordination/Standalone/ThreadStatusExt.cpp @@ -11,8 +11,4 @@ void CurrentThread::attachToGroup(const ThreadGroupPtr &) { } -void ThreadStatus::initGlobalProfiler(UInt64, UInt64) -{ -} - } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 4b9bd069bc6..2b8e8bef6d4 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -458,27 +458,6 @@ void ThreadStatus::resetPerformanceCountersLastUsage() taskstats->reset(); } - -void ThreadStatus::initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period) -{ - - try - { - if (global_profiler_real_time_period > 0) - query_profiler_real = std::make_unique(thread_id, - /* period= */ static_cast(global_profiler_real_time_period)); - - if (global_profiler_cpu_time_period > 0) - query_profiler_cpu = std::make_unique(thread_id, - /* period= */ static_cast(global_profiler_cpu_time_period)); - } - catch (...) - { - tryLogCurrentException("ThreadStatus", "Cannot initialize GlobalProfiler"); - } - -} - void ThreadStatus::initQueryProfiler() { if (internal_thread) diff --git a/tests/integration/test_trace_collector_serverwide/test.py b/tests/integration/test_trace_collector_serverwide/test.py index 88d235642b9..9bd107ac365 100644 --- a/tests/integration/test_trace_collector_serverwide/test.py +++ b/tests/integration/test_trace_collector_serverwide/test.py @@ -22,6 +22,9 @@ def start_cluster(): def test_global_thread_profiler(start_cluster): + if node1.is_built_with_sanitizer(): + return + node1.query( "CREATE TABLE t (key UInt32, value String) Engine = MergeTree() ORDER BY key" ) From 33aee0f599867da294cfc5327cc4ab932e761066 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 4 Apr 2024 13:00:51 +0200 Subject: [PATCH 128/243] Analyzer: Fix name resolution from parent scopes --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 8 +++++- ...alyzer_resolve_from_parent_scope.reference | 1 + ...033_analyzer_resolve_from_parent_scope.sql | 27 +++++++++++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.reference create mode 100644 tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index f5474ddb662..91832f6060d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3993,9 +3993,15 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifierInParentScopes(const } else if (resolved_identifier->as()) { - lookup_result.resolved_identifier = resolved_identifier; return lookup_result; } + else if (auto * resolved_function = resolved_identifier->as()) + { + /// Special case: scalar subquery was executed and replaced by __getScalar function. + /// Handle it as a constant. + if (resolved_function->getFunctionName() == "__getScalar") + return lookup_result; + } throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Resolve identifier '{}' from parent scope only supported for constants and CTE. Actual {} node type {}. In scope {}", diff --git a/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.reference b/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.reference new file mode 100644 index 00000000000..f599e28b8ab --- /dev/null +++ b/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.reference @@ -0,0 +1 @@ +10 diff --git a/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.sql b/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.sql new file mode 100644 index 00000000000..22f103c9bd5 --- /dev/null +++ b/tests/queries/0_stateless/03033_analyzer_resolve_from_parent_scope.sql @@ -0,0 +1,27 @@ +CREATE TABLE vecs_Float32 (v Array(Float32)) ENGINE=Memory; +INSERT INTO vecs_Float32 +SELECT v FROM ( + SELECT + number AS n, + [ + rand(n*10), rand(n*10+1), rand(n*10+2), rand(n*10+3), rand(n*10+4), rand(n*10+5), rand(n*10+6), rand(n*10+7), rand(n*10+8), rand(n*10+9), + rand(n*10+10), rand(n*10+11), rand(n*10+12), rand(n*10+13), rand(n*10+14), rand(n*10+15), rand(n*10+16), rand(n*10+17), rand(n*10+18), rand(n*10+19), + rand(n*10+20), rand(n*10+21), rand(n*10+22), rand(n*10+23), rand(n*10+24), rand(n*10+25), rand(n*10+26), rand(n*10+27), rand(n*10+28), rand(n*10+29), + rand(n*10+30), rand(n*10+31), rand(n*10+32), rand(n*10+33), rand(n*10+34), rand(n*10+35), rand(n*10+36), rand(n*10+37), rand(n*10+38), rand(n*10+39), + rand(n*10+40), rand(n*10+41), rand(n*10+42), rand(n*10+43), rand(n*10+44), rand(n*10+45), rand(n*10+46), rand(n*10+47), rand(n*10+48), rand(n*10+49), + rand(n*10+50), rand(n*10+51), rand(n*10+52), rand(n*10+53), rand(n*10+54), rand(n*10+55), rand(n*10+56), rand(n*10+57), rand(n*10+58), rand(n*10+59), + rand(n*10+60), rand(n*10+61), rand(n*10+62), rand(n*10+63), rand(n*10+64), rand(n*10+65), rand(n*10+66), rand(n*10+67), rand(n*10+68), rand(n*10+69), + rand(n*10+70), rand(n*10+71), rand(n*10+72), rand(n*10+73), rand(n*10+74), rand(n*10+75), rand(n*10+76), rand(n*10+77), rand(n*10+78), rand(n*10+79), + rand(n*10+80), rand(n*10+81), rand(n*10+82), rand(n*10+83), rand(n*10+84), rand(n*10+85), rand(n*10+86), rand(n*10+87), rand(n*10+88), rand(n*10+89), + rand(n*10+90), rand(n*10+91), rand(n*10+92), rand(n*10+93), rand(n*10+94), rand(n*10+95), rand(n*10+96), rand(n*10+97), rand(n*10+98), rand(n*10+99), + rand(n*10+100), rand(n*10+101), rand(n*10+102), rand(n*10+103), rand(n*10+104), rand(n*10+105), rand(n*10+106), rand(n*10+107), rand(n*10+108), rand(n*10+109), + rand(n*10+110), rand(n*10+111), rand(n*10+112), rand(n*10+113), rand(n*10+114), rand(n*10+115), rand(n*10+116), rand(n*10+117), rand(n*10+118), rand(n*10+119), + rand(n*10+120), rand(n*10+121), rand(n*10+122), rand(n*10+123), rand(n*10+124), rand(n*10+125), rand(n*10+126), rand(n*10+127), rand(n*10+128), rand(n*10+129), + rand(n*10+130), rand(n*10+131), rand(n*10+132), rand(n*10+133), rand(n*10+134), rand(n*10+135), rand(n*10+136), rand(n*10+137), rand(n*10+138), rand(n*10+139), + rand(n*10+140), rand(n*10+141), rand(n*10+142), rand(n*10+143), rand(n*10+144), rand(n*10+145), rand(n*10+146), rand(n*10+147), rand(n*10+148), rand(n*10+149) + ] AS v + FROM system.numbers + LIMIT 10 +); + +WITH (SELECT v FROM vecs_Float32 limit 1) AS a SELECT count(dp) FROM (SELECT dotProduct(a, v) AS dp FROM vecs_Float32); From 62f9be052c3a1046fb492a313ad77df80e532009 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 13:41:30 +0200 Subject: [PATCH 129/243] Fix test --- src/Interpreters/Cache/FileCache.cpp | 10 ------- src/Interpreters/Cache/FileCacheFactory.cpp | 8 +++--- src/Interpreters/Cache/FileCacheSettings.cpp | 30 ++++++++++---------- tests/config/config.d/storage_conf.xml | 2 +- 4 files changed, 20 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8ab46e66a76..90671629e64 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1379,16 +1379,6 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, } } - { - auto cache_lock = lockCache(); - LOG_TRACE(log, "new max size: {}, old max size: {}, " - "new elements count: {}, old_elements_count: {}, " - "current max size: {}, current max elements: {}", - new_settings.max_size, actual_settings.max_size, - new_settings.max_elements, actual_settings.max_elements, - main_priority->getSizeLimit(cache_lock), main_priority->getElementsLimit(cache_lock)); - } - if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index a046c4c2b6a..747b31bff64 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -162,10 +162,10 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig continue; } - LOG_TRACE(log, "Will apply settings changes for cache {}. " - "Settings changes: {} (new settings: {}, old_settings: {})", - cache_name, fmt::join(new_settings.getSettingsDiff(old_settings), ", "), - new_settings.toString(), old_settings.toString()); + // LOG_TRACE(log, "Will apply settings changes for cache {}. " + // "Settings changes: {} (new settings: {}, old_settings: {})", + // cache_name, fmt::join(new_settings.getSettingsDiff(old_settings), ", "), + // new_settings.toString(), old_settings.toString()); try { diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index fef17d0df28..ff5f48503b7 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -102,21 +102,21 @@ void FileCacheSettings::loadFromCollection(const NamedCollection & collection) std::string FileCacheSettings::toString() const { WriteBufferFromOwnString res; - res << "base_path: " << base_path << "\n"; - res << "max_size: " << max_size << "\n"; - res << "max_elements: " << max_elements << "\n"; - res << "max_file_segment_size: " << max_file_segment_size << "\n"; - res << "cache_on_write_operations: " << cache_on_write_operations << "\n"; - res << "cache_hits_threshold: " << cache_hits_threshold << "\n"; - res << "enable_filesystem_query_cache_limit: " << enable_filesystem_query_cache_limit << "\n"; - res << "bypass_cache_threshold: " << bypass_cache_threshold << "\n"; - res << "boundary_alignment: " << boundary_alignment << "\n"; - res << "background_download_threads: " << background_download_threads << "\n"; - res << "background_download_queue_size_limit: " << background_download_queue_size_limit << "\n"; - res << "load_metadata_threads: " << load_metadata_threads << "\n"; - res << "write_cache_per_user_id_directory: " << write_cache_per_user_id_directory << "\n"; - res << "cache_policy: " << cache_policy << "\n"; - res << "slru_size_ratio: " << slru_size_ratio << "\n"; + res << "base_path: " << base_path << ", "; + res << "max_size: " << max_size << ", "; + res << "max_elements: " << max_elements << ", "; + res << "max_file_segment_size: " << max_file_segment_size << ", "; + res << "cache_on_write_operations: " << cache_on_write_operations << ", "; + res << "cache_hits_threshold: " << cache_hits_threshold << ", "; + res << "enable_filesystem_query_cache_limit: " << enable_filesystem_query_cache_limit << ", "; + res << "bypass_cache_threshold: " << bypass_cache_threshold << ", "; + res << "boundary_alignment: " << boundary_alignment << ", "; + res << "background_download_threads: " << background_download_threads << ", "; + res << "background_download_queue_size_limit: " << background_download_queue_size_limit << ", "; + res << "load_metadata_threads: " << load_metadata_threads << ", "; + res << "write_cache_per_user_id_directory: " << write_cache_per_user_id_directory << ", "; + res << "cache_policy: " << cache_policy << ", "; + res << "slru_size_ratio: " << slru_size_ratio << ", "; return res.str(); } diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 00d8cb3aea5..d40854247cd 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,7 @@ cache s3_disk s3_cache/ - 64Mi + 104857600 1 100 LRU From b7c0501ac292f64992a06a4b880520174cf3cf00 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Apr 2024 09:26:37 -0300 Subject: [PATCH 130/243] fix ut --- src/IO/tests/gtest_s3_uri.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 9c1f7bd5219..175550acccc 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -103,7 +103,7 @@ const TestCase TestCases[] = { false}, // Non standard port {S3::URI("https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com:65535/root/nested/file.txt"), - "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com:65535", + "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com:65535", "root", "nested/file.txt", "", From 368dcf03ecc9426497aaaf739e9efa5fb43e1f63 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Apr 2024 15:37:38 +0200 Subject: [PATCH 131/243] Run new analyzer tests with enabled allow_experimental_analyzer --- tests/queries/0_stateless/03033_cte_numbers_memory.sql | 1 + tests/queries/0_stateless/03033_with_fill_interpolate.sql | 1 + tests/queries/0_stateless/03034_normalized_ast.sql | 1 + tests/queries/0_stateless/03035_alias_column_bug_distributed.sql | 1 + tests/queries/0_stateless/03036_with_numbers.sql | 1 + tests/queries/0_stateless/03037_union_view.sql | 1 + tests/queries/0_stateless/03038_ambiguous_column.sql | 1 + .../0_stateless/03039_unknown_identifier_window_function.sql | 1 + tests/queries/0_stateless/03040_alias_column_join.sql | 1 + tests/queries/0_stateless/03040_array_sum_and_join.sql | 1 + tests/queries/0_stateless/03041_analyzer_gigachad_join.sql | 1 + tests/queries/0_stateless/03041_select_with_query_result.sql | 1 + tests/queries/0_stateless/03042_analyzer_alias_join.sql | 1 + tests/queries/0_stateless/03042_not_found_column_c1.sql | 1 + .../queries/0_stateless/03043_group_array_result_is_expected.sql | 1 + tests/queries/0_stateless/03044_analyzer_alias_join.sql | 1 + .../0_stateless/03044_array_join_columns_in_nested_table.sql | 1 + tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql | 1 + .../0_stateless/03045_unknown_identifier_alias_substitution.sql | 1 + tests/queries/0_stateless/03046_column_in_block_array_join.sql | 1 + tests/queries/0_stateless/03047_analyzer_alias_join.sql | 1 + .../0_stateless/03047_group_by_field_identified_aggregation.sql | 1 + .../queries/0_stateless/03048_not_found_column_xxx_in_block.sql | 1 + tests/queries/0_stateless/03049_analyzer_group_by_alias.sql | 1 + .../0_stateless/03049_unknown_identifier_materialized_column.sql | 1 + tests/queries/0_stateless/03050_select_one_one_one.sql | 1 + tests/queries/0_stateless/03051_many_ctes.sql | 1 + tests/queries/0_stateless/03052_query_hash_includes_aliases.sql | 1 + tests/queries/0_stateless/03053_analyzer_join_alias.sql | 1 + tests/queries/0_stateless/03054_analyzer_join_alias.sql | 1 + .../queries/0_stateless/03055_analyzer_subquery_group_array.sql | 1 + .../queries/0_stateless/03056_analyzer_double_subquery_alias.sql | 1 + tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql | 1 + tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql | 1 + .../0_stateless/03059_analyzer_join_engine_missing_column.sql | 1 + tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql | 1 + .../0_stateless/03061_analyzer_alias_as_right_key_in_join.sql | 1 + .../0_stateless/03062_analyzer_join_engine_missing_column.sql | 1 + .../03063_analyzer_multi_join_wrong_table_specifier.sql | 1 + tests/queries/0_stateless/03064_analyzer_named_subqueries.sql | 1 + .../0_stateless/03065_analyzer_cross_join_and_array_join.sql | 1 + .../queries/0_stateless/03066_analyzer_global_with_statement.sql | 1 + tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql | 1 + tests/queries/0_stateless/03068_analyzer_distributed_join.sql | 1 + .../0_stateless/03069_analyzer_with_alias_in_array_join.sql | 1 + .../queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql | 1 + .../03071_analyzer_array_join_forbid_non_existing_columns.sql | 1 + .../0_stateless/03072_analyzer_missing_columns_from_subquery.sql | 1 + .../queries/0_stateless/03073_analyzer_alias_as_column_name.sql | 1 + .../queries/0_stateless/03074_analyzer_alias_column_in_view.sql | 1 + tests/queries/0_stateless/03075_analyzer_subquery_alias.sql | 1 + .../queries/0_stateless/03076_analyzer_multiple_joins_alias.sql | 1 + .../0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql | 1 + .../0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql | 1 + .../03079_analyzer_numeric_literals_as_column_names.sql | 1 + ...080_analyzer_prefer_column_name_to_alias__virtual_columns.sql | 1 + tests/queries/0_stateless/03080_incorrect_join_with_merge.sql | 1 + tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql | 1 + .../0_stateless/03082_analyzer_left_join_correct_column.sql | 1 + tests/queries/0_stateless/03084_analyzer_join_column_alias.sql | 1 + .../queries/0_stateless/03085_analyzer_alias_column_group_by.sql | 1 + .../0_stateless/03086_analyzer_window_func_part_of_group_by.sql | 1 + tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql | 1 + .../0_stateless/03088_analyzer_ambiguous_column_multi_call.sql | 1 + tests/queries/0_stateless/03089_analyzer_alias_replacement.sql | 1 + 65 files changed, 65 insertions(+) diff --git a/tests/queries/0_stateless/03033_cte_numbers_memory.sql b/tests/queries/0_stateless/03033_cte_numbers_memory.sql index 0e3ee9abd65..66b11cbfaa5 100644 --- a/tests/queries/0_stateless/03033_cte_numbers_memory.sql +++ b/tests/queries/0_stateless/03033_cte_numbers_memory.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61238 +SET allow_experimental_analyzer=1; WITH (SELECT number FROM system.numbers LIMIT 1) as w1, diff --git a/tests/queries/0_stateless/03033_with_fill_interpolate.sql b/tests/queries/0_stateless/03033_with_fill_interpolate.sql index 816633af757..0ec0050a922 100644 --- a/tests/queries/0_stateless/03033_with_fill_interpolate.sql +++ b/tests/queries/0_stateless/03033_with_fill_interpolate.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55794 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS 03033_example_table; CREATE TABLE 03033_example_table diff --git a/tests/queries/0_stateless/03034_normalized_ast.sql b/tests/queries/0_stateless/03034_normalized_ast.sql index dd109eb5113..385af4e2c34 100644 --- a/tests/queries/0_stateless/03034_normalized_ast.sql +++ b/tests/queries/0_stateless/03034_normalized_ast.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/49472 +SET allow_experimental_analyzer=1; SELECT concat(database, table) AS name, count() diff --git a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql index 3a7b4890bf0..74463743b01 100644 --- a/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql +++ b/tests/queries/0_stateless/03035_alias_column_bug_distributed.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44414 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS alias_bug; DROP TABLE IF EXISTS alias_bug_dist; CREATE TABLE alias_bug diff --git a/tests/queries/0_stateless/03036_with_numbers.sql b/tests/queries/0_stateless/03036_with_numbers.sql index 5e08bb6e065..3463ce826e2 100644 --- a/tests/queries/0_stateless/03036_with_numbers.sql +++ b/tests/queries/0_stateless/03036_with_numbers.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/13843 +SET allow_experimental_analyzer=1; WITH 10 AS n SELECT * FROM numbers(n); diff --git a/tests/queries/0_stateless/03037_union_view.sql b/tests/queries/0_stateless/03037_union_view.sql index fb8aa7df954..3ea81b829ba 100644 --- a/tests/queries/0_stateless/03037_union_view.sql +++ b/tests/queries/0_stateless/03037_union_view.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/55803 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS broken_table; DROP TABLE IF EXISTS broken_view; diff --git a/tests/queries/0_stateless/03038_ambiguous_column.sql b/tests/queries/0_stateless/03038_ambiguous_column.sql index 69c8e52d734..9df3cd9bc9b 100644 --- a/tests/queries/0_stateless/03038_ambiguous_column.sql +++ b/tests/queries/0_stateless/03038_ambiguous_column.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/48308 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS 03038_table; CREATE TABLE 03038_table diff --git a/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql index ca3bb521eba..640d217d2f9 100644 --- a/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql +++ b/tests/queries/0_stateless/03039_unknown_identifier_window_function.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/45535 +SET allow_experimental_analyzer=1; SELECT *, diff --git a/tests/queries/0_stateless/03040_alias_column_join.sql b/tests/queries/0_stateless/03040_alias_column_join.sql index f4ea2e5914d..54f579c0feb 100644 --- a/tests/queries/0_stateless/03040_alias_column_join.sql +++ b/tests/queries/0_stateless/03040_alias_column_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44365 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS 03040_test; CREATE TABLE 03040_test diff --git a/tests/queries/0_stateless/03040_array_sum_and_join.sql b/tests/queries/0_stateless/03040_array_sum_and_join.sql index 0084f0e4c7b..9aeddc9f765 100644 --- a/tests/queries/0_stateless/03040_array_sum_and_join.sql +++ b/tests/queries/0_stateless/03040_array_sum_and_join.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; select t.1 as cnt, t.2 as name, diff --git a/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql index 462e63b121b..7906e65f8b8 100644 --- a/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql +++ b/tests/queries/0_stateless/03041_analyzer_gigachad_join.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; CREATE TABLE IF NOT EXISTS first engine = MergeTree PARTITION BY (inn, toYYYYMM(received)) ORDER BY (inn, sessionId) AS SELECT now() AS received, '123456789' AS inn, '42' AS sessionId; diff --git a/tests/queries/0_stateless/03041_select_with_query_result.sql b/tests/queries/0_stateless/03041_select_with_query_result.sql index 3edf51d635e..061223b43e1 100644 --- a/tests/queries/0_stateless/03041_select_with_query_result.sql +++ b/tests/queries/0_stateless/03041_select_with_query_result.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/44153 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS parent; DROP TABLE IF EXISTS join_table_1; DROP TABLE IF EXISTS join_table_2; diff --git a/tests/queries/0_stateless/03042_analyzer_alias_join.sql b/tests/queries/0_stateless/03042_analyzer_alias_join.sql index f3341fd314a..dac3b6a4983 100644 --- a/tests/queries/0_stateless/03042_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03042_analyzer_alias_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/14978 +SET allow_experimental_analyzer=1; CREATE TABLE test1(id UInt64, t1value UInt64) ENGINE=MergeTree ORDER BY tuple(); CREATE TABLE test2(id UInt64, t2value String) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/03042_not_found_column_c1.sql b/tests/queries/0_stateless/03042_not_found_column_c1.sql index 8ce7dcd9d4f..b4dce2af489 100644 --- a/tests/queries/0_stateless/03042_not_found_column_c1.sql +++ b/tests/queries/0_stateless/03042_not_found_column_c1.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/42399 +SET allow_experimental_analyzer=1; CREATE TABLE IF NOT EXISTS t0 (c0 Int32) ENGINE = Memory() ; CREATE TABLE t1 (c0 Int32, c1 Int32, c2 Int32) ENGINE = Memory() ; diff --git a/tests/queries/0_stateless/03043_group_array_result_is_expected.sql b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql index df77ca66647..5311927ae3c 100644 --- a/tests/queries/0_stateless/03043_group_array_result_is_expected.sql +++ b/tests/queries/0_stateless/03043_group_array_result_is_expected.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/27115 +SET allow_experimental_analyzer=1; drop table if exists fill_ex; create table fill_ex ( diff --git a/tests/queries/0_stateless/03044_analyzer_alias_join.sql b/tests/queries/0_stateless/03044_analyzer_alias_join.sql index 5202b57a7b1..3ab8edb005f 100644 --- a/tests/queries/0_stateless/03044_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03044_analyzer_alias_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/17319 +SET allow_experimental_analyzer=1; CREATE TEMPORARY TABLE hits (date Date, data Float64) engine=Memory(); SELECT diff --git a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql index f3ec80b8a94..0cf05763202 100644 --- a/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql +++ b/tests/queries/0_stateless/03044_array_join_columns_in_nested_table.sql @@ -1,2 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11813 +SET allow_experimental_analyzer=1; select 1 from (select 1 x) l join (select 1 y, [1] a) r on l.x = r.y array join r.a; diff --git a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql index a0546f57736..ee8756b9460 100644 --- a/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql +++ b/tests/queries/0_stateless/03045_analyzer_alias_join_with_if.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/13210 +SET allow_experimental_analyzer=1; CREATE TABLE test_a_table ( name String, a_col String diff --git a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql index cadcbdc0ce5..d97dfc880b3 100644 --- a/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql +++ b/tests/queries/0_stateless/03045_unknown_identifier_alias_substitution.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23053 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS repl_tbl; CREATE TEMPORARY TABLE repl_tbl diff --git a/tests/queries/0_stateless/03046_column_in_block_array_join.sql b/tests/queries/0_stateless/03046_column_in_block_array_join.sql index 9a2bb19d81e..c6b4613af3f 100644 --- a/tests/queries/0_stateless/03046_column_in_block_array_join.sql +++ b/tests/queries/0_stateless/03046_column_in_block_array_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/37729 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS nested_test; DROP TABLE IF EXISTS join_test; diff --git a/tests/queries/0_stateless/03047_analyzer_alias_join.sql b/tests/queries/0_stateless/03047_analyzer_alias_join.sql index ef8c067bb72..7d44c92b6f1 100644 --- a/tests/queries/0_stateless/03047_analyzer_alias_join.sql +++ b/tests/queries/0_stateless/03047_analyzer_alias_join.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; SELECT 1 AS value, * diff --git a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql index e1363ea4dda..cfaf1df44bd 100644 --- a/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql +++ b/tests/queries/0_stateless/03047_group_by_field_identified_aggregation.sql @@ -1,3 +1,4 @@ -- https://github.com/ClickHouse/ClickHouse/issues/32639 +SET allow_experimental_analyzer=1; SELECT 0 AND id ? 1 : 2 AS a, sum(id) FROM (SELECT 1 AS id) GROUP BY a; diff --git a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql index 25f88050eb1..42fd581e142 100644 --- a/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql +++ b/tests/queries/0_stateless/03048_not_found_column_xxx_in_block.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/41964 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS ab_12_aaa; DROP TABLE IF EXISTS ab_12_bbb; diff --git a/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql index 67df48e0cad..d25babe6788 100644 --- a/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql +++ b/tests/queries/0_stateless/03049_analyzer_group_by_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/7520 +SET allow_experimental_analyzer=1; CREATE TABLE test (`a` UInt32, `b` UInt32) ENGINE = Memory; INSERT INTO test VALUES (1,2), (1,3), (2,4); diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql index 2f12799addb..a1c858a329c 100644 --- a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql +++ b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54317 +SET allow_experimental_analyzer=1; DROP DATABASE IF EXISTS 03049_database; DROP TABLE IF EXISTS 03049_database.l; DROP TABLE IF EXISTS 03049_database.r; diff --git a/tests/queries/0_stateless/03050_select_one_one_one.sql b/tests/queries/0_stateless/03050_select_one_one_one.sql index eee973fe936..28a55e0c471 100644 --- a/tests/queries/0_stateless/03050_select_one_one_one.sql +++ b/tests/queries/0_stateless/03050_select_one_one_one.sql @@ -1,3 +1,4 @@ -- https://github.com/ClickHouse/ClickHouse/issues/36973 +SET allow_experimental_analyzer=1; SELECT 1, 1, 1; SELECT * FROM (SELECT 1, 1, 1); diff --git a/tests/queries/0_stateless/03051_many_ctes.sql b/tests/queries/0_stateless/03051_many_ctes.sql index 412a1e6b544..d4e613bd279 100644 --- a/tests/queries/0_stateless/03051_many_ctes.sql +++ b/tests/queries/0_stateless/03051_many_ctes.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/40955 +SET allow_experimental_analyzer=1; WITH toInt64(2) AS new_x SELECT new_x AS x FROM (SELECT 1 AS x) t; WITH toInt64(2) AS new_x SELECT * replace(new_x as x) FROM (SELECT 1 AS x) t; SELECT 2 AS x FROM (SELECT 1 AS x) t; diff --git a/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql index e76108c7842..24e9ab0f36e 100644 --- a/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql +++ b/tests/queries/0_stateless/03052_query_hash_includes_aliases.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/pull/40065 +SET allow_experimental_analyzer=1; SELECT ( diff --git a/tests/queries/0_stateless/03053_analyzer_join_alias.sql b/tests/queries/0_stateless/03053_analyzer_join_alias.sql index 7e11cc7c810..ef51ec73026 100644 --- a/tests/queries/0_stateless/03053_analyzer_join_alias.sql +++ b/tests/queries/0_stateless/03053_analyzer_join_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23104 +SET allow_experimental_analyzer=1; DROP DATABASE IF EXISTS test_03053; CREATE DATABASE test_03053; diff --git a/tests/queries/0_stateless/03054_analyzer_join_alias.sql b/tests/queries/0_stateless/03054_analyzer_join_alias.sql index 0bf93258aa6..e124aa33a9b 100644 --- a/tests/queries/0_stateless/03054_analyzer_join_alias.sql +++ b/tests/queries/0_stateless/03054_analyzer_join_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/21584 +SET allow_experimental_analyzer=1; SELECT count() FROM ( diff --git a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql index 071d8f8e1c8..25b6dcb3564 100644 --- a/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql +++ b/tests/queries/0_stateless/03055_analyzer_subquery_group_array.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23344 +SET allow_experimental_analyzer=1; SELECT logTrace(repeat('Hello', 100)), ignore(*) FROM ( SELECT ignore((SELECT groupArrayState(([number], [number])) FROM numbers(19000))) diff --git a/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql index b86ae97c8bf..de471c1a091 100644 --- a/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql +++ b/tests/queries/0_stateless/03056_analyzer_double_subquery_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/22627 +SET allow_experimental_analyzer=1; WITH x AS ( diff --git a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql index 13852471dca..2217af327fa 100644 --- a/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql +++ b/tests/queries/0_stateless/03057_analyzer_subquery_alias_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/10276 +SET allow_experimental_analyzer=1; SELECT sum(x.n) as n, sum(z.n) as n2 diff --git a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql index 47df6e76a38..3cce77f0240 100644 --- a/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql +++ b/tests/queries/0_stateless/03058_analyzer_ambiguous_columns.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/4567 +SET allow_experimental_analyzer=1; DROP TABLE IF EXISTS fact; DROP TABLE IF EXISTS animals; DROP TABLE IF EXISTS colors; diff --git a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql index 76150335f7e..27782462075 100644 --- a/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql +++ b/tests/queries/0_stateless/03059_analyzer_join_engine_missing_column.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/17710 +SET allow_experimental_analyzer=1; CREATE TABLE id_val(id UInt32, val UInt32) ENGINE = Memory; CREATE TABLE id_val_join0(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 0; CREATE TABLE id_val_join1(id UInt32, val UInt8) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1; diff --git a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql index ba0257d7b3b..f8cd8690ee5 100644 --- a/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql +++ b/tests/queries/0_stateless/03060_analyzer_regular_view_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11068 +SET allow_experimental_analyzer=1; create table vt(datetime_value DateTime, value Float64) Engine=Memory; create view computed_datum_hours as diff --git a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql index e223909a5a8..6fee6d1f73d 100644 --- a/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql +++ b/tests/queries/0_stateless/03061_analyzer_alias_as_right_key_in_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/24395 +SET allow_experimental_analyzer=1; CREATE TABLE xxxx_yyy (key UInt32, key_b ALIAS key) ENGINE=MergeTree() ORDER BY key; INSERT INTO xxxx_yyy SELECT number FROM numbers(10); diff --git a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql index 6c24ef6f66d..9748175e4d4 100644 --- a/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql +++ b/tests/queries/0_stateless/03062_analyzer_join_engine_missing_column.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23416 +SET allow_experimental_analyzer=1; create table test (TOPIC String, PARTITION UInt64, OFFSET UInt64, ID UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test_03062', 'r2') ORDER BY (TOPIC, PARTITION, OFFSET); create table test_join (TOPIC String, PARTITION UInt64, OFFSET UInt64) ENGINE = Join(ANY, LEFT, `TOPIC`, `PARTITION`) SETTINGS join_any_take_last_row = 1; diff --git a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql index c2c29b688cd..7eab1fa846a 100644 --- a/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql +++ b/tests/queries/0_stateless/03063_analyzer_multi_join_wrong_table_specifier.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23162 +SET allow_experimental_analyzer=1; CREATE TABLE t1 ( k Int64, x Int64) ENGINE = Memory; CREATE TABLE t2( x Int64 ) ENGINE = Memory; diff --git a/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql index ef8aca2fefa..59ebb9d9af3 100644 --- a/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql +++ b/tests/queries/0_stateless/03064_analyzer_named_subqueries.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/25655 +SET allow_experimental_analyzer=1; SELECT sum(t.b) / 1 a, sum(t.a) diff --git a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql index c270a0f4504..7e6befe181e 100644 --- a/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql +++ b/tests/queries/0_stateless/03065_analyzer_cross_join_and_array_join.sql @@ -1,2 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/11757 +SET allow_experimental_analyzer=1; select * from (select [1, 2] a) aa cross join (select [3, 4] b) bb array join aa.a, bb.b; diff --git a/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql index 338eb30e6ff..8983be242c3 100644 --- a/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql +++ b/tests/queries/0_stateless/03066_analyzer_global_with_statement.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; WITH 0 AS test SELECT * FROM diff --git a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql index 7d1264a6116..052a9eaf734 100644 --- a/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql +++ b/tests/queries/0_stateless/03067_analyzer_complex_alias_join.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; with d as (select 'key'::Varchar(255) c, 'x'::Varchar(255) s) SELECT r1, c as r2 FROM ( diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql index 714a64a01b6..82f58e9a750 100644 --- a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/6571 +SET allow_experimental_analyzer=1; CREATE TABLE LINEITEM_shard ON CLUSTER test_shard_localhost ( L_ORDERKEY UInt64, diff --git a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql index 84ad0b4e199..09d2985fe60 100644 --- a/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql +++ b/tests/queries/0_stateless/03069_analyzer_with_alias_in_array_join.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/4432 +SET allow_experimental_analyzer=1; WITH [1, 2] AS zz SELECT x FROM system.one diff --git a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql index 672c4f53e5f..7aadab2ca73 100644 --- a/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql +++ b/tests/queries/0_stateless/03070_analyzer_CTE_scalar_as_numbers.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/8259 +SET allow_experimental_analyzer=1; with (select 25) as something select *, something diff --git a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql index af81e3c2819..e2eb758d649 100644 --- a/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql +++ b/tests/queries/0_stateless/03071_analyzer_array_join_forbid_non_existing_columns.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/9233 +SET allow_experimental_analyzer=1; SELECT * FROM ( diff --git a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql index 68ff81413b7..e2846033913 100644 --- a/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql +++ b/tests/queries/0_stateless/03072_analyzer_missing_columns_from_subquery.sql @@ -1,2 +1,3 @@ -- https://github.com/ClickHouse/ClickHouse/issues/14699 +SET allow_experimental_analyzer=1; select * from (select number from numbers(1)) where not ignore(*); diff --git a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql index 0e7d2eb95c7..5599324c62b 100644 --- a/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql +++ b/tests/queries/0_stateless/03073_analyzer_alias_as_column_name.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/27068 +SET allow_experimental_analyzer=1; CREATE TABLE test ( id String, create_time DateTime ) ENGINE = MergeTree ORDER BY id; insert into test values(1,'1970-02-01 00:00:00'); diff --git a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql index 6f970421788..4df5f6f48e6 100644 --- a/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql +++ b/tests/queries/0_stateless/03074_analyzer_alias_column_in_view.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/28687 +SET allow_experimental_analyzer=1; create view alias (dummy int, n alias dummy) as select * from system.one; select n from alias; diff --git a/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql index 897b189b133..416815e761b 100644 --- a/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql +++ b/tests/queries/0_stateless/03075_analyzer_subquery_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/28777 +SET allow_experimental_analyzer=1; SELECT sum(q0.a2) AS a1, sum(q0.a1) AS a9 diff --git a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql index 8b8b76a5be1..7ac9fe6b446 100644 --- a/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql +++ b/tests/queries/0_stateless/03076_analyzer_multiple_joins_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29734 +SET allow_experimental_analyzer=1; SELECT * FROM ( diff --git a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql index 3d558bdd602..5a181023c57 100644 --- a/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql +++ b/tests/queries/0_stateless/03077_analyzer_multi_scalar_subquery_aliases.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/33825 +SET allow_experimental_analyzer=1; CREATE TABLE t1 (i Int64, j Int64) ENGINE = Memory; INSERT INTO t1 SELECT number, number FROM system.numbers LIMIT 10; SELECT diff --git a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql index ded6bfbe4e3..d91a9ed106d 100644 --- a/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql +++ b/tests/queries/0_stateless/03078_analyzer_multi_scalar_subquery_aliases.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/33825 +SET allow_experimental_analyzer=1; CREATE TABLE t2 (first_column Int64, second_column Int64) ENGINE = Memory; INSERT INTO t2 SELECT number, number FROM system.numbers LIMIT 10; diff --git a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql index e6bcad34506..955d3b49a00 100644 --- a/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql +++ b/tests/queries/0_stateless/03079_analyzer_numeric_literals_as_column_names.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer=1; CREATE TABLE testdata (`1` String) ENGINE=MergeTree ORDER BY tuple(); INSERT INTO testdata VALUES ('testdata'); diff --git a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql index 1fe19cdad2a..01ab868f9ea 100644 --- a/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql +++ b/tests/queries/0_stateless/03080_analyzer_prefer_column_name_to_alias__virtual_columns.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/35652 +SET allow_experimental_analyzer=1; CREATE TABLE test ( id UInt64 ) diff --git a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql index ae8e40f6d56..4985d3abfb6 100644 --- a/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql +++ b/tests/queries/0_stateless/03080_incorrect_join_with_merge.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/29838 +SET allow_experimental_analyzer=1; CREATE TABLE first_table_lr ( diff --git a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql index 81dbbb3b62d..e6a540dc5df 100644 --- a/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql +++ b/tests/queries/0_stateless/03081_analyzer_agg_func_CTE.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/36189 +SET allow_experimental_analyzer=1; CREATE TABLE test ( `dt` Date, diff --git a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql index 93702fee550..8f17248ed0d 100644 --- a/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql +++ b/tests/queries/0_stateless/03082_analyzer_left_join_correct_column.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/39634 +SET allow_experimental_analyzer=1; CREATE TABLE test1 ( `pk` String, diff --git a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql index caf65823532..8337c0ce987 100644 --- a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql +++ b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/47432 +SET allow_experimental_analyzer=1; create or replace table t1 engine = MergeTree() order by tuple() diff --git a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql index f4eaa5d9710..fd67194b08b 100644 --- a/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql +++ b/tests/queries/0_stateless/03085_analyzer_alias_column_group_by.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54910 +SET allow_experimental_analyzer=1; SELECT toTypeName(stat_standard_id) AS stat_standard_id_1, count(1) AS value FROM ( SELECT 'string value' AS stat_standard_id ) GROUP BY stat_standard_id_1 LIMIT 1 diff --git a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql index 55a60873a5a..ac03019de20 100644 --- a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql +++ b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/57321 +SET allow_experimental_analyzer=1; SELECT ver, max(ver) OVER () AS ver_max diff --git a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql index 98aca76fe49..6546e50c99e 100644 --- a/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql +++ b/tests/queries/0_stateless/03087_analyzer_subquery_with_alias.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/59154 +SET allow_experimental_analyzer=1; SELECT * FROM ( diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql index 84afdb295c2..09425d2e503 100644 --- a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql +++ b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61014 +SET allow_experimental_analyzer=1; create database test_03088; create table test_03088.a (i int) engine = Log(); diff --git a/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql index 00a3795eab8..069da5fdd65 100644 --- a/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql +++ b/tests/queries/0_stateless/03089_analyzer_alias_replacement.sql @@ -1,4 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61950 +SET allow_experimental_analyzer=1; with dummy + 1 as dummy select dummy from system.one; From cd1e96a8a10c026214a07bc6bf251c008236fa3c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Apr 2024 16:22:17 +0200 Subject: [PATCH 132/243] Close: https://github.com/ClickHouse/ClickHouse/issues/55647 --- ...analyzer_multiple_using_statements.reference | 1 + ...03090_analyzer_multiple_using_statements.sql | 17 +++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/03090_analyzer_multiple_using_statements.reference create mode 100644 tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql diff --git a/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.reference b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql new file mode 100644 index 00000000000..c35f33782ff --- /dev/null +++ b/tests/queries/0_stateless/03090_analyzer_multiple_using_statements.sql @@ -0,0 +1,17 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/55647 +SET allow_experimental_analyzer=1; + +SELECT +* +FROM ( + SELECT * + FROM system.one +) a +JOIN ( + SELECT * + FROM system.one +) b USING dummy +JOIN ( + SELECT * + FROM system.one +) c USING dummy From 32d124e9033bd3785eb1b301f908bf8733ff433d Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Apr 2024 16:30:42 +0200 Subject: [PATCH 133/243] Close: https://github.com/ClickHouse/ClickHouse/issues/61947 --- ...able_name_in_different_databases.reference | 4 +++ ...same_table_name_in_different_databases.sql | 28 +++++++++++++++++++ ...able_name_in_different_databases.reference | 1 + ...same_table_name_in_different_databases.sql | 18 ++++++++++++ 4 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.reference create mode 100644 tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql create mode 100644 tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.reference create mode 100644 tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql diff --git a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.reference b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.reference new file mode 100644 index 00000000000..ce45f6636b2 --- /dev/null +++ b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.reference @@ -0,0 +1,4 @@ +1 0 + +using asterisk 1 0 +using field name 1 0 diff --git a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql new file mode 100644 index 00000000000..2185b5f450a --- /dev/null +++ b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql @@ -0,0 +1,28 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61947 +SET allow_experimental_analyzer=1; + +DROP DATABASE IF EXISTS d1; +DROP DATABASE IF EXISTS d2; + +CREATE DATABASE d1; +CREATE DATABASE d2; +CREATE TABLE d1.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE d2.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE d2.`2-1` (field Int8) ENGINE = Memory; + +INSERT INTO d1.`1-1` VALUES (1); + +SELECT * +FROM d1.`1-1` +LEFT JOIN d2.`1-1` ON d1.`1-1`.field = d2.`1-1`.field; + +SELECT ''; + +SELECT 'using asterisk', d1.`1-1`.*, d2.`1-1`.* +FROM d1.`1-1` +LEFT JOIN d2.`1-1` USING field +UNION ALL +SELECT 'using field name', d1.`1-1`.field, d2.`1-1`.field +FROM d1.`1-1` +LEFT JOIN d2.`1-1` USING field +ORDER BY *; diff --git a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.reference b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql new file mode 100644 index 00000000000..03ad9c97d94 --- /dev/null +++ b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql @@ -0,0 +1,18 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61947 +SET allow_experimental_analyzer=1; + +DROP DATABASE IF EXISTS d3; +DROP DATABASE IF EXISTS d4; + +CREATE DATABASE d3; +CREATE DATABASE d4; +CREATE TABLE d3.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE d4.`2-1` (field Int8) ENGINE = Memory; +CREATE TABLE d4.`3-1` (field Int8) ENGINE = Memory; + +INSERT INTO d3.`1-1` VALUES (1); + +SELECT d3.`1-1`.* +FROM d3.`1-1` +LEFT JOIN d4.`2-1` ON d3.`1-1`.field = d4.`2-1`.field +LEFT JOIN d4.`3-1` ON d4.`2-1`.field = d4.`3-1`.field; From 58e6bd82cff7c18e19ce52ea97c993456aba60df Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 4 Apr 2024 16:37:49 +0200 Subject: [PATCH 134/243] Add missing ORDER BY --- .../0_stateless/03086_analyzer_window_func_part_of_group_by.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql index ac03019de20..31747328d1f 100644 --- a/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql +++ b/tests/queries/0_stateless/03086_analyzer_window_func_part_of_group_by.sql @@ -10,3 +10,4 @@ FROM SELECT 2 AS ver ) GROUP BY ver +ORDER BY ver; From 85d98e1a421cecb8be9051331f8fdd46bfd557ea Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 4 Apr 2024 16:51:46 +0200 Subject: [PATCH 135/243] Modernize code a little --- src/Common/QueryProfiler.cpp | 16 ++-------------- src/Common/ThreadStatus.cpp | 2 +- 2 files changed, 3 insertions(+), 15 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index b616b795405..d9ff3a86e4a 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -210,23 +210,13 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_) +QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) : log(getLogger("QueryProfiler")) , pause_signal(pause_signal_) { #if defined(SANITIZER) - UNUSED(thread_id); - UNUSED(clock_type); - UNUSED(period); - UNUSED(pause_signal); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); #elif defined(__APPLE__) - UNUSED(thread_id); - UNUSED(clock_type); - UNUSED(period); - UNUSED(pause_signal); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler cannot work on OSX"); #else /// Sanity check. @@ -262,13 +252,11 @@ QueryProfilerBase::QueryProfilerBase(UInt64 thread_id, int clock_t template -void QueryProfilerBase::setPeriod(UInt32 period_) +void QueryProfilerBase::setPeriod([[maybe_unused]] UInt32 period_) { #if defined(SANITIZER) - UNUSED(period); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); #elif defined(__APPLE__) - UNUSED(period); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler cannot work on OSX"); #else timer.set(period_); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index a3a7257b472..d4a40360a3b 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -124,7 +124,7 @@ ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) #endif } -void ThreadStatus::initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period) +void ThreadStatus::initGlobalProfiler([[maybe_unused]] UInt64 global_profiler_real_time_period, [[maybe_unused]] UInt64 global_profiler_cpu_time_period) { #if !defined(SANITIZER) && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) && !defined(__APPLE__) try From eede7c3acf5e1fcc113349b4014aaf5b981ce285 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 19:01:31 +0200 Subject: [PATCH 136/243] Try fix stress test --- src/Interpreters/Cache/EvictionCandidates.cpp | 2 ++ src/Interpreters/Cache/FileCache.cpp | 31 ++++++++++++++----- src/Interpreters/Cache/Metadata.cpp | 1 - .../Cache/SLRUFileCachePriority.cpp | 10 +++++- 4 files changed, 35 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index 5fa2b337e64..da9e3efbffe 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -63,6 +63,8 @@ void EvictionCandidates::add( void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & lock) { + auto log = getLogger("EvictionCandidates"); + LOG_TEST(log, "Will remove {} eviction candidates", size()); for (const auto & [key, key_candidates] : candidates) { for (const auto & candidate : key_candidates.candidates) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 90671629e64..5e97c0110f4 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1383,6 +1383,7 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, || new_settings.max_elements != actual_settings.max_elements) { std::optional eviction_candidates; + bool modified_size_limits = false; { cache_is_being_resized.store(true, std::memory_order_relaxed); SCOPE_EXIT({ @@ -1397,8 +1398,21 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, eviction_candidates->removeQueueEntries(cache_lock); - main_priority->modifySizeLimits( - new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); + modified_size_limits = main_priority->getSize(cache_lock) <= new_settings.max_size + && main_priority->getElementsCount(cache_lock) <= new_settings.max_elements; + + if (modified_size_limits) + { + main_priority->modifySizeLimits( + new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); + } + else + { + LOG_WARNING(log, "Unable to modify size limit from {} to {}, " + "elements limit from {} to {}", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); + } } try @@ -1412,12 +1426,15 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, throw; } - LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", - actual_settings.max_size, new_settings.max_size, - actual_settings.max_elements, new_settings.max_elements); + if (modified_size_limits) + { + LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); - actual_settings.max_size = new_settings.max_size; - actual_settings.max_elements = new_settings.max_elements; + actual_settings.max_size = new_settings.max_size; + actual_settings.max_elements = new_settings.max_elements; + } } if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 26611f02379..631c1aa2ae6 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -941,7 +941,6 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( file_segment->detach(segment_lock, *this); - // if (!remove_only_metadata) { try { diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 79ca489cea2..ff583c440c8 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -262,6 +262,9 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( auto res = probationary_queue.collectCandidatesForEviction( desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); + LOG_TEST(log, "Collected {} eviction candidates from probationary queue (size: {})", + res.size(), stat.total_stat.releasable_size); + chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); @@ -271,10 +274,15 @@ EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( const auto desired_protected_size = getRatio(max_size, size_ratio); const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); + FileCacheReserveStat protected_stat; auto res_add = protected_queue.collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, - max_candidates_to_evict ? max_candidates_to_evict - res.size() : 0, stat, lock); + max_candidates_to_evict ? max_candidates_to_evict - res.size() : 0, protected_stat, lock); + LOG_TEST(log, "Collected {} eviction candidates from protected queue (size: {})", + res_add.size(), protected_stat.total_stat.releasable_size); + + stat += protected_stat; res.insert(std::move(res_add), lock); return res; } From b1bd34f66e82173bfc48c7e1a612a967562fcbc6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 4 Apr 2024 20:25:49 +0000 Subject: [PATCH 137/243] fix --- src/Processors/QueryPlan/PartsSplitter.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 2af1bcb0260..ec51875587e 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -125,14 +126,18 @@ int compareValues(const Values & lhs, const Values & rhs) class IndexAccess { public: - explicit IndexAccess(const RangesInDataParts & parts_) : parts(parts_) { } + explicit IndexAccess(const RangesInDataParts & parts_) : parts(parts_) + { + for (const auto & part : parts) + loaded_columns = std::min(loaded_columns, part.data_part->getIndex().size()); + } Values getValue(size_t part_idx, size_t mark) const { const auto & index = parts[part_idx].data_part->getIndex(); - size_t size = index.size(); - Values values(size); - for (size_t i = 0; i < size; ++i) + chassert(index.size() >= loaded_columns); + Values values(loaded_columns); + for (size_t i = 0; i < loaded_columns; ++i) { index[i]->get(mark, values[i]); if (values[i].isNull()) @@ -199,6 +204,7 @@ public: } private: const RangesInDataParts & parts; + size_t loaded_columns = std::numeric_limits::max(); }; class RangesInDataPartsBuilder From 6be747bf32a7f1fcd9fee8f86c72dd2b03e48c02 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 4 Apr 2024 22:28:29 +0000 Subject: [PATCH 138/243] add test --- .../__init__.py | 0 .../test.py | 47 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/integration/test_final_bug_with_pk_columns_loading/__init__.py create mode 100644 tests/integration/test_final_bug_with_pk_columns_loading/test.py diff --git a/tests/integration/test_final_bug_with_pk_columns_loading/__init__.py b/tests/integration/test_final_bug_with_pk_columns_loading/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_final_bug_with_pk_columns_loading/test.py b/tests/integration/test_final_bug_with_pk_columns_loading/test.py new file mode 100644 index 00000000000..e710b9942dc --- /dev/null +++ b/tests/integration/test_final_bug_with_pk_columns_loading/test.py @@ -0,0 +1,47 @@ +import pytest +import logging + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", stay_alive=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_simple_query_after_restart(start_cluster): + node.query( + """ + create table t(a UInt32, b UInt32) engine=MergeTree order by (a, b) settings index_granularity=1; + + insert into t select 42, number from numbers_mt(100); + insert into t select number, number from numbers_mt(100); + """ + ) + + node.restart_clickhouse() + + assert ( + int( + node.query( + "select count() from t where not ignore(*)", + settings={ + "max_threads": 4, + "merge_tree_min_bytes_for_concurrent_read": 1, + "merge_tree_min_rows_for_concurrent_read": 1, + "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": 1, + }, + ) + ) + == 200 + ) From dd852da33925af8ba52a89034da774d512add241 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Apr 2024 15:57:42 +0200 Subject: [PATCH 139/243] add more debug logs --- .../Net/include/Poco/Net/HTTPClientSession.h | 5 -- base/poco/Net/src/HTTPClientSession.cpp | 26 +++++-- base/poco/Net/src/HTTPMessage.cpp | 2 +- src/Common/HTTPConnectionPool.cpp | 68 ++++++++++++++++--- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 + src/IO/S3/Client.h | 8 +-- src/IO/S3/PocoHTTPClient.cpp | 3 +- src/IO/S3/PocoHTTPClient.h | 1 + src/IO/S3/tests/gtest_aws_s3_client.cpp | 2 +- 9 files changed, 91 insertions(+), 26 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index b418937c4d5..cbf4619834b 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -458,11 +458,6 @@ namespace Net return _lastRequest; } - inline void HTTPClientSession::setLastRequest(Poco::Timestamp time) - { - _lastRequest = time; - } - inline double HTTPClientSession::getKeepAliveReliability() const { return _defaultKeepAliveReliabilityLevel; diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index 59800232ba9..afa1eff68a2 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -223,12 +223,24 @@ void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout) { if (connected()) { - throw Poco::IllegalStateException("cannot change keep alive timeout on initiated connection"); + throw Poco::IllegalStateException("cannot change keep alive timeout on initiated connection, " + "That value is managed privately after connection is established."); } _keepAliveTimeout = timeout; } +void HTTPClientSession::setLastRequest(Poco::Timestamp time) +{ + if (connected()) + { + throw Poco::IllegalStateException("cannot change last request on initiated connection, " + "That value is managed privately after connection is established."); + } + _lastRequest = time; +} + + std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) { _pRequestStream = 0; @@ -246,8 +258,8 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) { if (!connected()) reconnect(); - if (!keepAlive) - request.setKeepAlive(false); + if (!request.has(HTTPMessage::CONNECTION)) + request.setKeepAlive(keepAlive); if (keepAlive && !request.has(HTTPMessage::CONNECTION_KEEP_ALIVE) && _keepAliveTimeout.totalSeconds() > 0) request.setKeepAliveTimeout(_keepAliveTimeout.totalSeconds()); if (!request.has(HTTPRequest::HOST) && !_host.empty()) @@ -528,14 +540,16 @@ void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session) if (buffered()) throw Poco::LogicException("assign to a session with not empty buffered data"); - setLastRequest(session.getLastRequest()); + poco_assert(!connected()); + setResolvedHost(session.getResolvedHost()); setProxyConfig(session.getProxyConfig()); setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout()); setKeepAlive(session.getKeepAlive()); - if (!connected()) - setKeepAliveTimeout(session.getKeepAliveTimeout()); + + setLastRequest(session.getLastRequest()); + setKeepAliveTimeout(session.getKeepAliveTimeout()); attachSocket(session.detachSocket()); diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index 2f974b8bf0b..af743dfa2eb 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -182,7 +182,7 @@ bool HTTPMessage::getKeepAlive() const void HTTPMessage::setKeepAliveTimeout(int timeout) { - add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}", timeout)); + add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}, max=1000", timeout)); } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 2d3a87dda6b..f64d6658a55 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -193,6 +193,18 @@ public: return total_connections_in_group >= limits.store_limit; } + size_t getStored() const + { + std::lock_guard lock(mutex); + return total_connections_in_group; + } + + size_t getStoreLimit() const + { + std::lock_guard lock(mutex); + return limits.store_limit; + } + void atConnectionCreate() { std::lock_guard lock(mutex); @@ -221,12 +233,6 @@ public: } } - void atPoolDestroy(size_t connections) - { - std::lock_guard lock(mutex); - total_connections_in_group -= connections; - } - HTTPConnectionGroupType getType() const { return type; } const IHTTPConnectionPoolForEndpoint::Metrics & getMetrics() const { return metrics; } @@ -345,11 +351,29 @@ private: Session::flushRequest(); } + String printAllHeaders(Poco::Net::HTTPMessage & message) const + { + String out; + out.reserve(300); + for (auto & [k, v] : message) + { + out.append(fmt::format("<{}: {}> ", k, v)); + } + return out; + } + std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override { std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); + // that line is for temporary debug, will be removed + LOG_INFO(log, "Send request to {} with: usage count {}, keep-alive timeout={}, headers: {}", + getTarget(), + usage_cnt, + Session::getKeepAliveTimeout().totalSeconds(), + printAllHeaders(request)); + request_stream = &result; request_stream_completed = false; @@ -368,9 +392,12 @@ private: // that line is for temporary debug, will be removed if (response.has(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE)) - LOG_INFO(log, "received keep alive header: {}, original was {}", + LOG_INFO(log, "Received response from {} with: usage count {}, keep alive header: {}, original ka {}, headers: {}", + getTarget(), + usage_cnt, response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), - originKA); + originKA, + printAllHeaders(response)); response_stream = &result; response_stream_completed = false; @@ -415,8 +442,19 @@ private: group->atConnectionDestroy(); if (!isExpired) + { if (auto lock = pool.lock()) lock->atConnectionDestroy(*this); + } + else + { + Poco::Timestamp now; + LOG_INFO(log, "Expired connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}", + getTarget(), + usage_cnt, + Session::getKeepAliveTimeout().totalSeconds(), + Poco::Timespan(now - Session::getLastRequest()).totalSeconds()); + } CurrentMetrics::sub(metrics.active_count); } @@ -459,6 +497,7 @@ private: ConnectionGroup::Ptr group; IHTTPConnectionPoolForEndpoint::Metrics metrics; bool isExpired = false; + size_t usage_cnt = 1; LoggerPtr log = getLogger("PooledConnection"); @@ -527,6 +566,8 @@ public: stored_connections.pop(); setTimeouts(*it, timeouts); + it->usage_cnt += 1; + ProfileEvents::increment(getMetrics().reused, 1); CurrentMetrics::sub(getMetrics().stored_count, 1); @@ -647,12 +688,23 @@ private: if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() || group->isStoreLimitReached()) { + Poco::Timestamp now; + LOG_INFO(getLogger("PooledConnection"), + "Reset connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}, is completed {}, store limit reached {} as {}/{}", + getTarget(), + connection.usage_cnt, + connection.getKeepAliveTimeout().totalSeconds(), + Poco::Timespan(now - connection.getLastRequest()).totalSeconds(), + connection.isCompleted(), + group->isStoreLimitReached(), group->getStored(), group->getStoreLimit()); + ProfileEvents::increment(getMetrics().reset, 1); return; } auto connection_to_store = allocateNewConnection(); connection_to_store->assign(connection); + connection_to_store->usage_cnt = connection.usage_cnt; { MemoryTrackerSwitcher switcher{&total_memory_tracker}; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index df1ccbb32d9..7ce94699053 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -76,6 +76,8 @@ std::unique_ptr getClient( client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS); client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); + client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + client_configuration.endpointOverride = uri.endpoint; client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index c7bc727bf32..c79ec05c8c6 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -96,9 +96,9 @@ bool isS3ExpressEndpoint(const std::string & endpoint); struct ClientSettings { - bool use_virtual_addressing; + bool use_virtual_addressing = false; /// Disable checksum to avoid extra read of the input stream - bool disable_checksum; + bool disable_checksum = false; /// Should client send ComposeObject request after upload to GCS. /// /// Previously ComposeObject request was required to make Copy possible, @@ -108,8 +108,8 @@ struct ClientSettings /// /// Ability to enable it preserved since likely it is required for old /// files. - bool gcs_issue_compose_request; - bool is_s3express_bucket; + bool gcs_issue_compose_request = false; + bool is_s3express_bucket = false; }; /// Client that improves the client from the AWS SDK diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index a29a4b0b8ee..150b8146147 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -146,7 +146,8 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio .withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withTCPKeepAliveTimeout(Poco::Timespan( - client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)); + client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)) + .withHTTPKeepAliveTimeout(Poco::Timespan(client_configuration.http_keep_alive_timeout, 0)); } PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration) diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index ebbddbb2c7e..f568eb5ddb8 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -51,6 +51,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration /// See PoolBase::BehaviourOnLimit bool s3_use_adaptive_timeouts = true; + size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT; std::function error_report; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 25786619241..0a28c578f69 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -159,7 +159,7 @@ void testServerSideEncryption( DB::S3::CredentialsConfiguration { .use_environment_credentials = use_environment_credentials, - .use_insecure_imds_request = use_insecure_imds_request + .use_insecure_imds_request = use_insecure_imds_request, } ); From 5cab8d185fb5ad1f8607a4ad7140a15469754e99 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Apr 2024 19:29:42 +0200 Subject: [PATCH 140/243] more details --- base/poco/Net/src/HTTPClientSession.cpp | 2 +- src/Common/HTTPConnectionPool.cpp | 42 ++++++++++++++++--------- 2 files changed, 29 insertions(+), 15 deletions(-) diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index afa1eff68a2..bc70559c5eb 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -348,7 +348,7 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response) /// when server sends its keep alive timeout, client has to follow that value auto timeout = response.getKeepAliveTimeout(); if (timeout > 0) - _keepAliveTimeout = Poco::Timespan(timeout, 0); + _keepAliveTimeout = std::min(_keepAliveTimeout, Poco::Timespan(timeout, 0)); } if (!_expectResponseBody || response.getStatus() < 200 || response.getStatus() == HTTPResponse::HTTP_NO_CONTENT || response.getStatus() == HTTPResponse::HTTP_NOT_MODIFIED) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index f64d6658a55..eb6ce00e611 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -322,6 +322,11 @@ private: Session::getPort()); } + Poco::Timespan idleTime() { + Poco::Timestamp now; + return now - Session::getLastRequest(); + } + void flushRequest() override { if (bool(request_stream)) @@ -364,14 +369,18 @@ private: std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override { + auto idle = idleTime(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); // that line is for temporary debug, will be removed - LOG_INFO(log, "Send request to {} with: usage count {}, keep-alive timeout={}, headers: {}", + LOG_INFO(log, "Send request to {} with: version {}, method {}, usage count {}, keep-alive timeout={}, last usage ago: {}ms, headers: {}", + request.getVersion(), + request.getMethod(), getTarget(), usage_cnt, Session::getKeepAliveTimeout().totalSeconds(), + idle.totalMilliseconds(), printAllHeaders(request)); request_stream = &result; @@ -391,13 +400,15 @@ private: result.exceptions(std::ios::badbit); // that line is for temporary debug, will be removed - if (response.has(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE)) - LOG_INFO(log, "Received response from {} with: usage count {}, keep alive header: {}, original ka {}, headers: {}", - getTarget(), - usage_cnt, - response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), - originKA, - printAllHeaders(response)); + LOG_INFO(log, "Received response from {} with: version {}, code {}, usage count {}, keep alive header: {}, original ka {}, last usage ago: {}ms, headers: {}", + getTarget(), + response.getVersion(), + int(response.getStatus()), + usage_cnt, + response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), + originKA, + idleTime().totalMilliseconds(), + printAllHeaders(response)); response_stream = &result; response_stream_completed = false; @@ -449,11 +460,11 @@ private: else { Poco::Timestamp now; - LOG_INFO(log, "Expired connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}", + LOG_INFO(log, "Expired connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}s", getTarget(), usage_cnt, Session::getKeepAliveTimeout().totalSeconds(), - Poco::Timespan(now - Session::getLastRequest()).totalSeconds()); + idleTime().totalSeconds()); } CurrentMetrics::sub(metrics.active_count); @@ -498,6 +509,7 @@ private: IHTTPConnectionPoolForEndpoint::Metrics metrics; bool isExpired = false; size_t usage_cnt = 1; + size_t exception_level = std::uncaught_exceptions(); LoggerPtr log = getLogger("PooledConnection"); @@ -568,7 +580,6 @@ public: setTimeouts(*it, timeouts); it->usage_cnt += 1; - ProfileEvents::increment(getMetrics().reused, 1); CurrentMetrics::sub(getMetrics().stored_count, 1); @@ -690,13 +701,16 @@ private: { Poco::Timestamp now; LOG_INFO(getLogger("PooledConnection"), - "Reset connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}, is completed {}, store limit reached {} as {}/{}", + "Reset connection to {} with: usage count {}, keep alive timeout: {}, connected {}, must recon {}, last usage ago: {}, is completed {}, store limit reached {} as {}/{}, there is exception {}", getTarget(), connection.usage_cnt, connection.getKeepAliveTimeout().totalSeconds(), - Poco::Timespan(now - connection.getLastRequest()).totalSeconds(), + connection.connected(), + connection.mustReconnect(), + connection.idleTime().totalSeconds(), connection.isCompleted(), - group->isStoreLimitReached(), group->getStored(), group->getStoreLimit()); + group->isStoreLimitReached(), group->getStored(), group->getStoreLimit(), + connection.exception_level - std::uncaught_exceptions()); ProfileEvents::increment(getMetrics().reset, 1); return; From ae3a1999398b4f16880e2d892cb11bb414944b81 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 4 Apr 2024 22:49:52 +0200 Subject: [PATCH 141/243] support max requests for keep alive --- .../Net/include/Poco/Net/HTTPClientSession.h | 20 ++++ base/poco/Net/include/Poco/Net/HTTPMessage.h | 3 +- base/poco/Net/src/HTTPClientSession.cpp | 33 +++++- base/poco/Net/src/HTTPMessage.cpp | 37 ++++--- src/Common/HTTPConnectionPool.cpp | 69 +++++++----- src/Common/tests/gtest_connection_pool.cpp | 103 ++++++++++++++++-- src/Core/Defines.h | 1 + src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +- src/IO/ConnectionTimeouts.cpp | 1 + src/IO/ConnectionTimeouts.h | 2 + src/IO/S3/Credentials.h | 2 + src/IO/S3/PocoHTTPClient.h | 1 + 12 files changed, 219 insertions(+), 56 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index cbf4619834b..edbb135d8c6 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -213,6 +213,12 @@ namespace Net Poco::Timespan getKeepAliveTimeout() const; /// Returns the connection timeout for HTTP connections. + void setKeepAliveMaxRequests(int max_requests); + + int getKeepAliveMaxRequests() const; + + int getKeepAliveRequest() const; + bool isKeepAliveExpired(double reliability = 1.0) const; /// Returns if the connection is expired with some margin as fraction of timeout as reliability @@ -352,6 +358,8 @@ namespace Net void assign(HTTPClientSession & session); + void setKeepAliveRequest(int request); + HTTPSessionFactory _proxySessionFactory; /// Factory to create HTTPClientSession to proxy. private: @@ -360,6 +368,8 @@ namespace Net Poco::UInt16 _port; ProxyConfig _proxyConfig; Poco::Timespan _keepAliveTimeout; + int _keepAliveCurrentRequest = 0; + int _keepAliveMaxRequests = 1000; Poco::Timestamp _lastRequest; bool _reconnect; bool _mustReconnect; @@ -463,6 +473,16 @@ namespace Net return _defaultKeepAliveReliabilityLevel; } + inline int HTTPClientSession::getKeepAliveMaxRequests() const + { + return _keepAliveMaxRequests; + } + + inline int HTTPClientSession::getKeepAliveRequest() const + { + return _keepAliveCurrentRequest; + } + } } // namespace Poco::Net diff --git a/base/poco/Net/include/Poco/Net/HTTPMessage.h b/base/poco/Net/include/Poco/Net/HTTPMessage.h index 994807ffbff..8bc95ccc1af 100644 --- a/base/poco/Net/include/Poco/Net/HTTPMessage.h +++ b/base/poco/Net/include/Poco/Net/HTTPMessage.h @@ -120,8 +120,9 @@ namespace Net /// The value is set to "Keep-Alive" if keepAlive is /// true, or to "Close" otherwise. - void setKeepAliveTimeout(int timeout); + void setKeepAliveTimeout(int timeout, int max_requests); int getKeepAliveTimeout() const; + int getKeepAliveMaxRequests() const; bool getKeepAlive() const; /// Returns true if diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index bc70559c5eb..e489ab56b98 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -230,7 +230,25 @@ void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout) } -void HTTPClientSession::setLastRequest(Poco::Timestamp time) +void HTTPClientSession::setKeepAliveMaxRequests(int max_requests) +{ + if (connected()) + { + throw Poco::IllegalStateException("cannot change keep alive max requests on initiated connection, " + "That value is managed privately after connection is established."); + } + _keepAliveMaxRequests = max_requests; +} + + +void HTTPClientSession::setKeepAliveRequest(int request) +{ + _keepAliveCurrentRequest = request; +} + + + + void HTTPClientSession::setLastRequest(Poco::Timestamp time) { if (connected()) { @@ -248,6 +266,8 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) clearException(); _responseReceived = false; + _keepAliveCurrentRequest += 1; + bool keepAlive = getKeepAlive(); if (((connected() && !keepAlive) || mustReconnect()) && !_host.empty()) { @@ -261,7 +281,7 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request) if (!request.has(HTTPMessage::CONNECTION)) request.setKeepAlive(keepAlive); if (keepAlive && !request.has(HTTPMessage::CONNECTION_KEEP_ALIVE) && _keepAliveTimeout.totalSeconds() > 0) - request.setKeepAliveTimeout(_keepAliveTimeout.totalSeconds()); + request.setKeepAliveTimeout(_keepAliveTimeout.totalSeconds(), _keepAliveMaxRequests); if (!request.has(HTTPRequest::HOST) && !_host.empty()) request.setHost(_host, _port); if (!_proxyConfig.host.empty() && !bypassProxy()) @@ -349,6 +369,9 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response) auto timeout = response.getKeepAliveTimeout(); if (timeout > 0) _keepAliveTimeout = std::min(_keepAliveTimeout, Poco::Timespan(timeout, 0)); + auto max_requests = response.getKeepAliveMaxRequests(); + if (max_requests > 0) + _keepAliveMaxRequests = std::min(_keepAliveMaxRequests, max_requests); } if (!_expectResponseBody || response.getStatus() < 200 || response.getStatus() == HTTPResponse::HTTP_NO_CONTENT || response.getStatus() == HTTPResponse::HTTP_NOT_MODIFIED) @@ -460,7 +483,8 @@ std::string HTTPClientSession::proxyRequestPrefix() const bool HTTPClientSession::isKeepAliveExpired(double reliability) const { Poco::Timestamp now; - return Timespan(Timestamp::TimeDiff(reliability *_keepAliveTimeout.totalMicroseconds())) <= now - _lastRequest; + return Timespan(Timestamp::TimeDiff(reliability *_keepAliveTimeout.totalMicroseconds())) <= now - _lastRequest + || _keepAliveCurrentRequest > _keepAliveMaxRequests; } bool HTTPClientSession::mustReconnect() const @@ -551,6 +575,9 @@ void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session) setLastRequest(session.getLastRequest()); setKeepAliveTimeout(session.getKeepAliveTimeout()); + _keepAliveMaxRequests = session._keepAliveMaxRequests; + _keepAliveCurrentRequest = session._keepAliveCurrentRequest; + attachSocket(session.detachSocket()); session.reset(); diff --git a/base/poco/Net/src/HTTPMessage.cpp b/base/poco/Net/src/HTTPMessage.cpp index af743dfa2eb..c0083ec410c 100644 --- a/base/poco/Net/src/HTTPMessage.cpp +++ b/base/poco/Net/src/HTTPMessage.cpp @@ -180,27 +180,25 @@ bool HTTPMessage::getKeepAlive() const } -void HTTPMessage::setKeepAliveTimeout(int timeout) +void HTTPMessage::setKeepAliveTimeout(int timeout, int max_requests) { - add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}, max=1000", timeout)); + add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}, max={}", timeout, max_requests)); } -int parseTimeoutFromHeaderValue(const std::string_view header_value) +int parseFromHeaderValues(const std::string_view header_value, const std::string_view param_name) { - static const std::string_view timeout_param = "timeout="; + auto param_value_pos = header_value.find(param_name); + if (param_value_pos == std::string::npos) + param_value_pos = header_value.size(); + if (param_value_pos != header_value.size()) + param_value_pos += param_name.size(); - auto timeout_pos = header_value.find(timeout_param); - if (timeout_pos == std::string::npos) - timeout_pos = header_value.size(); - if (timeout_pos != header_value.size()) - timeout_pos += timeout_param.size(); + auto param_value_end = header_value.find(',', param_value_pos); + if (param_value_end == std::string::npos) + param_value_end = header_value.size(); - auto timeout_end = header_value.find(',', timeout_pos); - if (timeout_end == std::string::npos) - timeout_end = header_value.size(); - - auto timeout_value_substr = header_value.substr(timeout_pos, timeout_end - timeout_pos); + auto timeout_value_substr = header_value.substr(param_value_pos, param_value_end - param_value_pos); if (timeout_value_substr.empty()) return -1; @@ -217,7 +215,16 @@ int parseTimeoutFromHeaderValue(const std::string_view header_value) int HTTPMessage::getKeepAliveTimeout() const { const std::string& ka_header = get(HTTPMessage::CONNECTION_KEEP_ALIVE, HTTPMessage::EMPTY); - return parseTimeoutFromHeaderValue(ka_header); + static const std::string_view timeout_param = "timeout="; + return parseFromHeaderValues(ka_header, timeout_param); +} + + +int HTTPMessage::getKeepAliveMaxRequests() const +{ + const std::string& ka_header = get(HTTPMessage::CONNECTION_KEEP_ALIVE, HTTPMessage::EMPTY); + static const std::string_view timeout_param = "max="; + return parseFromHeaderValues(ka_header, timeout_param); } } } // namespace Poco::Net diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index eb6ce00e611..926222934e4 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -301,6 +301,8 @@ private: auto timeouts = getTimeouts(*this); auto new_connection = lock->getConnection(timeouts); Session::assign(*new_connection); + if (Session::getKeepAliveRequest() == 0) + Session::setKeepAliveRequest(1); } else { @@ -322,7 +324,8 @@ private: Session::getPort()); } - Poco::Timespan idleTime() { + Poco::Timespan idleTime() + { Poco::Timestamp now; return now - Session::getLastRequest(); } @@ -374,11 +377,11 @@ private: result.exceptions(std::ios::badbit); // that line is for temporary debug, will be removed - LOG_INFO(log, "Send request to {} with: version {}, method {}, usage count {}, keep-alive timeout={}, last usage ago: {}ms, headers: {}", + LOG_INFO(log, "Send request to {} with: version {}, method {}, request no {}, keep-alive timeout={}, last usage ago: {}ms, headers: {}", request.getVersion(), request.getMethod(), getTarget(), - usage_cnt, + Session::getKeepAliveRequest(), Session::getKeepAliveTimeout().totalSeconds(), idle.totalMilliseconds(), printAllHeaders(request)); @@ -400,11 +403,11 @@ private: result.exceptions(std::ios::badbit); // that line is for temporary debug, will be removed - LOG_INFO(log, "Received response from {} with: version {}, code {}, usage count {}, keep alive header: {}, original ka {}, last usage ago: {}ms, headers: {}", + LOG_INFO(log, "Received response from {} with: version {}, code {}, request no {}, keep alive header: {}, original ka {}, last usage ago: {}ms, headers: {}", getTarget(), response.getVersion(), int(response.getStatus()), - usage_cnt, + Session::getKeepAliveRequest(), response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), originKA, idleTime().totalMilliseconds(), @@ -460,9 +463,9 @@ private: else { Poco::Timestamp now; - LOG_INFO(log, "Expired connection to {} with: usage count {}, keep alive timeout: {}, last usage ago: {}s", + LOG_INFO(log, "Expired connection to {} with: request no {}, keep alive timeout: {}, last usage ago: {}s", getTarget(), - usage_cnt, + Session::getKeepAliveRequest(), Session::getKeepAliveTimeout().totalSeconds(), idleTime().totalSeconds()); } @@ -474,8 +477,15 @@ private: friend class EndpointConnectionPool; template - explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, ConnectionGroup::Ptr group_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) - : Session(args...), pool(std::move(pool_)), group(group_), metrics(std::move(metrics_)) + explicit PooledConnection( + EndpointConnectionPool::WeakPtr pool_, + ConnectionGroup::Ptr group_, + IHTTPConnectionPoolForEndpoint::Metrics metrics_, + Args &&... args) + : Session(args...) + , pool(std::move(pool_)) + , group(group_) + , metrics(std::move(metrics_)) { CurrentMetrics::add(metrics.active_count); group->atConnectionCreate(); @@ -508,7 +518,7 @@ private: ConnectionGroup::Ptr group; IHTTPConnectionPoolForEndpoint::Metrics metrics; bool isExpired = false; - size_t usage_cnt = 1; + size_t exception_level = std::uncaught_exceptions(); LoggerPtr log = getLogger("PooledConnection"); @@ -578,7 +588,6 @@ public: stored_connections.pop(); setTimeouts(*it, timeouts); - it->usage_cnt += 1; ProfileEvents::increment(getMetrics().reused, 1); CurrentMetrics::sub(getMetrics().stored_count, 1); @@ -655,47 +664,50 @@ private: return connection->isKeepAliveExpired(0.8); } - ConnectionPtr allocateNewConnection() + + ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts) { - ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); + auto connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); + connection->setKeepAlive(true); + setTimeouts(*connection, timeouts); if (!proxy_configuration.isEmpty()) { connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration)); } - return connection; - } - - ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts) - { auto address = HostResolversPool::instance().getResolver(host)->resolve(); - - auto session = allocateNewConnection(); - - setTimeouts(*session, timeouts); - session->setResolvedHost(*address); + connection->setResolvedHost(*address); try { auto timer = CurrentThread::getProfileEvents().timer(getMetrics().elapsed_microseconds); - session->doConnect(); + connection->doConnect(); } catch (...) { address.setFail(); ProfileEvents::increment(getMetrics().errors); - session->reset(); + connection->reset(); throw; } ProfileEvents::increment(getMetrics().created); - return session; + return connection; } void atConnectionDestroy(PooledConnection & connection) { + if (connection.getKeepAliveRequest() >= connection.getKeepAliveMaxRequests()) + { + LOG_INFO(getLogger("PooledConnection"), "Expired by connection number {}", + connection.getKeepAliveRequest()); + + ProfileEvents::increment(getMetrics().expired, 1); + return; + } + if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() || group->isStoreLimitReached()) { @@ -703,7 +715,7 @@ private: LOG_INFO(getLogger("PooledConnection"), "Reset connection to {} with: usage count {}, keep alive timeout: {}, connected {}, must recon {}, last usage ago: {}, is completed {}, store limit reached {} as {}/{}, there is exception {}", getTarget(), - connection.usage_cnt, + connection.getKeepAliveRequest(), connection.getKeepAliveTimeout().totalSeconds(), connection.connected(), connection.mustReconnect(), @@ -716,9 +728,8 @@ private: return; } - auto connection_to_store = allocateNewConnection(); + auto connection_to_store = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port); connection_to_store->assign(connection); - connection_to_store->usage_cnt = connection.usage_cnt; { MemoryTrackerSwitcher switcher{&total_memory_tracker}; diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp index 36bf8bc7dae..cc091d12bb0 100644 --- a/src/Common/tests/gtest_connection_pool.cpp +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -47,6 +47,7 @@ struct RequestOptions { size_t slowdown_receive = 0; int overwrite_keep_alive_timeout = 0; + int overwrite_keep_alive_max_requests = 10; }; size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits::max()) @@ -89,8 +90,10 @@ public: int value = request.getKeepAliveTimeout(); ASSERT_GT(value, 0); - if (options->get().overwrite_keep_alive_timeout > 0) - response.setKeepAliveTimeout(options->get().overwrite_keep_alive_timeout); + auto params = options->get(); + + if (params.overwrite_keep_alive_timeout > 0) + response.setKeepAliveTimeout(params.overwrite_keep_alive_timeout, params.overwrite_keep_alive_max_requests); response.setStatus(Poco::Net::HTTPResponse::HTTP_OK); auto size = request.getContentLength(); @@ -99,8 +102,8 @@ public: else response.setChunkedTransferEncoding(true); // or chunk encoding - if (options->get().slowdown_receive > 0) - sleepForSeconds(options->get().slowdown_receive); + if (params.slowdown_receive > 0) + sleepForSeconds(params.slowdown_receive); stream_copy_n(request.stream(), response.send(), size); } @@ -189,10 +192,11 @@ protected: options->set(std::move(opt)); } - void setOverWriteTimeout(size_t seconds) + void setOverWriteKeepAlive(size_t seconds, int max_requests) { auto opt = options->get(); opt.overwrite_keep_alive_timeout = int(seconds); + opt.overwrite_keep_alive_max_requests= max_requests; options->set(std::move(opt)); } @@ -794,7 +798,7 @@ TEST_F(ConnectionPoolTest, ServerOverwriteKeepAlive) } { - setOverWriteTimeout(1); + setOverWriteKeepAlive(1, 10); auto connection = pool->getConnection(timeouts); echoRequest("Hello", *connection); ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds()); @@ -803,7 +807,7 @@ TEST_F(ConnectionPoolTest, ServerOverwriteKeepAlive) { // server do not overwrite it in the following requests but client has to remember last agreed value - setOverWriteTimeout(0); + setOverWriteKeepAlive(0, 0); auto connection = pool->getConnection(timeouts); echoRequest("Hello", *connection); ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds()); @@ -819,3 +823,88 @@ TEST_F(ConnectionPoolTest, ServerOverwriteKeepAlive) ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); } + +TEST_F(ConnectionPoolTest, MaxRequests) +{ + auto ka = Poco::Timespan(30, 0); // 30 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + auto max_requests = 5; + timeouts.http_keep_alive_max_requests = max_requests; + + auto pool = getPool(); + auto metrics = pool->getMetrics(); + + for (int i = 1; i <= max_requests - 1; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds()); + ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests()); + ASSERT_EQ(i, connection->getKeepAliveRequest()); + } + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(max_requests-2, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count)); + + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds()); + ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests()); + ASSERT_EQ(max_requests, connection->getKeepAliveRequest()); + } + + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); +} + + +TEST_F(ConnectionPoolTest, ServerOverwriteMaxRequests) +{ + auto ka = Poco::Timespan(30, 0); // 30 seconds + timeouts.withHTTPKeepAliveTimeout(ka); + + auto pool = getPool(); + auto metrics = pool->getMetrics(); + + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds()); + ASSERT_EQ(1000, connection->getKeepAliveMaxRequests()); + ASSERT_EQ(1, connection->getKeepAliveRequest()); + } + + auto max_requests = 3; + setOverWriteKeepAlive(5, max_requests); + + for (int i = 2; i <= 10*max_requests; ++i) + { + auto connection = pool->getConnection(timeouts); + echoRequest("Hello", *connection); + ASSERT_EQ(5, connection->getKeepAliveTimeout().totalSeconds()); + ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests()); + ASSERT_EQ(((i-1) % max_requests) + 1, connection->getKeepAliveRequest()); + } + + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.created]); + ASSERT_EQ(10*max_requests-10, DB::CurrentThread::getProfileEvents()[metrics.preserved]); + ASSERT_EQ(10*max_requests-10, DB::CurrentThread::getProfileEvents()[metrics.reused]); + ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]); + ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.expired]); + + ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count)); + ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count)); +} diff --git a/src/Core/Defines.h b/src/Core/Defines.h index a8dd26519c2..f2142bc764d 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -54,6 +54,7 @@ static constexpr auto DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT = 15; static constexpr auto DEFAULT_TCP_KEEP_ALIVE_TIMEOUT = 290; static constexpr auto DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT = 30; +static constexpr auto DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST = 1000; static constexpr auto DBMS_DEFAULT_PATH = "/var/lib/clickhouse/"; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 7ce94699053..c3114eb0b6f 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -76,7 +76,8 @@ std::unique_ptr getClient( client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS); client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); - client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); + client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT); + client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); client_configuration.endpointOverride = uri.endpoint; client_configuration.s3_use_adaptive_timeouts = config.getBool( diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 8813c958185..da6214ae477 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -148,6 +148,7 @@ void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeout if (!session.connected()) { session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); + session.setKeepAliveMaxRequests(int(timeouts.http_keep_alive_max_requests)); } } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 49305f42d85..f497285bd0c 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -35,6 +35,8 @@ struct ConnectionTimeouts Poco::Timespan tcp_keep_alive_timeout = Poco::Timespan(DEFAULT_TCP_KEEP_ALIVE_TIMEOUT, 0); Poco::Timespan http_keep_alive_timeout = Poco::Timespan(DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, 0); + size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST; + /// Timeouts for HedgedConnections Poco::Timespan hedged_connection_timeout = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0); diff --git a/src/IO/S3/Credentials.h b/src/IO/S3/Credentials.h index 34dc0c1d2bd..8d586223035 100644 --- a/src/IO/S3/Credentials.h +++ b/src/IO/S3/Credentials.h @@ -22,6 +22,8 @@ inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120; inline static constexpr uint64_t DEFAULT_CONNECT_TIMEOUT_MS = 1000; inline static constexpr uint64_t DEFAULT_REQUEST_TIMEOUT_MS = 30000; inline static constexpr uint64_t DEFAULT_MAX_CONNECTIONS = 100; +inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_TIMEOUT = 5; +inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_MAX_REQUESTS = 100; /// In GCP metadata service can be accessed via DNS regardless of IPv4 or IPv6. static inline constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal"; diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index f568eb5ddb8..a0b35e9b4a9 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -52,6 +52,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration /// See PoolBase::BehaviourOnLimit bool s3_use_adaptive_timeouts = true; size_t http_keep_alive_timeout = DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT; + size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST; std::function error_report; From dddb0d9f4a83569e9a64952b20acfc95da2cdf24 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 5 Apr 2024 03:02:45 +0200 Subject: [PATCH 142/243] fix http_keep_alive_max_requests set up --- src/IO/ConnectionTimeouts.h | 7 +++++++ src/IO/S3/PocoHTTPClient.cpp | 3 ++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index f497285bd0c..b86ec44d21c 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -71,6 +71,7 @@ APPLY_FOR_ALL_CONNECTION_TIMEOUT_MEMBERS(DECLARE_BUILDER_FOR_MEMBER) ConnectionTimeouts & withConnectionTimeout(size_t seconds); ConnectionTimeouts & withConnectionTimeout(Poco::Timespan span); + ConnectionTimeouts & withHTTPKeepAliveMaxRequests(size_t requests); }; /// NOLINTBEGIN(bugprone-macro-parentheses) @@ -116,6 +117,12 @@ inline ConnectionTimeouts & ConnectionTimeouts::withConnectionTimeout(Poco::Time return *this; } +inline ConnectionTimeouts & ConnectionTimeouts::withHTTPKeepAliveMaxRequests(size_t requests) +{ + http_keep_alive_max_requests = requests; + return *this; +} + void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts); ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 150b8146147..de20a712d4c 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -147,7 +147,8 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio .withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000)) .withTCPKeepAliveTimeout(Poco::Timespan( client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0)) - .withHTTPKeepAliveTimeout(Poco::Timespan(client_configuration.http_keep_alive_timeout, 0)); + .withHTTPKeepAliveTimeout(Poco::Timespan(client_configuration.http_keep_alive_timeout, 0)) + .withHTTPKeepAliveMaxRequests(client_configuration.http_keep_alive_max_requests); } PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration) From cf982cc114ef5b226815360590e2c207516de658 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 5 Apr 2024 05:00:01 +0200 Subject: [PATCH 143/243] remove debug logging --- src/Common/HTTPConnectionPool.cpp | 76 ------------------------------- 1 file changed, 76 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 81c36dcd970..7f99d6a647f 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -193,18 +193,6 @@ public: return total_connections_in_group >= limits.store_limit; } - size_t getStored() const - { - std::lock_guard lock(mutex); - return total_connections_in_group; - } - - size_t getStoreLimit() const - { - std::lock_guard lock(mutex); - return limits.store_limit; - } - void atConnectionCreate() { std::lock_guard lock(mutex); @@ -359,33 +347,12 @@ private: Session::flushRequest(); } - String printAllHeaders(Poco::Net::HTTPMessage & message) const - { - String out; - out.reserve(300); - for (auto & [k, v] : message) - { - out.append(fmt::format("<{}: {}> ", k, v)); - } - return out; - } - std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override { auto idle = idleTime(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); - // that line is for temporary debug, will be removed - LOG_INFO(log, "Send request to {} with: version {}, method {}, request no {}, keep-alive timeout={}, last usage ago: {}ms, headers: {}", - request.getVersion(), - request.getMethod(), - getTarget(), - Session::getKeepAliveRequest(), - Session::getKeepAliveTimeout().totalSeconds(), - idle.totalMilliseconds(), - printAllHeaders(request)); - request_stream = &result; request_stream_completed = false; @@ -397,22 +364,9 @@ private: std::istream & receiveResponse(Poco::Net::HTTPResponse & response) override { - int originKA = Session::getKeepAliveTimeout().totalSeconds(); - std::istream & result = Session::receiveResponse(response); result.exceptions(std::ios::badbit); - // that line is for temporary debug, will be removed - LOG_INFO(log, "Received response from {} with: version {}, code {}, request no {}, keep alive header: {}, original ka {}, last usage ago: {}ms, headers: {}", - getTarget(), - response.getVersion(), - int(response.getStatus()), - Session::getKeepAliveRequest(), - response.get(Poco::Net::HTTPMessage::CONNECTION_KEEP_ALIVE, Poco::Net::HTTPMessage::EMPTY), - originKA, - idleTime().totalMilliseconds(), - printAllHeaders(response)); - response_stream = &result; response_stream_completed = false; @@ -456,19 +410,8 @@ private: group->atConnectionDestroy(); if (!isExpired) - { if (auto lock = pool.lock()) lock->atConnectionDestroy(*this); - } - else - { - Poco::Timestamp now; - LOG_INFO(log, "Expired connection to {} with: request no {}, keep alive timeout: {}, last usage ago: {}s", - getTarget(), - Session::getKeepAliveRequest(), - Session::getKeepAliveTimeout().totalSeconds(), - idleTime().totalSeconds()); - } CurrentMetrics::sub(metrics.active_count); } @@ -519,8 +462,6 @@ private: IHTTPConnectionPoolForEndpoint::Metrics metrics; bool isExpired = false; - size_t exception_level = std::uncaught_exceptions(); - LoggerPtr log = getLogger("PooledConnection"); std::ostream * request_stream = nullptr; @@ -701,9 +642,6 @@ private: { if (connection.getKeepAliveRequest() >= connection.getKeepAliveMaxRequests()) { - LOG_INFO(getLogger("PooledConnection"), "Expired by connection number {}", - connection.getKeepAliveRequest()); - ProfileEvents::increment(getMetrics().expired, 1); return; } @@ -711,19 +649,6 @@ private: if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered() || group->isStoreLimitReached()) { - Poco::Timestamp now; - LOG_INFO(getLogger("PooledConnection"), - "Reset connection to {} with: usage count {}, keep alive timeout: {}, connected {}, must recon {}, last usage ago: {}, is completed {}, store limit reached {} as {}/{}, there is exception {}", - getTarget(), - connection.getKeepAliveRequest(), - connection.getKeepAliveTimeout().totalSeconds(), - connection.connected(), - connection.mustReconnect(), - connection.idleTime().totalSeconds(), - connection.isCompleted(), - group->isStoreLimitReached(), group->getStored(), group->getStoreLimit(), - connection.exception_level - std::uncaught_exceptions()); - ProfileEvents::increment(getMetrics().reset, 1); return; } @@ -833,7 +758,6 @@ private: ConnectionGroup::Ptr storage_group = std::make_shared(HTTPConnectionGroupType::STORAGE); ConnectionGroup::Ptr http_group = std::make_shared(HTTPConnectionGroupType::HTTP); - /// If multiple mutexes are held simultaneously, /// they should be locked in this order: /// HTTPConnectionPools::mutex, then EndpointConnectionPool::mutex, then ConnectionGroup::mutex. From 81eda37f7f0a62cd1a4499c56a66daa7ef981827 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Apr 2024 10:27:13 +0200 Subject: [PATCH 144/243] Print correct count --- utils/postprocess-traces/postprocess-traces.pl | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/postprocess-traces/postprocess-traces.pl b/utils/postprocess-traces/postprocess-traces.pl index 3e50f64d864..1c198908580 100755 --- a/utils/postprocess-traces/postprocess-traces.pl +++ b/utils/postprocess-traces/postprocess-traces.pl @@ -13,9 +13,9 @@ sub process_stacktrace my $group = \$grouped_stacks; for my $frame (reverse @current_stack) { + $group = \$$group->{children}{$frame}; $$group->{count} ||= 0; ++$$group->{count}; - $group = \$$group->{children}{$frame}; } @current_stack = (); @@ -47,7 +47,7 @@ sub print_group for my $key (sort { $group->{children}{$b}{count} <=> $group->{children}{$a}{count} } keys %{$group->{children}}) { - my $count = $group->{count}; + my $count = $group->{children}{$key}{count}; print(('| ' x $level) . $count . (' ' x (5 - (length $count))) . $key . "\n"); print_group($group->{children}{$key}, $level + 1); } From 9d8f643f5b306ff02ed8e55dd776afb04e67de49 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 4 Apr 2024 20:58:35 +0000 Subject: [PATCH 145/243] Cleanup SSH-based authentication code --- contrib/libssh-cmake/CMakeLists.txt | 55 ++++++-------- programs/client/Client.cpp | 4 +- src/Access/Authentication.cpp | 37 ++++++--- src/Access/AuthenticationData.cpp | 13 ++-- src/Access/AuthenticationData.h | 14 +++- src/Access/Common/AuthenticationType.h | 4 +- src/Access/Credentials.h | 5 +- src/Access/User.cpp | 2 +- src/Access/UsersConfigAccessStorage.cpp | 8 +- src/CMakeLists.txt | 5 +- src/Client/Connection.cpp | 36 ++++----- src/Client/Connection.h | 12 +-- src/Client/ConnectionParameters.cpp | 11 ++- src/Client/ConnectionParameters.h | 7 +- src/Client/ConnectionPool.h | 2 +- .../{SSH/Wrappers.cpp => SSHWrapper.cpp} | 75 +++++++++---------- src/Common/{SSH/Wrappers.h => SSHWrapper.h} | 36 ++++----- src/Core/Protocol.h | 9 ++- src/Parsers/Access/ParserPublicSSHKey.cpp | 2 +- src/Server/TCPHandler.cpp | 50 ++++++------- src/Server/TCPHandler.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/02867_create_user_ssh.sql | 12 ++- 23 files changed, 205 insertions(+), 199 deletions(-) rename src/Common/{SSH/Wrappers.cpp => SSHWrapper.cpp} (66%) rename src/Common/{SSH/Wrappers.h => SSHWrapper.h} (73%) diff --git a/contrib/libssh-cmake/CMakeLists.txt b/contrib/libssh-cmake/CMakeLists.txt index 7b589718140..ecd1fccb800 100644 --- a/contrib/libssh-cmake/CMakeLists.txt +++ b/contrib/libssh-cmake/CMakeLists.txt @@ -1,26 +1,18 @@ -option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES}) +option (ENABLE_SSH "Enable support for libssh" ${ENABLE_LIBRARIES}) if (NOT ENABLE_SSH) - message(STATUS "Not using SSH") + message(STATUS "Not using libssh") return() endif() +# CMake variables needed by libssh_version.h.cmake, update them when you update libssh +set(libssh_VERSION_MAJOR 0) +set(libssh_VERSION_MINOR 9) +set(libssh_VERSION_PATCH 8) + set(LIB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libssh") set(LIB_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/libssh") -# Set CMake variables which are used in libssh_version.h.cmake -project(libssh VERSION 0.9.8 LANGUAGES C) - -set(LIBRARY_VERSION "4.8.8") -set(LIBRARY_SOVERSION "4") - -set(CMAKE_THREAD_PREFER_PTHREADS ON) -set(THREADS_PREFER_PTHREAD_FLAG ON) - -set(WITH_ZLIB OFF) -set(WITH_SYMBOL_VERSIONING OFF) -set(WITH_SERVER ON) - set(libssh_SRCS ${LIB_SOURCE_DIR}/src/agent.c ${LIB_SOURCE_DIR}/src/auth.c @@ -28,15 +20,21 @@ set(libssh_SRCS ${LIB_SOURCE_DIR}/src/bignum.c ${LIB_SOURCE_DIR}/src/buffer.c ${LIB_SOURCE_DIR}/src/callbacks.c + ${LIB_SOURCE_DIR}/src/chachapoly.c ${LIB_SOURCE_DIR}/src/channels.c ${LIB_SOURCE_DIR}/src/client.c ${LIB_SOURCE_DIR}/src/config.c + ${LIB_SOURCE_DIR}/src/config_parser.c ${LIB_SOURCE_DIR}/src/connect.c ${LIB_SOURCE_DIR}/src/connector.c ${LIB_SOURCE_DIR}/src/curve25519.c ${LIB_SOURCE_DIR}/src/dh.c ${LIB_SOURCE_DIR}/src/ecdh.c ${LIB_SOURCE_DIR}/src/error.c + ${LIB_SOURCE_DIR}/src/external/bcrypt_pbkdf.c + ${LIB_SOURCE_DIR}/src/external/blowfish.c + ${LIB_SOURCE_DIR}/src/external/chacha.c + ${LIB_SOURCE_DIR}/src/external/poly1305.c ${LIB_SOURCE_DIR}/src/getpass.c ${LIB_SOURCE_DIR}/src/init.c ${LIB_SOURCE_DIR}/src/kdf.c @@ -55,37 +53,32 @@ set(libssh_SRCS ${LIB_SOURCE_DIR}/src/pcap.c ${LIB_SOURCE_DIR}/src/pki.c ${LIB_SOURCE_DIR}/src/pki_container_openssh.c + ${LIB_SOURCE_DIR}/src/pki_ed25519_common.c ${LIB_SOURCE_DIR}/src/poll.c - ${LIB_SOURCE_DIR}/src/session.c ${LIB_SOURCE_DIR}/src/scp.c + ${LIB_SOURCE_DIR}/src/session.c ${LIB_SOURCE_DIR}/src/socket.c ${LIB_SOURCE_DIR}/src/string.c ${LIB_SOURCE_DIR}/src/threads.c - ${LIB_SOURCE_DIR}/src/wrapper.c - ${LIB_SOURCE_DIR}/src/external/bcrypt_pbkdf.c - ${LIB_SOURCE_DIR}/src/external/blowfish.c - ${LIB_SOURCE_DIR}/src/external/chacha.c - ${LIB_SOURCE_DIR}/src/external/poly1305.c - ${LIB_SOURCE_DIR}/src/chachapoly.c - ${LIB_SOURCE_DIR}/src/config_parser.c ${LIB_SOURCE_DIR}/src/token.c - ${LIB_SOURCE_DIR}/src/pki_ed25519_common.c + ${LIB_SOURCE_DIR}/src/wrapper.c + # some files of libssh/src/ are missing - why? ${LIB_SOURCE_DIR}/src/threads/noop.c ${LIB_SOURCE_DIR}/src/threads/pthread.c + # files missing - why? # LIBCRYPT specific - ${libssh_SRCS} - ${LIB_SOURCE_DIR}/src/threads/libcrypto.c - ${LIB_SOURCE_DIR}/src/pki_crypto.c + ${LIB_SOURCE_DIR}/src/dh_crypto.c ${LIB_SOURCE_DIR}/src/ecdh_crypto.c ${LIB_SOURCE_DIR}/src/libcrypto.c - ${LIB_SOURCE_DIR}/src/dh_crypto.c + ${LIB_SOURCE_DIR}/src/pki_crypto.c + ${LIB_SOURCE_DIR}/src/threads/libcrypto.c - ${LIB_SOURCE_DIR}/src/options.c - ${LIB_SOURCE_DIR}/src/server.c ${LIB_SOURCE_DIR}/src/bind.c ${LIB_SOURCE_DIR}/src/bind_config.c + ${LIB_SOURCE_DIR}/src/options.c + ${LIB_SOURCE_DIR}/src/server.c ) if (NOT (ENABLE_OPENSSL OR ENABLE_OPENSSL_DYNAMIC)) @@ -94,7 +87,7 @@ endif() configure_file(${LIB_SOURCE_DIR}/include/libssh/libssh_version.h.cmake ${LIB_BINARY_DIR}/include/libssh/libssh_version.h @ONLY) -add_library(_ssh STATIC ${libssh_SRCS}) +add_library(_ssh ${libssh_SRCS}) add_library(ch_contrib::ssh ALIAS _ssh) target_link_libraries(_ssh PRIVATE OpenSSL::Crypto) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 192f9e61891..72cad1dac07 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -934,8 +934,8 @@ void Client::addOptions(OptionsDescription & options_description) ("user,u", po::value()->default_value("default"), "user") ("password", po::value(), "password") ("ask-password", "ask-password") - ("ssh-key-file", po::value(), "File containing ssh private key needed for authentication. If not set does password authentication.") - ("ssh-key-passphrase", po::value(), "Passphrase for imported ssh key.") + ("ssh-key-file", po::value(), "File containing the SSH private key for authenticate with the server.") + ("ssh-key-passphrase", po::value(), "Passphrase for the SSH private key specified by --ssh-key-file.") ("quota_key", po::value(), "A string to differentiate quotas when the user have keyed quotas configured on server") ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 47187d83154..bf1fe3feec3 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -4,11 +4,12 @@ #include #include #include -#include #include +#include +#include #include -#include +#include "config.h" namespace DB { @@ -74,7 +75,7 @@ namespace } #if USE_SSH - bool checkSshSignature(const std::vector & keys, std::string_view signature, std::string_view original) + bool checkSshSignature(const std::vector & keys, std::string_view signature, std::string_view original) { for (const auto & key: keys) if (key.isPublic() && key.verifySignature(signature, original)) @@ -114,7 +115,11 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: - throw Authentication::Require("Ssh Keys Authentication"); +#if USE_SSH + throw Authentication::Require("SSH Keys Authentication"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); +#endif case AuthenticationType::MAX: break; @@ -145,7 +150,11 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: - throw Authentication::Require("Ssh Keys Authentication"); +#if USE_SSH + throw Authentication::Require("SSH Keys Authentication"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); +#endif case AuthenticationType::MAX: break; @@ -178,7 +187,11 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: - throw Authentication::Require("Ssh Keys Authentication"); +#if USE_SSH + throw Authentication::Require("SSH Keys Authentication"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); +#endif case AuthenticationType::BCRYPT_PASSWORD: return checkPasswordBcrypt(basic_credentials->getPassword(), auth_data.getPasswordHashBinary()); @@ -216,13 +229,18 @@ bool Authentication::areCredentialsValid( return auth_data.getSSLCertificateCommonNames().contains(ssl_certificate_credentials->getCommonName()); case AuthenticationType::SSH_KEY: - throw Authentication::Require("Ssh Keys Authentication"); +#if USE_SSH + throw Authentication::Require("SSH Keys Authentication"); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); +#endif case AuthenticationType::MAX: break; } } +#if USE_SSH if (const auto * ssh_credentials = typeid_cast(&credentials)) { switch (auth_data.getType()) @@ -243,15 +261,12 @@ bool Authentication::areCredentialsValid( throw Authentication::Require("ClickHouse X.509 Authentication"); case AuthenticationType::SSH_KEY: -#if USE_SSH return checkSshSignature(auth_data.getSSHKeys(), ssh_credentials->getSignature(), ssh_credentials->getOriginal()); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); -#endif case AuthenticationType::MAX: break; } } +#endif if ([[maybe_unused]] const auto * always_allow_credentials = typeid_cast(&credentials)) return true; diff --git a/src/Access/AuthenticationData.cpp b/src/Access/AuthenticationData.cpp index da90a0f5842..a4c25b438e8 100644 --- a/src/Access/AuthenticationData.cpp +++ b/src/Access/AuthenticationData.cpp @@ -105,7 +105,10 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash) && (lhs.ldap_server_name == rhs.ldap_server_name) && (lhs.kerberos_realm == rhs.kerberos_realm) && (lhs.ssl_certificate_common_names == rhs.ssl_certificate_common_names) - && (lhs.ssh_keys == rhs.ssh_keys) && (lhs.http_auth_scheme == rhs.http_auth_scheme) +#if USE_SSH + && (lhs.ssh_keys == rhs.ssh_keys) +#endif + && (lhs.http_auth_scheme == rhs.http_auth_scheme) && (lhs.http_auth_server_name == rhs.http_auth_server_name); } @@ -326,7 +329,7 @@ std::shared_ptr AuthenticationData::toAST() const break; #else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); #endif } case AuthenticationType::HTTP: @@ -355,7 +358,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que { #if USE_SSH AuthenticationData auth_data(*query.type); - std::vector keys; + std::vector keys; size_t args_size = query.children.size(); for (size_t i = 0; i < args_size; ++i) @@ -366,7 +369,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que try { - keys.emplace_back(ssh::SSHKeyFactory::makePublicFromBase64(key_base64, type)); + keys.emplace_back(SSHKeyFactory::makePublicKeyFromBase64(key_base64, type)); } catch (const std::invalid_argument &) { @@ -377,7 +380,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que auth_data.setSSHKeys(std::move(keys)); return auth_data; #else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); #endif } diff --git a/src/Access/AuthenticationData.h b/src/Access/AuthenticationData.h index feef4d71d66..c97e0327b56 100644 --- a/src/Access/AuthenticationData.h +++ b/src/Access/AuthenticationData.h @@ -2,14 +2,16 @@ #include #include +#include #include #include -#include #include #include #include +#include "config.h" + namespace DB { @@ -59,8 +61,10 @@ public: const boost::container::flat_set & getSSLCertificateCommonNames() const { return ssl_certificate_common_names; } void setSSLCertificateCommonNames(boost::container::flat_set common_names_); - const std::vector & getSSHKeys() const { return ssh_keys; } - void setSSHKeys(std::vector && ssh_keys_) { ssh_keys = std::forward>(ssh_keys_); } +#if USE_SSH + const std::vector & getSSHKeys() const { return ssh_keys; } + void setSSHKeys(std::vector && ssh_keys_) { ssh_keys = std::forward>(ssh_keys_); } +#endif HTTPAuthenticationScheme getHTTPAuthenticationScheme() const { return http_auth_scheme; } void setHTTPAuthenticationScheme(HTTPAuthenticationScheme scheme) { http_auth_scheme = scheme; } @@ -94,7 +98,9 @@ private: String kerberos_realm; boost::container::flat_set ssl_certificate_common_names; String salt; - std::vector ssh_keys; +#if USE_SSH + std::vector ssh_keys; +#endif /// HTTP authentication properties String http_auth_server_name; HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC; diff --git a/src/Access/Common/AuthenticationType.h b/src/Access/Common/AuthenticationType.h index 48ace3ca00a..506c8abd3b1 100644 --- a/src/Access/Common/AuthenticationType.h +++ b/src/Access/Common/AuthenticationType.h @@ -34,8 +34,8 @@ enum class AuthenticationType /// Password is encrypted in bcrypt hash. BCRYPT_PASSWORD, - /// Server sends a random string named `challenge` which client needs to encrypt with private key. - /// The check is performed on server side by decrypting the data and comparing with the original string. + /// Server sends a random string named `challenge` to the client. The client encrypts it with its SSH private key. + /// The server decrypts the result using the SSH public key registered for the user and compares with the original string. SSH_KEY, /// Authentication through HTTP protocol diff --git a/src/Access/Credentials.h b/src/Access/Credentials.h index 77b90eaaebc..d04f8a66541 100644 --- a/src/Access/Credentials.h +++ b/src/Access/Credentials.h @@ -3,6 +3,7 @@ #include #include +#include "config.h" namespace DB { @@ -86,10 +87,11 @@ class MySQLNative41Credentials : public CredentialsWithScramble using CredentialsWithScramble::CredentialsWithScramble; }; +#if USE_SSH class SshCredentials : public Credentials { public: - explicit SshCredentials(const String& user_name_, const String& signature_, const String& original_) + SshCredentials(const String & user_name_, const String & signature_, const String & original_) : Credentials(user_name_), signature(signature_), original(original_) { is_ready = true; @@ -117,5 +119,6 @@ private: String signature; String original; }; +#endif } diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 39930c9cf76..ef5cf722113 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -31,7 +31,7 @@ void User::setName(const String & name_) throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name is empty"); if (name_ == EncodedUserInfo::USER_INTERSERVER_MARKER) throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name '{}' is reserved", name_); - if (startsWith(name_, EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER)) + if (name_.starts_with(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name '{}' is reserved", name_); name = name_; } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index b4b843fc77e..e3c45eb45ae 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -10,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -214,7 +214,7 @@ namespace Poco::Util::AbstractConfiguration::Keys entries; config.keys(ssh_keys_config, entries); - std::vector keys; + std::vector keys; for (const String& entry : entries) { const auto conf_pref = ssh_keys_config + "." + entry + "."; @@ -237,7 +237,7 @@ namespace try { - keys.emplace_back(ssh::SSHKeyFactory::makePublicFromBase64(base64_key, type)); + keys.emplace_back(SSHKeyFactory::makePublicKeyFromBase64(base64_key, type)); } catch (const std::invalid_argument &) { @@ -249,7 +249,7 @@ namespace } user->auth_data.setSSHKeys(std::move(keys)); #else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); #endif } else if (has_http_auth) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 73aa409e995..da17bc1f41f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -85,7 +85,6 @@ add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io Common/Scheduler) add_headers_and_sources(clickhouse_common_io Common/Scheduler/Nodes) -add_headers_and_sources(clickhouse_common_io Common/SSH) add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/S3) @@ -99,7 +98,6 @@ add_headers_and_sources(clickhouse_compression Core) #Included these specific files to avoid linking grpc add_glob(clickhouse_compression_headers Server/ServerType.h) add_glob(clickhouse_compression_sources Server/ServerType.cpp) -add_headers_and_sources(clickhouse_compression Common/SSH) add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) @@ -370,8 +368,7 @@ if (TARGET ch_contrib::crc32-vpmsum) endif() if (TARGET ch_contrib::ssh) - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh) - target_link_libraries(clickhouse_compression PUBLIC ch_contrib::ssh) + target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh) endif() dbms_target_link_libraries(PUBLIC ch_contrib::abseil_swiss_tables) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 180942e6b83..5a1d7a2acc4 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -67,7 +67,7 @@ Connection::~Connection() = default; Connection::Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, - const ssh::SSHKey & ssh_private_key_, + [[maybe_unused]] const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, const String & cluster_secret_, @@ -76,7 +76,9 @@ Connection::Connection(const String & host_, UInt16 port_, Protocol::Secure secure_) : host(host_), port(port_), default_database(default_database_) , user(user_), password(password_) +#if USE_SSH , ssh_private_key(ssh_private_key_) +#endif , quota_key(quota_key_) , cluster(cluster_) , cluster_secret(cluster_secret_) @@ -276,17 +278,6 @@ void Connection::disconnect() } -String Connection::packStringForSshSign(String challenge) -{ - String message; - message.append(std::to_string(DBMS_TCP_PROTOCOL_VERSION)); - message.append(default_database); - message.append(user); - message.append(challenge); - return message; -} - - void Connection::sendHello() { /** Disallow control characters in user controlled parameters @@ -334,10 +325,10 @@ void Connection::sendHello() #endif } #if USE_SSH - /// Just inform server that we will authenticate using SSH keys. else if (!ssh_private_key.isEmpty()) { - writeStringBinary(fmt::format("{}{}", EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER, user), *out); + /// Inform server that we will authenticate using SSH keys. + writeStringBinary(String(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER) + user, *out); writeStringBinary(password, *out); performHandshakeForSSHAuth(); @@ -361,9 +352,9 @@ void Connection::sendAddendum() } +#if USE_SSH void Connection::performHandshakeForSSHAuth() { -#if USE_SSH String challenge; { writeVarUInt(Protocol::Client::SSHChallengeRequest, *out); @@ -388,12 +379,23 @@ void Connection::performHandshakeForSSHAuth() } writeVarUInt(Protocol::Client::SSHChallengeResponse, *out); - String to_sign = packStringForSshSign(challenge); + + auto pack_string_for_ssh_sign = [&](String challenge_) + { + String message; + message.append(std::to_string(DBMS_TCP_PROTOCOL_VERSION)); + message.append(default_database); + message.append(user); + message.append(challenge_); + return message; + }; + + String to_sign = pack_string_for_ssh_sign(challenge); String signature = ssh_private_key.signString(to_sign); writeStringBinary(signature, *out); out->next(); -#endif } +#endif void Connection::receiveHello(const Poco::Timespan & handshake_timeout) diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 5d0411027a1..2cd325afed2 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -1,10 +1,9 @@ #pragma once - #include -#include #include +#include #include #include @@ -53,7 +52,7 @@ public: Connection(const String & host_, UInt16 port_, const String & default_database_, const String & user_, const String & password_, - const ssh::SSHKey & ssh_private_key_, + const SSHKey & ssh_private_key_, const String & quota_key_, const String & cluster_, const String & cluster_secret_, @@ -170,7 +169,9 @@ private: String default_database; String user; String password; - ssh::SSHKey ssh_private_key; +#if USE_SSH + SSHKey ssh_private_key; +#endif String quota_key; /// For inter-server authorization @@ -265,9 +266,10 @@ private: void connect(const ConnectionTimeouts & timeouts); void sendHello(); - String packStringForSshSign(String challenge); +#if USE_SSH void performHandshakeForSSHAuth(); +#endif void sendAddendum(); void receiveHello(const Poco::Timespan & handshake_timeout); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 16911f97e84..774f3375f63 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -1,11 +1,10 @@ #include "ConnectionParameters.h" -#include + #include #include #include #include #include -#include #include #include #include @@ -88,19 +87,19 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati } else { - std::string prompt{"Enter your private key passphrase (leave empty for no passphrase): "}; + std::string prompt{"Enter your SSH private key passphrase (leave empty for no passphrase): "}; char buf[1000] = {}; if (auto * result = readpassphrase(prompt.c_str(), buf, sizeof(buf), 0)) passphrase = result; } - ssh::SSHKey key = ssh::SSHKeyFactory::makePrivateFromFile(filename, passphrase); + SSHKey key = SSHKeyFactory::makePrivateKeyFromFile(filename, passphrase); if (!key.isPrivate()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found public key in file: {} but expected private", filename); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} did not contain a private key (is it a public key?)", filename); ssh_private_key = std::move(key); #else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh"); #endif } diff --git a/src/Client/ConnectionParameters.h b/src/Client/ConnectionParameters.h index 5f375f09c83..f23522d48b3 100644 --- a/src/Client/ConnectionParameters.h +++ b/src/Client/ConnectionParameters.h @@ -1,9 +1,10 @@ #pragma once -#include +#include #include #include -#include + +#include namespace Poco::Util { @@ -20,7 +21,7 @@ struct ConnectionParameters std::string user; std::string password; std::string quota_key; - ssh::SSHKey ssh_private_key; + SSHKey ssh_private_key; Protocol::Secure security = Protocol::Secure::Disable; Protocol::Compression compression = Protocol::Compression::Enable; ConnectionTimeouts timeouts; diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 574c4992d75..d35c2552461 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -123,7 +123,7 @@ protected: { return std::make_shared( host, port, - default_database, user, password, ssh::SSHKey(), quota_key, + default_database, user, password, SSHKey(), quota_key, cluster, cluster_secret, client_name, compression, secure); } diff --git a/src/Common/SSH/Wrappers.cpp b/src/Common/SSHWrapper.cpp similarity index 66% rename from src/Common/SSH/Wrappers.cpp rename to src/Common/SSHWrapper.cpp index a9b9f758c6e..0ed266f215c 100644 --- a/src/Common/SSH/Wrappers.cpp +++ b/src/Common/SSHWrapper.cpp @@ -1,4 +1,5 @@ -#include +#include + # if USE_SSH # include @@ -10,6 +11,14 @@ # pragma clang diagnostic pop +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LIBSSH_ERROR; +} + namespace { @@ -18,17 +27,19 @@ class SSHString public: explicit SSHString(std::string_view input) { - string = ssh_string_new(input.size()); - ssh_string_fill(string, input.data(), input.size()); + if (string = ssh_string_new(input.size()); string == nullptr) + throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString"); + if (int rc = ssh_string_fill(string, input.data(), input.size()); rc != SSH_OK) + throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString"); } - explicit SSHString(ssh_string c_other) { string = c_other; } + explicit SSHString(ssh_string other) { string = other; } ssh_string get() { return string; } String toString() { - return String(ssh_string_get_char(string), ssh_string_len(string)); + return {ssh_string_get_char(string), ssh_string_len(string)}; } ~SSHString() @@ -42,46 +53,28 @@ private: } -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LIBSSH_ERROR; -} - -namespace ssh -{ - -SSHKey SSHKeyFactory::makePrivateFromFile(String filename, String passphrase) +SSHKey SSHKeyFactory::makePrivateKeyFromFile(String filename, String passphrase) { ssh_key key; - int rc = ssh_pki_import_privkey_file(filename.c_str(), passphrase.c_str(), nullptr, nullptr, &key); - if (rc != SSH_OK) - { + if (int rc = ssh_pki_import_privkey_file(filename.c_str(), passphrase.c_str(), nullptr, nullptr, &key); rc != SSH_OK) throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't import SSH private key from file"); - } return SSHKey(key); } -SSHKey SSHKeyFactory::makePublicFromFile(String filename) +SSHKey SSHKeyFactory::makePublicKeyFromFile(String filename) { ssh_key key; - int rc = ssh_pki_import_pubkey_file(filename.c_str(), &key); - if (rc != SSH_OK) + if (int rc = ssh_pki_import_pubkey_file(filename.c_str(), &key); rc != SSH_OK) throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't import SSH public key from file"); - return SSHKey(key); } -SSHKey SSHKeyFactory::makePublicFromBase64(String base64_key, String type_name) +SSHKey SSHKeyFactory::makePublicKeyFromBase64(String base64_key, String type_name) { ssh_key key; auto key_type = ssh_key_type_from_name(type_name.c_str()); - int rc = ssh_pki_import_pubkey_base64(base64_key.c_str(), key_type, &key); - if (rc != SSH_OK) + if (int rc = ssh_pki_import_pubkey_base64(base64_key.c_str(), key_type, &key); rc != SSH_OK) throw Exception(ErrorCodes::LIBSSH_ERROR, "Bad SSH public key provided"); - return SSHKey(key); } @@ -90,6 +83,12 @@ SSHKey::SSHKey(const SSHKey & other) key = ssh_key_dup(other.key); } +SSHKey::SSHKey(SSHKey && other) noexcept +{ + key = other.key; + other.key = nullptr; +} + SSHKey & SSHKey::operator=(const SSHKey & other) { ssh_key_free(key); @@ -119,13 +118,11 @@ bool SSHKey::isEqual(const SSHKey & other) const String SSHKey::signString(std::string_view input) const { SSHString input_str(input); - ssh_string c_output = nullptr; - int rc = pki_sign_string(key, input_str.get(), &c_output); - if (rc != SSH_OK) + ssh_string output = nullptr; + if (int rc = pki_sign_string(key, input_str.get(), &output); rc != SSH_OK) throw Exception(ErrorCodes::LIBSSH_ERROR, "Error singing with ssh key"); - - SSHString output(c_output); - return output.toString(); + SSHString output_str(output); + return output_str.toString(); } bool SSHKey::verifySignature(std::string_view signature, std::string_view original) const @@ -149,18 +146,15 @@ namespace { struct CStringDeleter { - [[maybe_unused]] void operator()(char * ptr) const { std::free(ptr); } + void operator()(char * ptr) const { std::free(ptr); } }; } String SSHKey::getBase64() const { char * buf = nullptr; - int rc = ssh_pki_export_pubkey_base64(key, &buf); - - if (rc != SSH_OK) + if (int rc = ssh_pki_export_pubkey_base64(key, &buf); rc != SSH_OK) throw DB::Exception(DB::ErrorCodes::LIBSSH_ERROR, "Failed to export public key to base64"); - /// Create a String from cstring, which makes a copy of the first one and requires freeing memory after it /// This is to safely manage buf memory std::unique_ptr buf_ptr(buf); @@ -177,7 +171,6 @@ SSHKey::~SSHKey() ssh_key_free(key); // it's safe free from libssh } -} } #endif diff --git a/src/Common/SSH/Wrappers.h b/src/Common/SSHWrapper.h similarity index 73% rename from src/Common/SSH/Wrappers.h rename to src/Common/SSHWrapper.h index 699bba2b042..b6f0c577edc 100644 --- a/src/Common/SSH/Wrappers.h +++ b/src/Common/SSHWrapper.h @@ -1,20 +1,18 @@ #pragma once -#include -#include "config.h" -#if USE_SSH -# include -# include +#include + +#include +#include + +#include "config.h" + +#if USE_SSH using ssh_key = struct ssh_key_struct *; namespace DB { -namespace ssh -{ - -class SSHKeyFactory; - class SSHKey { public: @@ -22,11 +20,7 @@ public: ~SSHKey(); SSHKey(const SSHKey & other); - SSHKey(SSHKey && other) noexcept - { - key = other.key; - other.key = nullptr; - } + SSHKey(SSHKey && other) noexcept; SSHKey & operator=(const SSHKey & other); SSHKey & operator=(SSHKey && other) noexcept; @@ -43,7 +37,7 @@ public: String getBase64() const; String getKeyType() const; - friend SSHKeyFactory; + friend class SSHKeyFactory; private: explicit SSHKey(ssh_key key_) : key(key_) { } ssh_key key = nullptr; @@ -56,17 +50,14 @@ public: /// The check whether the path is allowed to read for ClickHouse has /// (e.g. a file is inside `user_files` directory) /// to be done outside of this functions. - static SSHKey makePrivateFromFile(String filename, String passphrase); - static SSHKey makePublicFromFile(String filename); - static SSHKey makePublicFromBase64(String base64_key, String type_name); + static SSHKey makePrivateKeyFromFile(String filename, String passphrase); + static SSHKey makePublicKeyFromFile(String filename); + static SSHKey makePublicKeyFromBase64(String base64_key, String type_name); }; -} } #else -namespace ssh -{ class SSHKey { public: @@ -74,5 +65,4 @@ public: [[ noreturn ]] bool isEmpty() { std::terminate(); } [[ noreturn ]] String signString(std::string_view) const { std::terminate(); } }; -} #endif diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 441e22f4a16..48107154753 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -56,10 +56,11 @@ namespace DB namespace EncodedUserInfo { -/// Marker of the inter-server secret (passed in the user name) +/// Marker for the inter-server secret (passed as the user name) /// (anyway user cannot be started with a whitespace) const char USER_INTERSERVER_MARKER[] = " INTERSERVER SECRET "; -/// Marker of the SSH keys based authentication (passed in the user name) + +/// Marker for SSH-keys-based authentication (passed as the user name) const char SSH_KEY_AUTHENTICAION_MARKER[] = " SSH KEY AUTHENTICATION "; }; @@ -160,8 +161,8 @@ namespace Protocol ReadTaskResponse = 9, /// A filename to read from s3 (used in s3Cluster) MergeTreeReadTaskResponse = 10, /// Coordinator's decision with a modified set of mark ranges allowed to read - SSHChallengeRequest = 11, /// Request for SSH signature challenge - SSHChallengeResponse = 12, /// Request for SSH signature challenge + SSHChallengeRequest = 11, /// Request SSH signature challenge + SSHChallengeResponse = 12, /// Reply to SSH signature challenge MAX = SSHChallengeResponse, }; diff --git a/src/Parsers/Access/ParserPublicSSHKey.cpp b/src/Parsers/Access/ParserPublicSSHKey.cpp index bc033e25bbb..9102044900d 100644 --- a/src/Parsers/Access/ParserPublicSSHKey.cpp +++ b/src/Parsers/Access/ParserPublicSSHKey.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5c08c697434..4e3d6ab69f6 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1371,17 +1371,6 @@ std::string formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(const Poco::Ut return result; } -[[ maybe_unused ]] String createChallenge() -{ -#if USE_SSL - pcg64_fast rng(randomSeed()); - UInt64 rand = rng(); - return encodeSHA256(&rand, sizeof(rand)); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Can't generate challenge, because ClickHouse was built without OpenSSL"); -#endif -} - } std::unique_ptr TCPHandler::makeSession() @@ -1399,16 +1388,6 @@ std::unique_ptr TCPHandler::makeSession() return res; } -String TCPHandler::prepareStringForSshValidation(String username, String challenge) -{ - String output; - output.append(std::to_string(client_tcp_protocol_version)); - output.append(default_database); - output.append(username); - output.append(challenge); - return output; -} - void TCPHandler::receiveHello() { /// Receive `hello` packet. @@ -1466,11 +1445,9 @@ void TCPHandler::receiveHello() return; } - is_ssh_based_auth = startsWith(user, EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER) && password.empty(); + is_ssh_based_auth = user.starts_with(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER) && password.empty(); if (is_ssh_based_auth) - { - user.erase(0, String(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER).size()); - } + user.erase(0, std::string_view(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER).size()); session = makeSession(); const auto & client_info = session->getClientInfo(); @@ -1498,7 +1475,9 @@ void TCPHandler::receiveHello() } } } +#endif +#if USE_SSH /// Perform handshake for SSH authentication if (is_ssh_based_auth) { @@ -1512,7 +1491,14 @@ void TCPHandler::receiveHello() if (packet_type != Protocol::Client::SSHChallengeRequest) throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Server expected to receive a packet for requesting a challenge string"); - auto challenge = createChallenge(); + auto create_challenge = []() + { + pcg64_fast rng(randomSeed()); + UInt64 rand = rng(); + return encodeSHA256(&rand, sizeof(rand)); + }; + + String challenge = create_challenge(); writeVarUInt(Protocol::Server::SSHChallenge, *out); writeStringBinary(challenge, *out); out->next(); @@ -1523,7 +1509,17 @@ void TCPHandler::receiveHello() throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Server expected to receive a packet with a response for a challenge"); readStringBinary(signature, *in); - auto cred = SshCredentials(user, signature, prepareStringForSshValidation(user, challenge)); + auto prepare_string_for_ssh_validation = [&](const String & username, const String & challenge_) + { + String output; + output.append(std::to_string(client_tcp_protocol_version)); + output.append(default_database); + output.append(username); + output.append(challenge_); + return output; + }; + + auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge)); session->authenticate(cred, getClientAddress(client_info)); return; } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 28259d3a325..191617f1905 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -216,7 +216,7 @@ private: String default_database; - bool is_ssh_based_auth = false; + bool is_ssh_based_auth = false; /// authentication is via SSH pub-key challenge /// For inter-server secret (remote_server.*.secret) bool is_interserver_mode = false; bool is_interserver_authenticated = false; @@ -248,7 +248,6 @@ private: void extractConnectionSettingsFromContext(const ContextPtr & context); std::unique_ptr makeSession(); - String prepareStringForSshValidation(String user, String challenge); bool receiveProxyHeader(); void receiveHello(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c4b84a0ae8c..29ebd114b9c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5656,7 +5656,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu { auto connection = std::make_shared( node.host_name, node.port, query_context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, ssh::SSHKey(), node.quota_key, node.cluster, node.cluster_secret, + node.user, node.password, SSHKey(), node.quota_key, node.cluster, node.cluster_secret, "ParallelInsertSelectInititiator", node.compression, node.secure diff --git a/tests/queries/0_stateless/02867_create_user_ssh.sql b/tests/queries/0_stateless/02867_create_user_ssh.sql index 08236bdbcfe..3e3cb30a601 100644 --- a/tests/queries/0_stateless/02867_create_user_ssh.sql +++ b/tests/queries/0_stateless/02867_create_user_ssh.sql @@ -1,10 +1,16 @@ -- Tags: no-fasttest, no-parallel +-- Tests user authentication with SSH public keys + DROP USER IF EXISTS test_user_02867; -CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'clickhouse' TYPE 'ssh-rsa'; -- { serverError LIBSSH_ERROR } -CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'clickhouse' TYPE 'clickhouse'; -- { serverError LIBSSH_ERROR } -CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'key1' TYPE 'ssh-rsa', KEY 'key2' TYPE 'ssh-rsa'; -- { serverError LIBSSH_ERROR } +-- negative tests +CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'invalid_key' TYPE 'ssh-rsa'; -- { serverError LIBSSH_ERROR } +CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'invalid_key' TYPE 'ssh-rsa', KEY 'invalid_key' TYPE 'ssh-rsa'; -- { serverError LIBSSH_ERROR } +CREATE USER test_user_02867 IDENTIFIED WITH ssh_key +BY KEY 'AAAAB3NzaC1yc2EAAAADAQABAAABgQCVTUso7/LQcBljfsHwyuL6fWfIvS3BaVpYB8lwf/ZylSOltBy6YlABtTU3mIb197d2DW99RcLKk174f5Zj5rUukXbV0fnufWvwd37fbb1eKM8zxBYvXs53EI5QBPZgKACIzMpYYZeJnAP0oZhUfWWtKXpy/SQ5CHiEIGD9RNYDL+uXZejMwC5r/+f2AmrATBo+Y+WJFZIvhj4uznFYvyvNTUz/YDvZCk+vwwIgiv4BpFCaZm2TeETTj6SvK567bZznLP5HXrkVbB5lhxjAkahc2w/Yjm//Fwto3xsMoJwROxJEU8L1kZ40QWPqjo7Tmr6C/hL2cKDNgWOEqrjLKQmh576s1+PfxwXpVPjLK4PHVSvuJLV88sn0iPdspLlKlDCdc7T9MqIrjJfxuhqnaoFQ7U+oBte8vkm1wGu76+WEC3iNWVAiIVZxLx9rUEsDqj3OovqfLiRsTmNLeY94p2asZjkx7rU48ZwuYN5XGafYsArPscj9Ve6RoRrof+5Q7cc=' +TYPE 'invalid_algorithm'; -- { serverError LIBSSH_ERROR } + CREATE USER test_user_02867 IDENTIFIED WITH ssh_key BY KEY 'AAAAB3NzaC1yc2EAAAADAQABAAABgQCVTUso7/LQcBljfsHwyuL6fWfIvS3BaVpYB8lwf/ZylSOltBy6YlABtTU3mIb197d2DW99RcLKk174f5Zj5rUukXbV0fnufWvwd37fbb1eKM8zxBYvXs53EI5QBPZgKACIzMpYYZeJnAP0oZhUfWWtKXpy/SQ5CHiEIGD9RNYDL+uXZejMwC5r/+f2AmrATBo+Y+WJFZIvhj4uznFYvyvNTUz/YDvZCk+vwwIgiv4BpFCaZm2TeETTj6SvK567bZznLP5HXrkVbB5lhxjAkahc2w/Yjm//Fwto3xsMoJwROxJEU8L1kZ40QWPqjo7Tmr6C/hL2cKDNgWOEqrjLKQmh576s1+PfxwXpVPjLK4PHVSvuJLV88sn0iPdspLlKlDCdc7T9MqIrjJfxuhqnaoFQ7U+oBte8vkm1wGu76+WEC3iNWVAiIVZxLx9rUEsDqj3OovqfLiRsTmNLeY94p2asZjkx7rU48ZwuYN5XGafYsArPscj9Ve6RoRrof+5Q7cc=' TYPE 'ssh-rsa'; From ce1f5144177c404c955bd006f0428ee932ad49ac Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 Apr 2024 10:39:05 +0000 Subject: [PATCH 146/243] Fix optimize_uniq_to_count when only prefix of key is matched --- src/Analyzer/Passes/UniqToCountPass.cpp | 13 +++++++++++-- .../02990_optimize_uniq_to_count_alias.reference | 1 + .../02990_optimize_uniq_to_count_alias.sql | 15 +++++++++++++++ 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp index d7d11e9a580..b801865c9a5 100644 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -29,7 +29,8 @@ NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) return {}; NamesAndTypes result; - for (const auto & group_by_ele : query_node->getGroupByNode()->getChildren()) + const auto & group_by_elements = query_node->getGroupByNode()->getChildren(); + for (const auto & group_by_element : group_by_elements) { const auto & projection_columns = query_node->getProjectionColumns(); const auto & projection_nodes = query_node->getProjection().getNodes(); @@ -38,10 +39,18 @@ NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) for (size_t i = 0; i < projection_columns.size(); i++) { - if (projection_nodes[i]->isEqual(*group_by_ele)) + if (projection_nodes[i]->isEqual(*group_by_element)) + { result.push_back(projection_columns[i]); + break; + } } } + /// If some group by keys are not matched, we cannot apply optimization, + /// because prefix of group by keys may not be unique. + if (result.size() != group_by_elements.size()) + return {}; + return result; } diff --git a/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.reference b/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.reference index 6ed281c757a..e8183f05f5d 100644 --- a/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.reference +++ b/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.reference @@ -1,2 +1,3 @@ 1 1 +1 diff --git a/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.sql b/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.sql index 5ba0be39991..54d19264c45 100644 --- a/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.sql +++ b/tests/queries/0_stateless/02990_optimize_uniq_to_count_alias.sql @@ -34,4 +34,19 @@ FROM ) AS t ) SETTINGS optimize_uniq_to_count=1; +-- https://github.com/ClickHouse/ClickHouse/issues/62298 +DROP TABLE IF EXISTS users; +CREATE TABLE users +( + `id` Int64, + `name` String +) +ENGINE = ReplacingMergeTree +ORDER BY (id, name); + +INSERT INTO users VALUES (1, 'pufit'), (1, 'pufit2'), (1, 'pufit3'); + +SELECT uniqExact(id) FROM ( SELECT id FROM users WHERE id = 1 GROUP BY id, name ); + +DROP TABLE IF EXISTS users; DROP TABLE IF EXISTS tags; From 500c3fe0fcb197f7d8b2f0a6148480727015acf1 Mon Sep 17 00:00:00 2001 From: Sean Haynes Date: Fri, 5 Apr 2024 10:38:28 +0000 Subject: [PATCH 147/243] Fix small typo in Dictionary source loader --- src/Interpreters/ExternalLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 36664cbd06f..53e91971d92 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -1186,7 +1186,7 @@ private: else { auto result = std::chrono::system_clock::now() + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); - LOG_TRACE(log, "Supposed update time for unspecified object is {} (backoff, {} errors.", to_string(result), error_count); + LOG_TRACE(log, "Supposed update time for unspecified object is {} (backoff, {} errors)", to_string(result), error_count); return result; } } From 6428868843eb4666a3ec1defff662f673c8a5e37 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Apr 2024 12:42:03 +0200 Subject: [PATCH 148/243] Fix build --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 9424156a9fb..5d75c9cb18c 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -259,9 +259,9 @@ bool LRUFileCachePriority::canFit( const size_t * max_elements_) const { return (max_size == 0 - || (state->current_size + size - released_size_assumption <= (max_size_ ? *max_size_ : max_size))) + || (state->current_size + size - released_size_assumption <= (max_size_ ? *max_size_ : max_size.load()))) && (max_elements == 0 - || state->current_elements_num + elements - released_elements_assumption <= (max_elements_ ? *max_elements_ : max_elements)); + || state->current_elements_num + elements - released_elements_assumption <= (max_elements_ ? *max_elements_ : max_elements.load())); } bool LRUFileCachePriority::collectCandidatesForEviction( From f7fdb2c4555db5530abfc2a7b10342d1e6e0217d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 5 Apr 2024 14:23:04 +0200 Subject: [PATCH 149/243] More complex locking in StackTrace --- src/Common/StackTrace.cpp | 69 ++++++++++++++++++++++++++++++++------- 1 file changed, 58 insertions(+), 11 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 891850ccb79..78ab43e8991 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -18,13 +18,10 @@ #include #include #include -#include #include #include #include -#include "config.h" - #include #if defined(OS_DARWIN) @@ -481,7 +478,17 @@ void StackTrace::toStringEveryLine(void ** frame_pointers_raw, size_t offset, si toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); } -using StackTraceCache = std::map>; +struct CacheEntry +{ + std::optional stacktrace_string; + bool to_string_in_progress = false; + + std::condition_variable cv; +}; + +using CacheEntryPtr = std::shared_ptr; + +using StackTraceCache = std::map>; static StackTraceCache & cacheInstance() { @@ -493,23 +500,63 @@ static std::mutex stacktrace_cache_mutex; String toStringCached(const StackTrace::FramePointers & pointers, size_t offset, size_t size) { + const StackTraceRefTriple key{pointers, offset, size}; + /// Calculation of stack trace text is extremely slow. /// We use simple cache because otherwise the server could be overloaded by trash queries. /// Note that this cache can grow unconditionally, but practically it should be small. - std::lock_guard lock{stacktrace_cache_mutex}; - + std::unique_lock lock{stacktrace_cache_mutex}; + CacheEntryPtr cache_entry; StackTraceCache & cache = cacheInstance(); - const StackTraceRefTriple key{pointers, offset, size}; - if (auto it = cache.find(key); it != cache.end()) - return it->second; + { + cache_entry = it->second; + } else + { + auto [new_it, inserted] = cache.emplace(StackTraceTriple{pointers, offset, size}, std::make_shared()); + chassert(inserted); + cache_entry = new_it->second; + } + + if (!cache_entry->to_string_in_progress && cache_entry->stacktrace_string.has_value()) + return *cache_entry->stacktrace_string; + + if (cache_entry->to_string_in_progress) + { + cache_entry->cv.wait(lock, [&]{ return !cache_entry->to_string_in_progress; }); + + if (cache_entry->stacktrace_string.has_value()) + return *cache_entry->stacktrace_string; + } + + cache_entry->to_string_in_progress = true; + + lock.unlock(); + + String stacktrace_string; + try { DB::WriteBufferFromOwnString out; toStringEveryLineImpl(false, key, [&](std::string_view str) { out << str << '\n'; }); - - return cache.emplace(StackTraceTriple{pointers, offset, size}, out.str()).first->second; + stacktrace_string = out.str(); } + catch (...) + { + lock.lock(); + cache_entry->to_string_in_progress = false; + lock.unlock(); + cache_entry->cv.notify_one(); + throw; + } + + lock.lock(); + cache_entry->to_string_in_progress = false; + cache_entry->stacktrace_string = stacktrace_string; + lock.unlock(); + + cache_entry->cv.notify_all(); + return stacktrace_string; } std::string StackTrace::toString() const From e53ba4fa9db4646ee3a0c193594379b33043bcf2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 Apr 2024 13:32:07 +0000 Subject: [PATCH 150/243] Analyzer: Fix PREWHERE with lambda functions --- src/Planner/CollectTableExpressionData.cpp | 4 +++- .../0_stateless/03036_prewhere_lambda_function.reference | 2 ++ .../0_stateless/03036_prewhere_lambda_function.sql | 8 ++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03036_prewhere_lambda_function.reference create mode 100644 tests/queries/0_stateless/03036_prewhere_lambda_function.sql diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 385381f1355..27b5909c13b 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -235,7 +235,9 @@ public: static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node) { auto child_node_type = child_node->getNodeType(); - return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION); + return child_node_type != QueryTreeNodeType::QUERY && + child_node_type != QueryTreeNodeType::UNION && + child_node_type != QueryTreeNodeType::LAMBDA; } private: diff --git a/tests/queries/0_stateless/03036_prewhere_lambda_function.reference b/tests/queries/0_stateless/03036_prewhere_lambda_function.reference new file mode 100644 index 00000000000..470e4427d96 --- /dev/null +++ b/tests/queries/0_stateless/03036_prewhere_lambda_function.reference @@ -0,0 +1,2 @@ +[4,5,6] +[4,5,6] diff --git a/tests/queries/0_stateless/03036_prewhere_lambda_function.sql b/tests/queries/0_stateless/03036_prewhere_lambda_function.sql new file mode 100644 index 00000000000..7a5da7ed689 --- /dev/null +++ b/tests/queries/0_stateless/03036_prewhere_lambda_function.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (A Array(Int64)) Engine = MergeTree ORDER BY tuple(); +INSERT INTO t VALUES ([1,2,3]), ([4,5,6]), ([7,8,9]); + +SELECT * FROM t PREWHERE arrayExists(x -> x = 5, A); +SELECT * FROM t PREWHERE arrayExists(lamdba(tuple(x), x = 5), A); + +DROP TABLE t; From 54ceb3d32a7bb490ba7f202a511607f0ea21ae5b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Apr 2024 12:47:00 +0000 Subject: [PATCH 151/243] add some comments --- src/Processors/QueryPlan/PartsSplitter.cpp | 2 ++ .../test_final_bug_with_pk_columns_loading/test.py | 8 +++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index ec51875587e..64af48dd53c 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -128,6 +128,8 @@ class IndexAccess public: explicit IndexAccess(const RangesInDataParts & parts_) : parts(parts_) { + /// Some suffix of index columns might not be loaded (see `primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns`) + /// and we need to use the same set of index columns across all parts. for (const auto & part : parts) loaded_columns = std::min(loaded_columns, part.data_part->getIndex().size()); } diff --git a/tests/integration/test_final_bug_with_pk_columns_loading/test.py b/tests/integration/test_final_bug_with_pk_columns_loading/test.py index e710b9942dc..61559913e05 100644 --- a/tests/integration/test_final_bug_with_pk_columns_loading/test.py +++ b/tests/integration/test_final_bug_with_pk_columns_loading/test.py @@ -19,18 +19,24 @@ def start_cluster(): cluster.shutdown() -def test_simple_query_after_restart(start_cluster): +def test_simple_query_after_index_reload(start_cluster): node.query( """ create table t(a UInt32, b UInt32) engine=MergeTree order by (a, b) settings index_granularity=1; + -- for this part the first columns is useless, so we have to use both insert into t select 42, number from numbers_mt(100); + + -- for this part the first columns is enough insert into t select number, number from numbers_mt(100); """ ) + # force reloading index node.restart_clickhouse() + # the bug happened when we used (a, b) index values for one part and only (a) for another in PartsSplitter. even a simple count query is enough, + # because some granules were assinged to wrong layers and hence not returned from the reading step (because they were filtered out by `FilterSortedStreamByRange`) assert ( int( node.query( From 0f4efdaa4788dc5fd9e4ee96ca611eb35d63a29a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 5 Apr 2024 14:48:39 +0000 Subject: [PATCH 152/243] remove case from 03036_prewhere_lambda_function --- .../queries/0_stateless/03036_prewhere_lambda_function.reference | 1 - tests/queries/0_stateless/03036_prewhere_lambda_function.sql | 1 - 2 files changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03036_prewhere_lambda_function.reference b/tests/queries/0_stateless/03036_prewhere_lambda_function.reference index 470e4427d96..2599763b762 100644 --- a/tests/queries/0_stateless/03036_prewhere_lambda_function.reference +++ b/tests/queries/0_stateless/03036_prewhere_lambda_function.reference @@ -1,2 +1 @@ [4,5,6] -[4,5,6] diff --git a/tests/queries/0_stateless/03036_prewhere_lambda_function.sql b/tests/queries/0_stateless/03036_prewhere_lambda_function.sql index 7a5da7ed689..8b9ebb775a3 100644 --- a/tests/queries/0_stateless/03036_prewhere_lambda_function.sql +++ b/tests/queries/0_stateless/03036_prewhere_lambda_function.sql @@ -3,6 +3,5 @@ CREATE TABLE t (A Array(Int64)) Engine = MergeTree ORDER BY tuple(); INSERT INTO t VALUES ([1,2,3]), ([4,5,6]), ([7,8,9]); SELECT * FROM t PREWHERE arrayExists(x -> x = 5, A); -SELECT * FROM t PREWHERE arrayExists(lamdba(tuple(x), x = 5), A); DROP TABLE t; From 39d706ba9f0c8e7f8c8d757e215f639f7d510fe2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Apr 2024 14:45:51 +0000 Subject: [PATCH 153/243] rework test --- .../__init__.py | 0 .../test.py | 53 ------------------- ...s_splitter_bug_and_index_loading.reference | 1 + ...3_parts_splitter_bug_and_index_loading.sql | 17 ++++++ 4 files changed, 18 insertions(+), 53 deletions(-) delete mode 100644 tests/integration/test_final_bug_with_pk_columns_loading/__init__.py delete mode 100644 tests/integration/test_final_bug_with_pk_columns_loading/test.py create mode 100644 tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.reference create mode 100644 tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql diff --git a/tests/integration/test_final_bug_with_pk_columns_loading/__init__.py b/tests/integration/test_final_bug_with_pk_columns_loading/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_final_bug_with_pk_columns_loading/test.py b/tests/integration/test_final_bug_with_pk_columns_loading/test.py deleted file mode 100644 index 61559913e05..00000000000 --- a/tests/integration/test_final_bug_with_pk_columns_loading/test.py +++ /dev/null @@ -1,53 +0,0 @@ -import pytest -import logging - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", stay_alive=True) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") - - yield cluster - finally: - cluster.shutdown() - - -def test_simple_query_after_index_reload(start_cluster): - node.query( - """ - create table t(a UInt32, b UInt32) engine=MergeTree order by (a, b) settings index_granularity=1; - - -- for this part the first columns is useless, so we have to use both - insert into t select 42, number from numbers_mt(100); - - -- for this part the first columns is enough - insert into t select number, number from numbers_mt(100); - """ - ) - - # force reloading index - node.restart_clickhouse() - - # the bug happened when we used (a, b) index values for one part and only (a) for another in PartsSplitter. even a simple count query is enough, - # because some granules were assinged to wrong layers and hence not returned from the reading step (because they were filtered out by `FilterSortedStreamByRange`) - assert ( - int( - node.query( - "select count() from t where not ignore(*)", - settings={ - "max_threads": 4, - "merge_tree_min_bytes_for_concurrent_read": 1, - "merge_tree_min_rows_for_concurrent_read": 1, - "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": 1, - }, - ) - ) - == 200 - ) diff --git a/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.reference b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.reference new file mode 100644 index 00000000000..08839f6bb29 --- /dev/null +++ b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.reference @@ -0,0 +1 @@ +200 diff --git a/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql new file mode 100644 index 00000000000..541ac67fd24 --- /dev/null +++ b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql @@ -0,0 +1,17 @@ +create table t(a UInt32, b UInt32) engine=MergeTree order by (a, b) settings index_granularity=1; + +-- for this part the first columns is useless, so we have to use both +insert into t select 42, number from numbers_mt(100); + +-- for this part the first columns is enough +insert into t select number, number from numbers_mt(100); + +-- force reloading index +detach table t; +attach table t; + +set merge_tree_min_bytes_for_concurrent_read=1, merge_tree_min_rows_for_concurrent_read=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=1.0, max_threads=4; + +-- the bug happened when we used (a, b) index values for one part and only (a) for another in PartsSplitter. even a simple count query is enough, +-- because some granules were assinged to wrong layers and hence not returned from the reading step (because they were filtered out by `FilterSortedStreamByRange`) +select count() from t where not ignore(*); From b2bcfaf344047f629879143d6bb4efa00c22f7cb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 5 Apr 2024 17:18:22 +0200 Subject: [PATCH 154/243] Reduce log levels for ReadWriteBufferFromHTTP retries --- src/IO/ReadWriteBufferFromHTTP.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index c99b08d0c9d..303ffb744b5 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -345,7 +345,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, if (last_attempt || !is_retriable) { if (!mute_logging) - LOG_ERROR(log, + LOG_DEBUG(log, "Failed to make request to '{}'{}. " "Error: '{}'. " "Failed at try {}/{}.", @@ -361,7 +361,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function && callable, on_retry(); if (!mute_logging) - LOG_INFO(log, + LOG_TRACE(log, "Failed to make request to '{}'{}. " "Error: {}. " "Failed at try {}/{}. " From 7d50bb8c4bf2f7d51d5aece0bb42a1ca4e8afac0 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 5 Apr 2024 12:33:43 -0300 Subject: [PATCH 155/243] fix ut once again --- src/IO/tests/gtest_s3_uri.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 175550acccc..0ec28f80072 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -96,7 +96,7 @@ const TestCase TestCases[] = { false}, // Zonal {S3::URI("https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com/root/nested/file.txt"), - "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w.s3.us-east-1.vpce.amazonaws.com", + "https://bucket.vpce-07a1cd78f1bd55c5f-j3a3vg6w-us-east-1a.s3.us-east-1.vpce.amazonaws.com", "root", "nested/file.txt", "", From 6e413223c2560007bab6422117e4d284c3aefdd4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 5 Apr 2024 09:47:24 +0200 Subject: [PATCH 156/243] Use DETACHED_DIR_NAME everywhere Signed-off-by: Azat Khuzhin --- .../MergeTree/DataPartStorageOnDiskBase.cpp | 10 ++++--- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +-- src/Storages/MergeTree/MergeTreeData.cpp | 27 +++++++++---------- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 16 +++++------ 6 files changed, 32 insertions(+), 31 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 18e4c87b298..052e3ba4b74 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include namespace DB @@ -64,7 +66,7 @@ std::optional DataPartStorageOnDiskBase::getRelativePathForPrefix(Logger auto full_relative_path = fs::path(root_path); if (detached) - full_relative_path /= "detached"; + full_relative_path /= MergeTreeData::DETACHED_DIR_NAME; std::optional original_checksums_content; std::optional original_files_list; @@ -109,7 +111,7 @@ bool DataPartStorageOnDiskBase::looksLikeBrokenDetachedPartHasTheSameContent(con if (!exists("checksums.txt")) return false; - auto storage_from_detached = create(volume, fs::path(root_path) / "detached", detached_part_path, /*initialize=*/ true); + auto storage_from_detached = create(volume, fs::path(root_path) / MergeTreeData::DETACHED_DIR_NAME, detached_part_path, /*initialize=*/ true); if (!storage_from_detached->exists("checksums.txt")) return false; @@ -490,7 +492,7 @@ MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze( auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); /// Do not initialize storage in case of DETACH because part may be broken. - bool to_detached = dir_path.starts_with("detached/"); + bool to_detached = dir_path.starts_with(std::string_view((fs::path(MergeTreeData::DETACHED_DIR_NAME) / "").string())); return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !params.external_transaction); } @@ -618,7 +620,7 @@ void DataPartStorageOnDiskBase::remove( if (part_dir_without_slash.has_parent_path()) { auto parent_path = part_dir_without_slash.parent_path(); - if (parent_path == "detached") + if (parent_path == MergeTreeData::DETACHED_DIR_NAME) throw Exception( ErrorCodes::LOGICAL_ERROR, "Trying to remove detached part {} with path {} in remove function. It shouldn't happen", diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 91444d76a52..cf7889c0aee 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -21,7 +22,6 @@ #include #include #include -#include #include @@ -803,7 +803,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "`tmp_prefix` and `part_name` cannot be empty or contain '.' or '/' characters."); auto part_dir = tmp_prefix + part_name; - auto part_relative_path = data.getRelativeDataPath() + String(to_detached ? "detached/" : ""); + auto part_relative_path = data.getRelativeDataPath() + String(to_detached ? MergeTreeData::DETACHED_DIR_NAME : ""); auto volume = std::make_shared("volume_" + part_name, disk); /// Create temporary part storage to write sent files. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 8da46b39801..441437855ab 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1844,7 +1844,7 @@ try } catch (...) { - if (startsWith(new_relative_path, "detached/")) + if (startsWith(new_relative_path, fs::path(MergeTreeData::DETACHED_DIR_NAME) / "")) { // Don't throw when the destination is to the detached folder. It might be able to // recover in some cases, such as fetching parts into multi-disks while some of the @@ -1957,7 +1957,7 @@ std::optional IMergeTreeDataPart::getRelativePathForDetachedPart(const S DetachedPartInfo::DETACH_REASONS.end(), prefix) != DetachedPartInfo::DETACH_REASONS.end()); if (auto path = getRelativePathForPrefix(prefix, /* detached */ true, broken)) - return "detached/" + *path; + return fs::path(MergeTreeData::DETACHED_DIR_NAME) / *path; return {}; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 461d9a31eaa..dc15b8ab940 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -262,7 +262,7 @@ void MergeTreeData::initializeDirectoriesAndFormatVersion(const std::string & re if (need_create_directories) { disk->createDirectories(relative_data_path); - disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); + disk->createDirectories(fs::path(relative_data_path) / DETACHED_DIR_NAME); } if (disk->exists(format_version_path)) @@ -1713,7 +1713,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks, std::optionalname(), "tmp") || it->name() == MergeTreeData::FORMAT_VERSION_FILE_NAME - || it->name() == MergeTreeData::DETACHED_DIR_NAME) + || it->name() == DETACHED_DIR_NAME) continue; if (auto part_info = MergeTreePartInfo::tryParsePartName(it->name(), format_version)) @@ -2796,7 +2796,7 @@ void MergeTreeData::dropAllData() && settings_ptr->allow_remote_fs_zero_copy_replication; try { - bool keep_shared = removeDetachedPart(part.disk, fs::path(relative_data_path) / "detached" / part.dir_name / "", part.dir_name); + bool keep_shared = removeDetachedPart(part.disk, fs::path(relative_data_path) / DETACHED_DIR_NAME / part.dir_name / "", part.dir_name); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", part.dir_name, keep_shared); } catch (...) @@ -2879,8 +2879,8 @@ void MergeTreeData::dropIfEmpty() if (disk->isBroken()) continue; /// Non recursive, exception is thrown if there are more files. - disk->removeFileIfExists(fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME); - disk->removeDirectory(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); + disk->removeFileIfExists(fs::path(relative_data_path) / FORMAT_VERSION_FILE_NAME); + disk->removeDirectory(fs::path(relative_data_path) / DETACHED_DIR_NAME); disk->removeDirectory(relative_data_path); } } @@ -3443,7 +3443,7 @@ void MergeTreeData::changeSettings( { auto disk = new_storage_policy->getDiskByName(disk_name); disk->createDirectories(relative_data_path); - disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); + disk->createDirectories(fs::path(relative_data_path) / DETACHED_DIR_NAME); } /// FIXME how would that be done while reloading configuration??? @@ -6037,7 +6037,7 @@ DetachedPartsInfo MergeTreeData::getDetachedParts() const for (const auto & disk : getDisks()) { - String detached_path = fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME; + String detached_path = fs::path(relative_data_path) / DETACHED_DIR_NAME; /// Note: we don't care about TOCTOU issue here. if (disk->exists(detached_path)) @@ -6063,7 +6063,7 @@ void MergeTreeData::validateDetachedPartName(const String & name) void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr local_context) { - PartsTemporaryRename renamed_parts(*this, "detached/"); + PartsTemporaryRename renamed_parts(*this, DETACHED_DIR_NAME); if (part) { @@ -6088,7 +6088,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names) { - bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name); + bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / DETACHED_DIR_NAME / new_name / "", old_name); LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared); old_name.clear(); } @@ -6097,14 +6097,14 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const ASTPtr & partition, bool attach_part, ContextPtr local_context, PartsTemporaryRename & renamed_parts) { - const String source_dir = "detached/"; + const fs::path source_dir = DETACHED_DIR_NAME; /// Let's compose a list of parts that should be added. if (attach_part) { const String part_id = partition->as().value.safeGet(); validateDetachedPartName(part_id); - if (temporary_parts.contains(String(DETACHED_DIR_NAME) + "/" + part_id)) + if (temporary_parts.contains(source_dir / part_id)) { LOG_WARNING(log, "Will not try to attach part {} because its directory is temporary, " "probably it's being detached right now", part_id); @@ -6181,7 +6181,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const LOG_DEBUG(log, "Checking part {}", new_name); auto single_disk_volume = std::make_shared("volume_" + old_name, disk); - auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir + new_name) + auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir / new_name) .withPartFormatFromDisk() .build(); @@ -7212,11 +7212,10 @@ String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const DiskPtr MergeTreeData::tryGetDiskForDetachedPart(const String & part_name) const { - String additional_path = "detached/"; const auto disks = getStoragePolicy()->getDisks(); for (const DiskPtr & disk : disks) - if (disk->exists(fs::path(relative_data_path) / additional_path / part_name)) + if (disk->exists(fs::path(relative_data_path) / DETACHED_DIR_NAME / part_name)) return disk; return nullptr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c9f451b6bb1..6861b615cd6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2024,7 +2024,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( bool attach_part, ContextPtr local_context) { PartitionCommandsResultInfo results; - PartsTemporaryRename renamed_parts(*this, "detached/"); + PartsTemporaryRename renamed_parts(*this, DETACHED_DIR_NAME); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, local_context, renamed_parts); for (size_t i = 0; i < loaded_parts.size(); ++i) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6ab56ba141c..73354e71e71 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1983,7 +1983,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo for (const DiskPtr & disk : getStoragePolicy()->getDisks()) { - for (const auto it = disk->iterateDirectory(fs::path(relative_data_path) / "detached/"); it->isValid(); it->next()) + for (const auto it = disk->iterateDirectory(fs::path(relative_data_path) / DETACHED_DIR_NAME); it->isValid(); it->next()) { const auto part_info = MergeTreePartInfo::tryParsePartName(it->name(), format_version); @@ -1993,7 +1993,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const auto part_old_name = part_info->getPartNameV1(); const auto volume = std::make_shared("volume_" + part_old_name, disk); - auto part = getDataPartBuilder(entry.new_part_name, volume, fs::path("detached") / part_old_name) + auto part = getDataPartBuilder(entry.new_part_name, volume, fs::path(DETACHED_DIR_NAME) / part_old_name) .withPartFormatFromDisk() .build(); @@ -2440,7 +2440,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) { String part_dir = part_to_detach->getDataPartStorage().getPartDirectory(); LOG_INFO(log, "Detaching {}", part_dir); - auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); + auto holder = getTemporaryPartDirectoryHolder(fs::path(DETACHED_DIR_NAME) / part_dir); part_to_detach->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } @@ -2967,7 +2967,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr part = get_part(); // The fetched part is valuable and should not be cleaned like a temp part. part->is_temp = false; - part->renameTo("detached/" + entry.new_part_name, true); + part->renameTo(fs::path(DETACHED_DIR_NAME) / entry.new_part_name, true); LOG_INFO(log, "Cloned part {} to detached directory", part->name); } @@ -4987,7 +4987,7 @@ bool StorageReplicatedMergeTree::fetchPart( { // The fetched part is valuable and should not be cleaned like a temp part. part->is_temp = false; - part->renameTo(fs::path("detached") / part_name, true); + part->renameTo(fs::path(DETACHED_DIR_NAME) / part_name, true); } } catch (const Exception & e) @@ -6547,7 +6547,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( assertNotReadonly(); PartitionCommandsResultInfo results; - PartsTemporaryRename renamed_parts(*this, "detached/"); + PartsTemporaryRename renamed_parts(*this, DETACHED_DIR_NAME); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. @@ -9986,7 +9986,7 @@ bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_n { fs::directory_iterator dir_end; for (const std::string & path : getDataPaths()) - for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it{fs::path(path) / DETACHED_DIR_NAME}; dir_it != dir_end; ++dir_it) if (dir_it->path().filename().string() == part_name) return true; return false; @@ -9999,7 +9999,7 @@ bool StorageReplicatedMergeTree::checkIfDetachedPartitionExists(const String & p for (const std::string & path : getDataPaths()) { - for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it{fs::path(path) / DETACHED_DIR_NAME}; dir_it != dir_end; ++dir_it) { const String file_name = dir_it->path().filename().string(); auto part_info = MergeTreePartInfo::tryParsePartName(file_name, format_version); From b2c9cb0653f6d4857a9ea1eb98904c0b4d1d7526 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 5 Apr 2024 18:19:47 +0200 Subject: [PATCH 157/243] Fix flaky tests --- .../03049_unknown_identifier_materialized_column.sql | 4 ---- tests/queries/0_stateless/03068_analyzer_distributed_join.sql | 2 ++ .../queries/0_stateless/03084_analyzer_join_column_alias.sql | 4 ++-- .../03088_analyzer_ambiguous_column_multi_call.sql | 2 ++ 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql index a1c858a329c..276e4845831 100644 --- a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql +++ b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql @@ -1,8 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54317 SET allow_experimental_analyzer=1; DROP DATABASE IF EXISTS 03049_database; -DROP TABLE IF EXISTS 03049_database.l; -DROP TABLE IF EXISTS 03049_database.r; CREATE DATABASE 03049_database; USE 03049_database; @@ -14,5 +12,3 @@ select * from l left join r on l.y = r.y where r.ty >= 2019; select * from 03049_database.l left join 03049_database.r on l.y = r.y where r.ty >= 2019; DROP DATABASE IF EXISTS 03049_database; -DROP TABLE IF EXISTS 03049_database.l; -DROP TABLE IF EXISTS 03049_database.r; diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql index 82f58e9a750..61b1199dc44 100644 --- a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -1,4 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/6571 +-- Tag: no-replicated-database + SET allow_experimental_analyzer=1; CREATE TABLE LINEITEM_shard ON CLUSTER test_shard_localhost ( diff --git a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql index 8337c0ce987..930726898b5 100644 --- a/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql +++ b/tests/queries/0_stateless/03084_analyzer_join_column_alias.sql @@ -1,13 +1,13 @@ -- https://github.com/ClickHouse/ClickHouse/issues/47432 SET allow_experimental_analyzer=1; -create or replace table t1 +create table t1 engine = MergeTree() order by tuple() as select 1 as user_id, 2 as level; -create or replace table t2 +create table t2 engine = MergeTree() order by tuple() as diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql index 09425d2e503..4ca5005fa1d 100644 --- a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql +++ b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql @@ -1,5 +1,7 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61014 SET allow_experimental_analyzer=1; + +DROP DATABASE IF EXISTS test_03088; create database test_03088; create table test_03088.a (i int) engine = Log(); From eb9690016a06ad544e0d819a44ad772b1d0cd2cc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Apr 2024 18:37:46 +0200 Subject: [PATCH 158/243] Fix assertion --- src/Interpreters/Cache/EvictionCandidates.h | 6 ---- src/Interpreters/Cache/FileCache.cpp | 21 +++++------- src/Interpreters/Cache/IFileCachePriority.h | 3 +- .../Cache/LRUFileCachePriority.cpp | 6 ++-- src/Interpreters/Cache/LRUFileCachePriority.h | 3 +- .../Cache/SLRUFileCachePriority.cpp | 33 +++++++++++-------- .../Cache/SLRUFileCachePriority.h | 3 +- 7 files changed, 37 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index 140728ae704..baacbc0cfae 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -10,12 +10,6 @@ public: using FinalizeEvictionFunc = std::function; EvictionCandidates() = default; - EvictionCandidates(EvictionCandidates && other) noexcept - { - candidates = std::move(other.candidates); - candidates_size = std::move(other.candidates_size); - queue_entries_to_invalidate = std::move(other.queue_entries_to_invalidate); - } ~EvictionCandidates(); void add( diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index bf76a307364..12ea2c178bc 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1388,8 +1388,8 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, if (new_settings.max_size != actual_settings.max_size || new_settings.max_elements != actual_settings.max_elements) { - std::optional eviction_candidates; - bool modified_size_limits = false; + EvictionCandidates eviction_candidates; + bool limits_satisfied = false; { cache_is_being_resized.store(true, std::memory_order_relaxed); SCOPE_EXIT({ @@ -1399,15 +1399,12 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, auto cache_lock = lockCache(); FileCacheReserveStat stat; - eviction_candidates.emplace(main_priority->collectCandidatesForEviction( - new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, cache_lock)); + limits_satisfied = main_priority->collectCandidatesForEviction( + new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, eviction_candidates, cache_lock); - eviction_candidates->removeQueueEntries(cache_lock); + eviction_candidates.removeQueueEntries(cache_lock); - modified_size_limits = main_priority->getSize(cache_lock) <= new_settings.max_size - && main_priority->getElementsCount(cache_lock) <= new_settings.max_elements; - - if (modified_size_limits) + if (limits_satisfied) { main_priority->modifySizeLimits( new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); @@ -1423,16 +1420,16 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, try { - eviction_candidates->evict(); + eviction_candidates.evict(); } catch (...) { auto cache_lock = lockCache(); - eviction_candidates->finalize(nullptr, cache_lock); + eviction_candidates.finalize(nullptr, cache_lock); throw; } - if (modified_size_limits) + if (limits_satisfied) { LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", actual_settings.max_size, new_settings.max_size, diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 042234b9f6c..a727aab68ee 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -147,11 +147,12 @@ public: const CachePriorityGuard::Lock &) = 0; /// Collect eviction `candidates_num` candidates for eviction. - virtual EvictionCandidates collectCandidatesForEviction( + virtual bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & candidates, const CachePriorityGuard::Lock &) = 0; virtual bool modifySizeLimits( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 5d75c9cb18c..e859529f5e7 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -322,14 +322,14 @@ bool LRUFileCachePriority::collectCandidatesForEviction( } } -EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( +bool LRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock & lock) { - EvictionCandidates res; auto stop_condition = [&, this]() { return canFit(0, 0, stat.total_stat.releasable_size, stat.total_stat.releasable_count, @@ -337,7 +337,7 @@ EvictionCandidates LRUFileCachePriority::collectCandidatesForEviction( || (max_candidates_to_evict && res.size() >= max_candidates_to_evict); }; iterateForEviction(res, stat, stop_condition, lock); - return res; + return stop_condition(); } void LRUFileCachePriority::iterateForEviction( diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 1c8cfa6795f..d31a3fb0f10 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -62,11 +62,12 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - EvictionCandidates collectCandidatesForEviction( + bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override; diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.cpp b/src/Interpreters/Cache/SLRUFileCachePriority.cpp index 802a88db23f..68bf182dd2e 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/SLRUFileCachePriority.cpp @@ -251,42 +251,47 @@ bool SLRUFileCachePriority::collectCandidatesForEvictionInProtected( return true; } -EvictionCandidates SLRUFileCachePriority::collectCandidatesForEviction( +bool SLRUFileCachePriority::collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock & lock) { const auto desired_probationary_size = getRatio(desired_size, 1 - size_ratio); const auto desired_probationary_elements_num = getRatio(desired_elements_count, 1 - size_ratio); - auto res = probationary_queue.collectCandidatesForEviction( - desired_probationary_size, desired_probationary_elements_num, max_candidates_to_evict, stat, lock); + FileCacheReserveStat probationary_stat; + const bool probationary_limit_satisfied = probationary_queue.collectCandidatesForEviction( + desired_probationary_size, desired_probationary_elements_num, + max_candidates_to_evict, probationary_stat, res, lock); - LOG_TEST(log, "Collected {} eviction candidates from probationary queue (size: {})", - res.size(), stat.total_stat.releasable_size); + stat += probationary_stat; + + LOG_TEST(log, "Collected {} to evict from probationary queue. Total size: {}", + res.size(), probationary_stat.total_stat.releasable_size); chassert(!max_candidates_to_evict || res.size() <= max_candidates_to_evict); chassert(res.size() == stat.total_stat.releasable_count); - if (max_candidates_to_evict && res.size() == max_candidates_to_evict) - return res; + if (max_candidates_to_evict && res.size() >= max_candidates_to_evict) + return probationary_limit_satisfied; const auto desired_protected_size = getRatio(max_size, size_ratio); const auto desired_protected_elements_num = getRatio(max_elements, size_ratio); FileCacheReserveStat protected_stat; - auto res_add = protected_queue.collectCandidatesForEviction( + const bool protected_limit_satisfied = protected_queue.collectCandidatesForEviction( desired_protected_size, desired_protected_elements_num, - max_candidates_to_evict ? max_candidates_to_evict - res.size() : 0, protected_stat, lock); - - LOG_TEST(log, "Collected {} eviction candidates from protected queue (size: {})", - res_add.size(), protected_stat.total_stat.releasable_size); + max_candidates_to_evict - res.size(), protected_stat, res, lock); stat += protected_stat; - res.insert(std::move(res_add), lock); - return res; + + LOG_TEST(log, "Collected {} to evict from protected queue. Total size: {}", + res.size(), protected_stat.total_stat.releasable_size); + + return probationary_limit_satisfied && protected_limit_satisfied; } void SLRUFileCachePriority::downgrade(IteratorPtr iterator, const CachePriorityGuard::Lock & lock) diff --git a/src/Interpreters/Cache/SLRUFileCachePriority.h b/src/Interpreters/Cache/SLRUFileCachePriority.h index e837b8541c2..ee3cafe322d 100644 --- a/src/Interpreters/Cache/SLRUFileCachePriority.h +++ b/src/Interpreters/Cache/SLRUFileCachePriority.h @@ -58,11 +58,12 @@ public: const UserID & user_id, const CachePriorityGuard::Lock &) override; - EvictionCandidates collectCandidatesForEviction( + bool collectCandidatesForEviction( size_t desired_size, size_t desired_elements_count, size_t max_candidates_to_evict, FileCacheReserveStat & stat, + EvictionCandidates & res, const CachePriorityGuard::Lock &) override; void shuffle(const CachePriorityGuard::Lock &) override; From 378d330d9dfa289c413f80c2addaf6dee5503093 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Apr 2024 17:07:43 +0000 Subject: [PATCH 159/243] better --- .../0_stateless/03033_parts_splitter_bug_and_index_loading.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql index 541ac67fd24..25ec1c8fd80 100644 --- a/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql +++ b/tests/queries/0_stateless/03033_parts_splitter_bug_and_index_loading.sql @@ -1,5 +1,7 @@ create table t(a UInt32, b UInt32) engine=MergeTree order by (a, b) settings index_granularity=1; +system stop merges t; + -- for this part the first columns is useless, so we have to use both insert into t select 42, number from numbers_mt(100); From 0bce544779bd881aa3218694545fe5a8017ee9a4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 5 Apr 2024 23:07:00 +0200 Subject: [PATCH 160/243] Update base/poco/Net/src/HTTPClientSession.cpp Co-authored-by: Nikita Taranov --- base/poco/Net/src/HTTPClientSession.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/Net/src/HTTPClientSession.cpp b/base/poco/Net/src/HTTPClientSession.cpp index e489ab56b98..c9899266be7 100644 --- a/base/poco/Net/src/HTTPClientSession.cpp +++ b/base/poco/Net/src/HTTPClientSession.cpp @@ -248,7 +248,7 @@ void HTTPClientSession::setKeepAliveRequest(int request) - void HTTPClientSession::setLastRequest(Poco::Timestamp time) +void HTTPClientSession::setLastRequest(Poco::Timestamp time) { if (connected()) { From f766ec678206c0b0e5f0eac0d142583fa47d89cd Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 5 Apr 2024 23:19:30 +0200 Subject: [PATCH 161/243] review remarks --- src/Common/HTTPConnectionPool.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 7f99d6a647f..167aeee68f3 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -213,7 +213,8 @@ public: --total_connections_in_group; - const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 20 : 1; + const size_t gap = 20; + const size_t reduced_warning_limit = limits.warning_limit > gap ? limits.warning_limit - gap : 1; if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit) { LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group); @@ -289,8 +290,7 @@ private: auto timeouts = getTimeouts(*this); auto new_connection = lock->getConnection(timeouts); Session::assign(*new_connection); - if (Session::getKeepAliveRequest() == 0) - Session::setKeepAliveRequest(1); + Session::setKeepAliveRequest(Session::getKeepAliveRequest() + 1); } else { @@ -425,7 +425,7 @@ private: ConnectionGroup::Ptr group_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args) - : Session(args...) + : Session(std::forward(args)...) , pool(std::move(pool_)) , group(group_) , metrics(std::move(metrics_)) From c0ffee763cd90da02310e94c99e04d6cc5a2afa6 Mon Sep 17 00:00:00 2001 From: Anita Hammer <166057949+anitahammer@users.noreply.github.com> Date: Sat, 6 Apr 2024 02:48:50 +0300 Subject: [PATCH 162/243] Fix contributor name vulnerability --- tests/ci/version_helper.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 30b0c2d96be..f649732171f 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -357,8 +357,9 @@ def update_contributors( # format: " 1016 Alexey Arno" shortlog = git_runner.run("git shortlog HEAD --summary") + escaping = str.maketrans({"\\": "\\\\", '"': '\\"'}) contributors = sorted( - [c.split(maxsplit=1)[-1].replace('"', r"\"") for c in shortlog.split("\n")], + [c.split(maxsplit=1)[-1].translate(escaping) for c in shortlog.split("\n")], ) contributors = [f' "{c}",' for c in contributors] From aae16fb41e960895c384fcacdbbef4940af18e28 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 6 Apr 2024 11:19:48 +0200 Subject: [PATCH 163/243] Remove macro --- src/Common/ThreadStatus.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 3e7f27f4d4f..ad96018a17e 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -141,9 +141,6 @@ void ThreadStatus::initGlobalProfiler([[maybe_unused]] UInt64 global_profiler_re { tryLogCurrentException("ThreadStatus", "Cannot initialize GlobalProfiler"); } -#else - UNUSED(global_profiler_real_time_period); - UNUSED(global_profiler_cpu_time_period); #endif } From 933d6e86dab5d0c489134d7c215b4e40f0c19ed0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 6 Apr 2024 19:33:54 +0200 Subject: [PATCH 164/243] Remove config --- src/Common/ThreadStatus.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 02bf82e2da3..0c02ab8fdb0 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -1,6 +1,5 @@ #pragma once -#include "config.h" #include #include #include From a6dbaf540882b18950e28ec5f1c9155e34aeadf1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Apr 2024 21:45:26 +0200 Subject: [PATCH 165/243] dhparams are not enabled by default --- programs/install/Install.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 0ff0faff624..20c1a0ad4a8 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -662,7 +662,6 @@ int mainEntryClickHouseInstall(int argc, char ** argv) " \n" " " << (config_dir / "server.crt").string() << "\n" " " << (config_dir / "server.key").string() << "\n" - " " << (config_dir / "dhparam.pem").string() << "\n" " \n" " \n" "\n"; From 664823463b23d00d2aa4293bdea763112b652ddb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Apr 2024 21:46:43 +0200 Subject: [PATCH 166/243] Do not create a directory for UDF in clickhouse-client if it does not exist --- .../UserDefined/UserDefinedSQLObjectsDiskStorage.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp index b083c540083..d874612ad04 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsDiskStorage.cpp @@ -56,7 +56,6 @@ UserDefinedSQLObjectsDiskStorage::UserDefinedSQLObjectsDiskStorage(const Context , dir_path{makeDirectoryPathCanonical(dir_path_)} , log{getLogger("UserDefinedSQLObjectsLoaderFromDisk")} { - createDirectory(); } @@ -122,7 +121,12 @@ void UserDefinedSQLObjectsDiskStorage::reloadObjects() void UserDefinedSQLObjectsDiskStorage::loadObjectsImpl() { LOG_INFO(log, "Loading user defined objects from {}", dir_path); - createDirectory(); + + if (!std::filesystem::exists(dir_path)) + { + LOG_DEBUG(log, "The directory for user defined objects ({}) does not exist: nothing to load", dir_path); + return; + } std::vector> function_names_and_queries; @@ -157,7 +161,6 @@ void UserDefinedSQLObjectsDiskStorage::loadObjectsImpl() void UserDefinedSQLObjectsDiskStorage::reloadObject(UserDefinedSQLObjectType object_type, const String & object_name) { - createDirectory(); auto ast = tryLoadObject(object_type, object_name); if (ast) setObject(object_name, *ast); @@ -185,6 +188,7 @@ bool UserDefinedSQLObjectsDiskStorage::storeObjectImpl( bool replace_if_exists, const Settings & settings) { + createDirectory(); String file_path = getFilePath(object_type, object_name); LOG_DEBUG(log, "Storing user-defined object {} to file {}", backQuote(object_name), file_path); From 982d3db5b1fa220323c3f7d911b11b375945ca93 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 6 Apr 2024 21:51:28 +0200 Subject: [PATCH 167/243] disable window view with analyzer properly --- src/Storages/WindowView/StorageWindowView.cpp | 38 +++++++++++++++++-- src/Storages/WindowView/StorageWindowView.h | 4 ++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 0b822b9aab3..f82f5b07910 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -439,6 +439,7 @@ bool StorageWindowView::optimize( bool cleanup, ContextPtr local_context) { + throwIfWindowViewIsDisabled(local_context); auto storage_ptr = getInnerTable(); auto metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); return getInnerTable()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); @@ -449,6 +450,7 @@ void StorageWindowView::alter( ContextPtr local_context, AlterLockHolder &) { + throwIfWindowViewIsDisabled(local_context); auto table_id = getStorageID(); StorageInMemoryMetadata new_metadata = getInMemoryMetadata(); StorageInMemoryMetadata old_metadata = getInMemoryMetadata(); @@ -508,8 +510,9 @@ void StorageWindowView::alter( startup(); } -void StorageWindowView::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /*local_context*/) const +void StorageWindowView::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const { + throwIfWindowViewIsDisabled(local_context); for (const auto & command : commands) { if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_QUERY) @@ -519,6 +522,7 @@ void StorageWindowView::checkAlterIsPossible(const AlterCommands & commands, Con std::pair StorageWindowView::getNewBlocks(UInt32 watermark) { + throwIfWindowViewIsDisabled(); UInt32 w_start = addTime(watermark, window_kind, -window_num_units, *time_zone); auto inner_table = getInnerTable(); @@ -654,6 +658,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) inline void StorageWindowView::fire(UInt32 watermark) { + throwIfWindowViewIsDisabled(); LOG_TRACE(log, "Watch streams number: {}, target table: {}", watch_streams.size(), target_table_id.empty() ? "None" : target_table_id.getNameForLogs()); @@ -722,6 +727,7 @@ inline void StorageWindowView::fire(UInt32 watermark) ASTPtr StorageWindowView::getSourceTableSelectQuery() { + throwIfWindowViewIsDisabled(); auto query = select_query->clone(); auto & modified_select = query->as(); @@ -947,6 +953,7 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) void StorageWindowView::addFireSignal(std::set & signals) { + throwIfWindowViewIsDisabled(); std::lock_guard lock(fire_signal_mutex); for (const auto & signal : signals) fire_signal.push_back(signal); @@ -962,6 +969,7 @@ void StorageWindowView::updateMaxTimestamp(UInt32 timestamp) void StorageWindowView::updateMaxWatermark(UInt32 watermark) { + throwIfWindowViewIsDisabled(); if (is_proctime) { max_watermark = watermark; @@ -1014,6 +1022,7 @@ void StorageWindowView::cleanup() void StorageWindowView::threadFuncCleanup() { + throwIfWindowViewIsDisabled(); if (shutdown_called) return; @@ -1033,6 +1042,7 @@ void StorageWindowView::threadFuncCleanup() void StorageWindowView::threadFuncFireProc() { + throwIfWindowViewIsDisabled(); if (shutdown_called) return; @@ -1069,6 +1079,7 @@ void StorageWindowView::threadFuncFireProc() void StorageWindowView::threadFuncFireEvent() { + throwIfWindowViewIsDisabled(); std::lock_guard lock(fire_signal_mutex); LOG_TRACE(log, "Fire events: {}", fire_signal.size()); @@ -1100,6 +1111,7 @@ void StorageWindowView::read( const size_t max_block_size, const size_t num_streams) { + throwIfWindowViewIsDisabled(local_context); if (target_table_id.empty()) return; @@ -1140,6 +1152,7 @@ Pipe StorageWindowView::watch( size_t /*max_block_size*/, const size_t /*num_streams*/) { + throwIfWindowViewIsDisabled(local_context); ASTWatchQuery & query = typeid_cast(*query_info.query); bool has_limit = false; @@ -1178,8 +1191,10 @@ StorageWindowView::StorageWindowView( , clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds()) { if (context_->getSettingsRef().allow_experimental_analyzer) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Experimental WINDOW VIEW feature is not supported with new infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); + disabled_due_to_analyzer = true; + + if (mode <= LoadingStrictnessLevel::CREATE) + throwIfWindowViewIsDisabled(); if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); @@ -1243,6 +1258,9 @@ StorageWindowView::StorageWindowView( } } + if (disabled_due_to_analyzer) + return; + clean_cache_task = getContext()->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanup(); }); fire_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName(), [this] { is_proctime ? threadFuncFireProc() : threadFuncFireEvent(); }); @@ -1400,6 +1418,7 @@ void StorageWindowView::eventTimeParser(const ASTCreateQuery & query) void StorageWindowView::writeIntoWindowView( StorageWindowView & window_view, const Block & block, ContextPtr local_context) { + window_view.throwIfWindowViewIsDisabled(local_context); while (window_view.modifying_query) std::this_thread::sleep_for(std::chrono::milliseconds(100)); @@ -1589,6 +1608,9 @@ void StorageWindowView::writeIntoWindowView( void StorageWindowView::startup() { + if (disabled_due_to_analyzer) + return; + DatabaseCatalog::instance().addViewDependency(select_table_id, getStorageID()); fire_task->activate(); @@ -1602,6 +1624,8 @@ void StorageWindowView::startup() void StorageWindowView::shutdown(bool) { shutdown_called = true; + if (disabled_due_to_analyzer) + return; fire_condition.notify_all(); @@ -1657,6 +1681,7 @@ Block StorageWindowView::getInputHeader() const const Block & StorageWindowView::getOutputHeader() const { + throwIfWindowViewIsDisabled(); std::lock_guard lock(sample_block_lock); if (!output_header) { @@ -1681,6 +1706,13 @@ StoragePtr StorageWindowView::getTargetTable() const return DatabaseCatalog::instance().getTable(target_table_id, getContext()); } +void StorageWindowView::throwIfWindowViewIsDisabled(ContextPtr local_context) const +{ + if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().allow_experimental_analyzer)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Experimental WINDOW VIEW feature is not supported " + "with new infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); +} + void registerStorageWindowView(StorageFactory & factory) { factory.registerStorage("WindowView", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 0b7cd54e3a7..f79867df424 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -271,5 +271,9 @@ private: StoragePtr getSourceTable() const; StoragePtr getInnerTable() const; StoragePtr getTargetTable() const; + + bool disabled_due_to_analyzer = false; + + void throwIfWindowViewIsDisabled(ContextPtr local_context = nullptr) const; }; } From c5e47bbe70e232188e36d0599e29605db4905861 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 6 Apr 2024 21:52:04 +0200 Subject: [PATCH 168/243] Add a test --- .../03033_analyzer_query_parameters.sh | 4 ++-- ...udf_user_defined_directory_in_client.reference | 1 + .../03036_udf_user_defined_directory_in_client.sh | 15 +++++++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference create mode 100755 tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh diff --git a/tests/queries/0_stateless/03033_analyzer_query_parameters.sh b/tests/queries/0_stateless/03033_analyzer_query_parameters.sh index c821791e437..cf46067df99 100755 --- a/tests/queries/0_stateless/03033_analyzer_query_parameters.sh +++ b/tests/queries/0_stateless/03033_analyzer_query_parameters.sh @@ -4,5 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -clickhouse-local --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 1" -clickhouse-local --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 0" +${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 1" +${CLICKHOUSE_LOCAL} --param_rounding 1 --query "SELECT 1 AS x ORDER BY x WITH FILL STEP {rounding:UInt32} SETTINGS allow_experimental_analyzer = 0" diff --git a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference new file mode 100644 index 00000000000..251d054748a --- /dev/null +++ b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.reference @@ -0,0 +1 @@ +Unknown function diff --git a/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh new file mode 100755 index 00000000000..e0a145d8456 --- /dev/null +++ b/tests/queries/0_stateless/03036_udf_user_defined_directory_in_client.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE test (s String) ENGINE = Memory" + +# Calling an unknown function should not lead to creation of a 'user_defined' directory in the current directory +${CLICKHOUSE_CLIENT} --query "INSERT INTO test VALUES (xyz('abc'))" 2>&1 | grep -o -F 'Unknown function' + +ls -ld user_defined 2> /dev/null + +${CLICKHOUSE_CLIENT} --query "DROP TABLE test" From 97281203b8e5009412f58338ff7e7fbab34b431a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 7 Apr 2024 09:51:45 +0000 Subject: [PATCH 169/243] Don't access static members through instance, pt. II clang-tidy rightfully complains (-readability-static-accessed-through-instance) that static member functions are accessed through non-static members --- .clang-tidy | 1 - programs/disks/DisksApp.cpp | 2 +- programs/keeper-client/KeeperClient.cpp | 2 +- programs/local/LocalServer.cpp | 2 +- ...regateFunctionExponentialMovingAverage.cpp | 14 +++++----- ...AggregateFunctionKolmogorovSmirnovTest.cpp | 14 +++++----- src/AggregateFunctions/DDSketch.h | 4 +++ src/AggregateFunctions/DDSketch/Store.h | 4 +++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 7 +++-- src/Client/Connection.cpp | 2 +- src/Columns/ColumnObject.cpp | 6 ++--- .../tests/gtest_compressionCodec.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 2 +- src/DataTypes/DataTypeDate32.cpp | 2 +- .../tests/gtest_json_parser.cpp | 4 +-- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Functions/FunctionsConversion.cpp | 4 +-- .../UserDefinedSQLFunctionFactory.cpp | 4 +-- src/IO/MMapReadBufferFromFileWithCache.cpp | 2 +- src/Interpreters/ActionsVisitor.cpp | 4 +-- src/Interpreters/Aggregator.cpp | 6 ++--- src/Interpreters/InterpreterAlterQuery.cpp | 4 +-- .../InterpreterCreateIndexQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++--- src/Interpreters/Set.cpp | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Interpreters/executeQuery.cpp | 2 +- src/Interpreters/tests/gtest_filecache.cpp | 16 ++++++------ src/Loggers/Loggers.cpp | 4 +++ src/Processors/Merges/Algorithms/Graphite.cpp | 26 +++++++++---------- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageDictionary.cpp | 2 +- src/Storages/StorageFuzzJSON.cpp | 2 +- .../System/StorageSystemFunctions.cpp | 2 +- src/TableFunctions/TableFunctionExplain.cpp | 2 +- 36 files changed, 88 insertions(+), 78 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index c98bee71d1a..dc1cebe9430 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -119,7 +119,6 @@ Checks: [ '-readability-named-parameter', '-readability-redundant-declaration', '-readability-simplify-boolean-expr', - '-readability-static-accessed-through-instance', '-readability-suspicious-call-argument', '-readability-uppercase-literal-suffix', '-readability-use-anyofallof', diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index b7c3c7f5c97..6c768799221 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -166,7 +166,7 @@ int DisksApp::main(const std::vector & /*args*/) { String config_path = config().getString("config-file", getDefaultConfigFileName()); ConfigProcessor config_processor(config_path, false, false); - config_processor.setConfigPath(fs::path(config_path).parent_path()); + ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); config().add(loaded_config.configuration.duplicate(), false, false); } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 8297fab5ed9..52d825f30e6 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -368,7 +368,7 @@ int KeeperClient::main(const std::vector & /* args */) DB::ConfigProcessor config_processor(config().getString("config-file", "config.xml")); /// This will handle a situation when clickhouse is running on the embedded config, but config.d folder is also present. - config_processor.registerEmbeddedConfig("config.xml", ""); + ConfigProcessor::registerEmbeddedConfig("config.xml", ""); auto clickhouse_config = config_processor.loadConfig(); Poco::Util::AbstractConfiguration::Keys keys; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 05c9830ee2c..72920fbd855 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -122,7 +122,7 @@ void LocalServer::initialize(Poco::Util::Application & self) { const auto config_path = config().getString("config-file", "config.xml"); ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(fs::path(config_path).parent_path()); + ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } diff --git a/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp index 8582c8c56fc..3d7d6eff608 100644 --- a/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp +++ b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp @@ -54,30 +54,30 @@ public: { const auto & value = columns[0]->getFloat64(row_num); const auto & time = columns[1]->getFloat64(row_num); - this->data(place).add(value, time, half_decay); + data(place).add(value, time, half_decay); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs), half_decay); + data(place).merge(data(rhs), half_decay); } void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - writeBinary(this->data(place).value, buf); - writeBinary(this->data(place).time, buf); + writeBinary(data(place).value, buf); + writeBinary(data(place).time, buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readBinary(this->data(place).value, buf); - readBinary(this->data(place).time, buf); + readBinary(data(place).value, buf); + readBinary(data(place).time, buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { auto & column = assert_cast &>(to); - column.getData().push_back(this->data(place).get(half_decay)); + column.getData().push_back(data(place).get(half_decay)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp index 882150325be..e1224fae2fb 100644 --- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp @@ -293,32 +293,32 @@ public: Float64 value = columns[0]->getFloat64(row_num); UInt8 is_second = columns[1]->getUInt(row_num); if (is_second) - this->data(place).addY(value, arena); + data(place).addY(value, arena); else - this->data(place).addX(value, arena); + data(place).addX(value, arena); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { - this->data(place).merge(this->data(rhs), arena); + data(place).merge(data(rhs), arena); } void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - this->data(place).write(buf); + data(place).write(buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { - this->data(place).read(buf, arena); + data(place).read(buf, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - if (!this->data(place).size_x || !this->data(place).size_y) + if (!data(place).size_x || !data(place).size_y) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName()); - auto [d_statistic, p_value] = this->data(place).getResult(alternative, method); + auto [d_statistic, p_value] = data(place).getResult(alternative, method); /// Because p-value is a probability. p_value = std::min(1.0, std::max(0.0, p_value)); diff --git a/src/AggregateFunctions/DDSketch.h b/src/AggregateFunctions/DDSketch.h index 65ce7322696..dae935bd43d 100644 --- a/src/AggregateFunctions/DDSketch.h +++ b/src/AggregateFunctions/DDSketch.h @@ -147,6 +147,8 @@ public: negative_store->merge(other.negative_store.get()); } + /// NOLINTBEGIN(readability-static-accessed-through-instance) + void serialize(WriteBuffer& buf) const { // Write the mapping @@ -201,6 +203,8 @@ public: count = static_cast(negative_store->count + zero_count + store->count); } + /// NOLINTEND(readability-static-accessed-through-instance) + private: std::unique_ptr mapping; std::unique_ptr store; diff --git a/src/AggregateFunctions/DDSketch/Store.h b/src/AggregateFunctions/DDSketch/Store.h index 428b2a6c9b8..f12c3875dcf 100644 --- a/src/AggregateFunctions/DDSketch/Store.h +++ b/src/AggregateFunctions/DDSketch/Store.h @@ -87,6 +87,8 @@ public: count += other->count; } + /// NOLINTBEGIN(readability-static-accessed-through-instance) + void serialize(WriteBuffer& buf) const { @@ -179,6 +181,8 @@ public: } } + /// NOLINTEND(readability-static-accessed-through-instance) + private: UInt32 chunk_size; DDSketchEncoding enc; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index fffb8f7f281..59c02f2b96f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5793,7 +5793,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi return result_projection_names; } - FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); + FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); /// NOLINT(readability-static-accessed-through-instance) bool is_executable_udf = true; IdentifierResolveScope::ResolvedFunctionsCache * function_cache = nullptr; @@ -5823,7 +5823,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi { std::vector possible_function_names; - auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context); + auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context); /// NOLINT(readability-static-accessed-through-instance) possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end()); function_names = UserDefinedSQLFunctionFactory::instance().getAllRegisteredNames(); @@ -5841,8 +5841,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi possible_function_names.push_back(name); } - NamePrompter<2> name_prompter; - auto hints = name_prompter.getHints(function_name, possible_function_names); + auto hints = NamePrompter<2>::getHints(function_name, possible_function_names); throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Function with name '{}' does not exists. In scope {}{}", diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 180942e6b83..f8607bf14c6 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -141,7 +141,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) async_callback(socket->impl()->sockfd(), connection_timeout, AsyncEventTimeoutType::CONNECT, description, AsyncTaskExecutor::READ | AsyncTaskExecutor::WRITE | AsyncTaskExecutor::ERROR); if (auto err = socket->impl()->socketError()) - socket->impl()->error(err); // Throws an exception + socket->impl()->error(err); // Throws an exception /// NOLINT(readability-static-accessed-through-instance) socket->setBlocking(true); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 0a4f90c2262..90ef974010c 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -940,7 +940,7 @@ void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & if (nested_node) { /// Find any leaf of Nested subcolumn. - const auto * leaf = subcolumns.findLeaf(nested_node, [&](const auto &) { return true; }); + const auto * leaf = Subcolumns::findLeaf(nested_node, [&](const auto &) { return true; }); assert(leaf); /// Recreate subcolumn with default values and the same sizes of arrays. @@ -983,7 +983,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons while (current_node) { /// Try to find the first Nested up to the current node. - const auto * node_nested = subcolumns.findParent(current_node, + const auto * node_nested = Subcolumns::findParent(current_node, [](const auto & candidate) { return candidate.isNested(); }); if (!node_nested) @@ -993,7 +993,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons /// for the last rows. /// If there are no leaves, skip current node and find /// the next node up to the current. - leaf = subcolumns.findLeaf(node_nested, + leaf = Subcolumns::findLeaf(node_nested, [&](const auto & candidate) { return candidate.data.size() > old_size; diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 16573e035e0..f3f6345a5b5 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -483,7 +483,7 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe ASSERT_TRUE(EqualByteContainers(test_sequence.data_type->getSizeOfValueInMemory(), source_data, decoded)); - const auto header_size = codec.getHeaderSize(); + const auto header_size = ICompressionCodec::getHeaderSize(); const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0); if (expected_compression_ratio) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index cc22db3969c..9f10069d452 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -665,7 +665,7 @@ void BaseDaemon::reloadConfiguration() */ config_path = config().getString("config-file", getDefaultConfigFileName()); ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(fs::path(config_path).parent_path()); + ConfigProcessor::setConfigPath(fs::path(config_path).parent_path()); loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true); if (last_configuration != nullptr) diff --git a/src/DataTypes/DataTypeDate32.cpp b/src/DataTypes/DataTypeDate32.cpp index 762552bcb4c..343e498d303 100644 --- a/src/DataTypes/DataTypeDate32.cpp +++ b/src/DataTypes/DataTypeDate32.cpp @@ -18,7 +18,7 @@ SerializationPtr DataTypeDate32::doGetDefaultSerialization() const Field DataTypeDate32::getDefault() const { - return -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); + return -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance) } void registerDataTypeDate32(DataTypeFactory & factory) diff --git a/src/DataTypes/Serializations/tests/gtest_json_parser.cpp b/src/DataTypes/Serializations/tests/gtest_json_parser.cpp index 9b0c8e44d02..1b5b02d579c 100644 --- a/src/DataTypes/Serializations/tests/gtest_json_parser.cpp +++ b/src/DataTypes/Serializations/tests/gtest_json_parser.cpp @@ -34,7 +34,7 @@ TEST(JSONDataParser, ReadJSON) JSONDataParser parser; ReadBufferFromString buf(json_bad); String res; - parser.readJSON(res, buf); + JSONDataParser::readJSON(res, buf); ASSERT_EQ(json1, res); } @@ -44,7 +44,7 @@ TEST(JSONDataParser, ReadJSON) JSONDataParser parser; ReadBufferFromString buf(json_bad); String res; - parser.readJSON(res, buf); + JSONDataParser::readJSON(res, buf); ASSERT_EQ(json2, res); } } diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 1e108b481ee..1fe369832ac 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -346,7 +346,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s } auto downloader_id = file_segment.getOrSetDownloader(); - if (downloader_id == file_segment.getCallerId()) + if (downloader_id == FileSegment::getCallerId()) { if (canStartFromCache(file_offset_of_buffer_end, file_segment)) { diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index cc2ec4fb045..5e072d406ad 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -279,7 +279,7 @@ struct ToDate32Transform32Or64Signed static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone) { - static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); + static const Int32 daynum_min_offset = -static_cast(DateLUTImpl::getDayNumOffsetEpoch()); if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) { @@ -1092,7 +1092,7 @@ struct ConvertThroughParsing { if constexpr (std::is_same_v) { - vec_to[i] = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); + vec_to[i] = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance) } else { diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp index e22cd6d0022..e6796874e50 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionFactory.cpp @@ -106,7 +106,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeRegistered(const ContextPtr & cont if (AggregateFunctionFactory::instance().hasNameOrAlias(function_name)) throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function '{}' already exists", function_name); - if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) + if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance) throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "User defined executable function '{}' already exists", function_name); validateFunction(assert_cast(create_function_query).function_core, function_name); @@ -118,7 +118,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeUnregistered(const ContextPtr & co AggregateFunctionFactory::instance().hasNameOrAlias(function_name)) throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop system function '{}'", function_name); - if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) + if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance) throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop user defined executable function '{}'", function_name); } diff --git a/src/IO/MMapReadBufferFromFileWithCache.cpp b/src/IO/MMapReadBufferFromFileWithCache.cpp index 68c0c7227ca..0cfb60d6527 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.cpp +++ b/src/IO/MMapReadBufferFromFileWithCache.cpp @@ -37,7 +37,7 @@ MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache( MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache( MMappedFileCache & cache, const std::string & file_name, size_t offset) { - mapped = cache.getOrSet(cache.hash(file_name, offset, -1), [&] + mapped = cache.getOrSet(MMappedFileCache::hash(file_name, offset, -1), [&] { return std::make_shared(file_name, offset); }); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 093c266c785..504b7257563 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1071,7 +1071,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & auto current_context = data.getContext(); - if (UserDefinedExecutableFunctionFactory::instance().has(node.name, current_context)) + if (UserDefinedExecutableFunctionFactory::instance().has(node.name, current_context)) /// NOLINT(readability-static-accessed-through-instance) { Array parameters; if (node.parameters) @@ -1087,7 +1087,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & } } - function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(node.name, current_context, parameters); + function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(node.name, current_context, parameters); /// NOLINT(readability-static-accessed-through-instance) } if (!function_builder) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index bd78c7a1bc1..2db07bb77f6 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1056,7 +1056,7 @@ void NO_INLINE Aggregator::executeImplBatch( /// During processing of row #i we will prefetch HashTable cell for row #(i + prefetch_look_ahead). PrefetchingHelper prefetching; - size_t prefetch_look_ahead = prefetching.getInitialLookAheadValue(); + size_t prefetch_look_ahead = PrefetchingHelper::getInitialLookAheadValue(); /// Optimization for special case when there are no aggregate functions. if (params.aggregates_size == 0) @@ -1077,7 +1077,7 @@ void NO_INLINE Aggregator::executeImplBatch( { if constexpr (prefetch && HasPrefetchMemberFunc) { - if (i == row_begin + prefetching.iterationsToMeasure()) + if (i == row_begin + PrefetchingHelper::iterationsToMeasure()) prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); if (i + prefetch_look_ahead < row_end) @@ -1163,7 +1163,7 @@ void NO_INLINE Aggregator::executeImplBatch( if constexpr (prefetch && HasPrefetchMemberFunc) { - if (i == key_start + prefetching.iterationsToMeasure()) + if (i == key_start + PrefetchingHelper::iterationsToMeasure()) prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); if (i + prefetch_look_ahead < row_end) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 7acaf95becc..2115dc57126 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -56,7 +56,7 @@ InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextP BlockIO InterpreterAlterQuery::execute() { - FunctionNameNormalizer().visit(query_ptr.get()); + FunctionNameNormalizer::visit(query_ptr.get()); const auto & alter = query_ptr->as(); if (alter.alter_object == ASTAlterQuery::AlterObjectType::DATABASE) { @@ -131,7 +131,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) if (modify_query) { // Expand CTE before filling default database - ApplyWithSubqueryVisitor().visit(*modify_query); + ApplyWithSubqueryVisitor::visit(*modify_query); } /// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc. diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 6045b5d2e24..a439cb672c8 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes BlockIO InterpreterCreateIndexQuery::execute() { - FunctionNameNormalizer().visit(query_ptr.get()); + FunctionNameNormalizer::visit(query_ptr.get()); auto current_context = getContext(); const auto & create_index = query_ptr->as(); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7c3bed7388c..c0da510a68b 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1114,7 +1114,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) // Table SQL definition is available even if the table is detached (even permanently) auto query = database->getCreateTableQuery(create.getTable(), getContext()); - FunctionNameNormalizer().visit(query.get()); + FunctionNameNormalizer::visit(query.get()); auto create_query = query->as(); if (!create.is_dictionary && create_query.is_dictionary) @@ -1184,7 +1184,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.select && create.isView()) { // Expand CTE before filling default database - ApplyWithSubqueryVisitor().visit(*create.select); + ApplyWithSubqueryVisitor::visit(*create.select); AddDefaultDatabaseVisitor visitor(getContext(), current_database); visitor.visit(*create.select); } @@ -1763,7 +1763,7 @@ BlockIO InterpreterCreateQuery::executeQueryOnCluster(ASTCreateQuery & create) BlockIO InterpreterCreateQuery::execute() { - FunctionNameNormalizer().visit(query_ptr.get()); + FunctionNameNormalizer::visit(query_ptr.get()); auto & create = query_ptr->as(); bool is_create_database = create.database && !create.table; diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 8f11754b3be..d1520c92dbc 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -168,7 +168,7 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) } /// Choose data structure to use for the set. - data.init(data.chooseMethod(key_columns, key_sizes)); + data.init(SetVariants::chooseMethod(key_columns, key_sizes)); } void Set::fillSetElements() diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index a341dae32fa..7b979088170 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -144,7 +144,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) } else { - FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function->name, context); + FunctionOverloadResolverPtr function_builder = UserDefinedExecutableFunctionFactory::instance().tryGet(function->name, context); /// NOLINT(readability-static-accessed-through-instance) if (!function_builder) function_builder = function_factory.get(function->name, context); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ea2f69bd2b1..c11ddc3467a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -923,7 +923,7 @@ static std::tuple executeQueryImpl( /// Propagate WITH statement to children ASTSelect. if (settings.enable_global_with_statement) { - ApplyWithGlobalVisitor().visit(ast); + ApplyWithGlobalVisitor::visit(ast); } { diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index 09afb01d7ff..41191ba1605 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -364,7 +364,7 @@ TEST_F(FileCacheTest, LRUPolicy) std::cerr << "Step 1\n"; auto cache = DB::FileCache("1", settings); cache.initialize(); - auto key = cache.createKeyForPath("key1"); + auto key = DB::FileCache::createKeyForPath("key1"); auto get_or_set = [&](size_t offset, size_t size) { @@ -728,7 +728,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache2 = DB::FileCache("2", settings); cache2.initialize(); - auto key = cache2.createKeyForPath("key1"); + auto key = DB::FileCache::createKeyForPath("key1"); /// Get [2, 29] assertEqual( @@ -747,7 +747,7 @@ TEST_F(FileCacheTest, LRUPolicy) fs::create_directories(settings2.base_path); auto cache2 = DB::FileCache("3", settings2); cache2.initialize(); - auto key = cache2.createKeyForPath("key1"); + auto key = DB::FileCache::createKeyForPath("key1"); /// Get [0, 24] assertEqual( @@ -762,7 +762,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache = FileCache("4", settings); cache.initialize(); - const auto key = cache.createKeyForPath("key10"); + const auto key = FileCache::createKeyForPath("key10"); const auto key_path = cache.getKeyPath(key, user); cache.removeAllReleasable(user.user_id); @@ -786,7 +786,7 @@ TEST_F(FileCacheTest, LRUPolicy) auto cache = DB::FileCache("5", settings); cache.initialize(); - const auto key = cache.createKeyForPath("key10"); + const auto key = FileCache::createKeyForPath("key10"); const auto key_path = cache.getKeyPath(key, user); cache.removeAllReleasable(user.user_id); @@ -823,7 +823,7 @@ TEST_F(FileCacheTest, writeBuffer) segment_settings.kind = FileSegmentKind::Temporary; segment_settings.unbounded = true; - auto cache_key = cache.createKeyForPath(key); + auto cache_key = FileCache::createKeyForPath(key); auto holder = cache.set(cache_key, 0, 3, segment_settings, user); /// The same is done in TemporaryDataOnDisk::createStreamToCacheFile. std::filesystem::create_directories(cache.getKeyPath(cache_key, user)); @@ -949,7 +949,7 @@ TEST_F(FileCacheTest, temporaryData) const auto user = FileCache::getCommonUser(); auto tmp_data_scope = std::make_shared(nullptr, &file_cache, TemporaryDataOnDiskSettings{}); - auto some_data_holder = file_cache.getOrSet(file_cache.createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user); + auto some_data_holder = file_cache.getOrSet(FileCache::createKeyForPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user); { ASSERT_EQ(some_data_holder->size(), 5); @@ -1199,7 +1199,7 @@ TEST_F(FileCacheTest, SLRUPolicy) { auto cache = DB::FileCache(std::to_string(++file_cache_name), settings); cache.initialize(); - auto key = cache.createKeyForPath("key1"); + auto key = FileCache::createKeyForPath("key1"); auto add_range = [&](size_t offset, size_t size) { diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index c5862b82f34..f794ad336e2 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -53,6 +53,8 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p return path.replace_filename(ss.str()); } +/// NOLINTBEGIN(readability-static-accessed-through-instance) + void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { auto current_logger = config.getString("logger", ""); @@ -393,6 +395,8 @@ void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Log } } +/// NOLINTEND(readability-static-accessed-through-instance) + void Loggers::closeLogs(Poco::Logger & logger) { if (log_file) diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index d175ec8d44b..0865ec8c0bb 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -86,7 +86,7 @@ static const Graphite::Pattern undef_pattern = .regexp_str = "", .function = nullptr, .retentions = Graphite::Retentions(), - .type = undef_pattern.TypeUndef, + .type = Graphite::Pattern::TypeUndef, }; inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, std::string_view path) @@ -118,18 +118,18 @@ Graphite::RollupRule selectPatternForPath( if (!pattern.regexp) { /// Default pattern - if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) + if (first_match->type == Graphite::Pattern::TypeUndef && pattern.type == Graphite::Pattern::TypeAll) { /// There is only default pattern for both retention and aggregation return {&pattern, &pattern}; } if (pattern.type != first_match->type) { - if (first_match->type == first_match->TypeRetention) + if (first_match->type == Graphite::Pattern::TypeRetention) { return {first_match, &pattern}; } - if (first_match->type == first_match->TypeAggregation) + if (first_match->type == Graphite::Pattern::TypeAggregation) { return {&pattern, first_match}; } @@ -140,23 +140,23 @@ Graphite::RollupRule selectPatternForPath( if (pattern.regexp->match(path.data(), path.size())) { /// General pattern with matched path - if (pattern.type == pattern.TypeAll) + if (pattern.type == Graphite::Pattern::TypeAll) { /// Only for not default patterns with both function and retention parameters return {&pattern, &pattern}; } - if (first_match->type == first_match->TypeUndef) + if (first_match->type == Graphite::Pattern::TypeUndef) { first_match = &pattern; continue; } if (pattern.type != first_match->type) { - if (first_match->type == first_match->TypeRetention) + if (first_match->type == Graphite::Pattern::TypeRetention) { return {first_match, &pattern}; } - if (first_match->type == first_match->TypeAggregation) + if (first_match->type == Graphite::Pattern::TypeAggregation) { return {&pattern, first_match}; } @@ -415,24 +415,24 @@ static const Pattern & appendGraphitePattern( if (!pattern.function) { - pattern.type = pattern.TypeRetention; + pattern.type = Graphite::Pattern::TypeRetention; } else if (pattern.retentions.empty()) { - pattern.type = pattern.TypeAggregation; + pattern.type = Graphite::Pattern::TypeAggregation; } else { - pattern.type = pattern.TypeAll; + pattern.type = Graphite::Pattern::TypeAll; } - if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll + if (pattern.type & Graphite::Pattern::TypeAggregation) /// TypeAggregation or TypeAll if (pattern.function->allocatesMemoryInArena()) throw Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Aggregate function {} isn't supported in GraphiteMergeTree", pattern.function->getName()); /// retention should be in descending order of age. - if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll + if (pattern.type & Graphite::Pattern::TypeRetention) /// TypeRetention or TypeAll ::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions); patterns.emplace_back(pattern); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7b0cfdf6a6c..f24b2d4f278 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -467,7 +467,7 @@ void StorageFileLog::openFilesAndSetPos() auto & reader = file_ctx.reader.value(); assertStreamGood(reader); - reader.seekg(0, reader.end); + reader.seekg(0, reader.end); /// NOLINT(readability-static-accessed-through-instance) assertStreamGood(reader); auto file_end = reader.tellg(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 461d9a31eaa..a301c229c77 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2994,7 +2994,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context commands.apply(new_metadata, local_context); - if (commands.hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) + if (AlterCommands::hasInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index a0c4156a704..447fd87cdc9 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -116,7 +116,7 @@ StorageDictionary::StorageDictionary( : StorageDictionary( table_id, table_id.getFullNameNotQuoted(), - context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration), + context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration), /// NOLINT(readability-static-accessed-through-instance) dictionary_configuration->getString("dictionary.comment", ""), Location::SameDatabaseAndNameAsDictionary, context_) diff --git a/src/Storages/StorageFuzzJSON.cpp b/src/Storages/StorageFuzzJSON.cpp index 918f54b1672..fbfc67f4c7c 100644 --- a/src/Storages/StorageFuzzJSON.cpp +++ b/src/Storages/StorageFuzzJSON.cpp @@ -437,7 +437,7 @@ void fuzzJSONObject( bool first = true; for (const auto & ptr : node_list) { - if (node_count >= config.value_number_limit) + if (node_count >= StorageFuzzJSON::Configuration::value_number_limit) break; WriteBufferFromOwnString child_out; diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index 967132e4d4a..c5c91294801 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -179,7 +179,7 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c } const auto & user_defined_executable_functions_factory = UserDefinedExecutableFunctionFactory::instance(); - const auto & user_defined_executable_functions_names = user_defined_executable_functions_factory.getRegisteredNames(context); + const auto & user_defined_executable_functions_names = user_defined_executable_functions_factory.getRegisteredNames(context); /// NOLINT(readability-static-accessed-through-instance) for (const auto & function_name : user_defined_executable_functions_names) { fillRow(res_columns, function_name, 0, {0}, "", FunctionOrigin::EXECUTABLE_USER_DEFINED, user_defined_executable_functions_factory); diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 8607597fa67..df2835dd630 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -136,7 +136,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt ColumnsDescription TableFunctionExplain::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { - Block sample_block = getInterpreter(context).getSampleBlock(query->as()->getKind()); + Block sample_block = getInterpreter(context).getSampleBlock(query->as()->getKind()); /// NOLINT(readability-static-accessed-through-instance) ColumnsDescription columns_description; for (const auto & column : sample_block.getColumnsWithTypeAndName()) columns_description.add(ColumnDescription(column.name, column.type)); From a4bec47a44ebb257f298f0254be0c2a7c3f62a95 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 7 Apr 2024 14:05:08 +0200 Subject: [PATCH 170/243] fix tests --- tests/queries/0_stateless/01056_window_view_proc_hop_watch.py | 2 ++ .../01059_window_view_event_hop_watch_strict_asc.py | 2 ++ .../0_stateless/01062_window_view_event_hop_watch_asc.py | 2 ++ .../0_stateless/01065_window_view_event_hop_watch_bounded.py | 2 ++ tests/queries/0_stateless/01070_window_view_watch_events.py | 2 ++ .../0_stateless/01078_window_view_alter_query_watch.py | 4 ++++ tests/queries/0_stateless/01082_window_view_watch_limit.py | 2 ++ 7 files changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 4c3e3ead455..2db14fcdddf 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -28,6 +28,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01056_window_view_proc_hop_watch") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py index 9adff06442e..2323ee5c838 100755 --- a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py +++ b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py @@ -26,6 +26,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client1.send("SET window_view_heartbeat_interval = 1") client1.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS db_01059_event_hop_watch_strict_asc") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index bb40b1df2f0..db9e8cef6c5 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -28,6 +28,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01062_window_view_event_hop_watch_asc") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py index 7f00130b184..b8d5ff02d37 100755 --- a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py +++ b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py @@ -27,6 +27,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send( "CREATE DATABASE IF NOT EXISTS 01065_window_view_event_hop_watch_bounded" diff --git a/tests/queries/0_stateless/01070_window_view_watch_events.py b/tests/queries/0_stateless/01070_window_view_watch_events.py index 8aeff041cc1..1cf7678a014 100755 --- a/tests/queries/0_stateless/01070_window_view_watch_events.py +++ b/tests/queries/0_stateless/01070_window_view_watch_events.py @@ -28,6 +28,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01070_window_view_watch_events") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01078_window_view_alter_query_watch.py b/tests/queries/0_stateless/01078_window_view_alter_query_watch.py index c32e508c5a5..3f3dfe0cda8 100755 --- a/tests/queries/0_stateless/01078_window_view_alter_query_watch.py +++ b/tests/queries/0_stateless/01078_window_view_alter_query_watch.py @@ -28,10 +28,14 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client3.send("SET allow_experimental_window_view = 1") client3.expect(prompt) client3.send("SET window_view_heartbeat_interval = 1") client3.expect(prompt) + client3.send("SET allow_experimental_analyzer = 0") + client3.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01078_window_view_alter_query_watch") client1.expect(prompt) diff --git a/tests/queries/0_stateless/01082_window_view_watch_limit.py b/tests/queries/0_stateless/01082_window_view_watch_limit.py index 12c8d295591..9938ebcab98 100755 --- a/tests/queries/0_stateless/01082_window_view_watch_limit.py +++ b/tests/queries/0_stateless/01082_window_view_watch_limit.py @@ -27,6 +27,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE IF NOT EXISTS 01082_window_view_watch_limit") client1.expect(prompt) From bbe8e2d751e2445afc07d97e18625d53868b2235 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 7 Apr 2024 20:35:27 +0300 Subject: [PATCH 171/243] Arrow schema to ClickHouse schema Nullable fix --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 365 ++++++++++++------ .../Formats/Impl/ArrowColumnToCHColumn.h | 23 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 4 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- .../DataLakes/DeltaLakeMetadataParser.cpp | 3 +- 6 files changed, 264 insertions(+), 135 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 206e244c75f..fc9a827be66 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -86,7 +86,7 @@ Chunk ArrowBlockInputFormat::read() /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, (*table_result)->num_rows(), block_missing_values_ptr); + res = arrow_column_to_ch_column->arrowTableToCHChunk(*table_result, (*table_result)->num_rows(), block_missing_values_ptr); /// There is no easy way to get original record batch size from Arrow metadata. /// Let's just use the number of bytes read from read buffer. diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 65704c85026..ec2d17d73cb 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -78,7 +78,7 @@ namespace ErrorCodes /// Inserts numeric data right into internal column data to reduce an overhead template > -static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithNumericData(const std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared>(); auto internal_column = internal_type->createColumn(); @@ -103,7 +103,7 @@ static ColumnWithTypeAndName readColumnWithNumericData(std::shared_ptr -static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithStringData(const std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); auto internal_column = internal_type->createColumn(); @@ -147,7 +147,7 @@ static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithFixedStringData(const std::shared_ptr & arrow_column, const String & column_name) { const auto * fixed_type = assert_cast(arrow_column->type().get()); size_t fixed_len = fixed_type->byte_width(); @@ -166,7 +166,7 @@ static ColumnWithTypeAndName readColumnWithFixedStringData(std::shared_ptr -static ColumnWithTypeAndName readColumnWithBigIntegerFromFixedBinaryData(std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & column_type) +static ColumnWithTypeAndName readColumnWithBigIntegerFromFixedBinaryData(const std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & column_type) { const auto * fixed_type = assert_cast(arrow_column->type().get()); size_t fixed_len = fixed_type->byte_width(); @@ -193,7 +193,7 @@ static ColumnWithTypeAndName readColumnWithBigIntegerFromFixedBinaryData(std::sh } template -static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData(std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & column_type) +static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData(const std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & column_type) { size_t total_size = 0; for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) @@ -229,7 +229,7 @@ static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData(std::shared_p return {std::move(internal_column), column_type, column_name}; } -static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithBooleanData(const std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = DataTypeFactory::instance().get("Bool"); auto internal_column = internal_type->createColumn(); @@ -248,7 +248,7 @@ static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name, +static ColumnWithTypeAndName readColumnWithDate32Data(const std::shared_ptr & arrow_column, const String & column_name, const DataTypePtr & type_hint, FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior) { DataTypePtr internal_type; @@ -310,7 +310,7 @@ static ColumnWithTypeAndName readColumnWithDate32Data(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithDate64Data(const std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); auto internal_column = internal_type->createColumn(); @@ -329,7 +329,7 @@ static ColumnWithTypeAndName readColumnWithDate64Data(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithTimestampData(const std::shared_ptr & arrow_column, const String & column_name) { const auto & arrow_type = static_cast(*(arrow_column->type())); const UInt8 scale = arrow_type.unit() * 3; @@ -350,7 +350,7 @@ static ColumnWithTypeAndName readColumnWithTimestampData(std::shared_ptr -static ColumnWithTypeAndName readColumnWithTimeData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithTimeData(const std::shared_ptr & arrow_column, const String & column_name) { const auto & arrow_type = static_cast(*(arrow_column->type())); const UInt8 scale = arrow_type.unit() * 3; @@ -373,18 +373,18 @@ static ColumnWithTypeAndName readColumnWithTimeData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithTime32Data(const std::shared_ptr & arrow_column, const String & column_name) { return readColumnWithTimeData(arrow_column, column_name); } -static ColumnWithTypeAndName readColumnWithTime64Data(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithTime64Data(const std::shared_ptr & arrow_column, const String & column_name) { return readColumnWithTimeData(arrow_column, column_name); } template -static ColumnWithTypeAndName readColumnWithDecimalDataImpl(std::shared_ptr & arrow_column, const String & column_name, DataTypePtr internal_type) +static ColumnWithTypeAndName readColumnWithDecimalDataImpl(const std::shared_ptr & arrow_column, const String & column_name, DataTypePtr internal_type) { auto internal_column = internal_type->createColumn(); auto & column = assert_cast &>(*internal_column); @@ -403,7 +403,7 @@ static ColumnWithTypeAndName readColumnWithDecimalDataImpl(std::shared_ptr -static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readColumnWithDecimalData(const std::shared_ptr & arrow_column, const String & column_name) { const auto * arrow_decimal_type = static_cast(arrow_column->type().get()); size_t precision = arrow_decimal_type->precision(); @@ -418,7 +418,7 @@ static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr & arrow_column) +static ColumnPtr readByteMapFromArrowColumn(const std::shared_ptr & arrow_column) { if (!arrow_column->null_count()) return ColumnUInt8::create(arrow_column->length(), 0); @@ -453,7 +453,7 @@ struct ArrowOffsetArray }; template -static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr & arrow_column) +static ColumnPtr readOffsetsFromArrowListColumn(const std::shared_ptr & arrow_column) { auto offsets_column = ColumnUInt64::create(); ColumnArray::Offsets & offsets_data = assert_cast &>(*offsets_column).getData(); @@ -463,7 +463,7 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr(*(arrow_column->chunk(chunk_i))); auto arrow_offsets_array = list_chunk.offsets(); - auto & arrow_offsets = dynamic_cast::type &>(*arrow_offsets_array); + auto & arrow_offsets = dynamic_cast::type &>(*arrow_offsets_array); /* * CH uses element size as "offsets", while arrow uses actual offsets as offsets. @@ -620,7 +620,7 @@ static ColumnPtr readColumnWithIndexesData(std::shared_ptr } template -static std::shared_ptr getNestedArrowColumn(std::shared_ptr & arrow_column) +static std::shared_ptr getNestedArrowColumn(const std::shared_ptr & arrow_column) { arrow::ArrayVector array_vector; array_vector.reserve(arrow_column->num_chunks()); @@ -648,7 +648,7 @@ static std::shared_ptr getNestedArrowColumn(std::shared_ptr return std::make_shared(array_vector); } -static ColumnWithTypeAndName readIPv6ColumnFromBinaryData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readIPv6ColumnFromBinaryData(const std::shared_ptr & arrow_column, const String & column_name) { size_t total_size = 0; for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) @@ -684,7 +684,7 @@ static ColumnWithTypeAndName readIPv6ColumnFromBinaryData(std::shared_ptr & arrow_column, const String & column_name) +static ColumnWithTypeAndName readIPv4ColumnWithInt32Data(const std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = std::make_shared(); auto internal_column = internal_type->createColumn(); @@ -705,35 +705,31 @@ static ColumnWithTypeAndName readIPv4ColumnWithInt32Data(std::shared_ptr & arrow_column, - const std::string & column_name, - const std::string & format_name, - bool is_nullable, - std::unordered_map & dictionary_infos, - bool allow_null_type, - bool skip_columns_with_unsupported_types, - bool & skipped, - FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore, - DataTypePtr type_hint = nullptr, - bool is_map_nested = false) +struct ReadColumnFromArrowColumnSettings { - if (!is_nullable && (arrow_column->null_count() || (type_hint && type_hint->isNullable())) && arrow_column->type()->id() != arrow::Type::LIST - && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT && - arrow_column->type()->id() != arrow::Type::DICTIONARY) - { - DataTypePtr nested_type_hint; - if (type_hint) - nested_type_hint = removeNullable(type_hint); - auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); - if (skipped) - return {}; - auto nullmap_column = readByteMapFromArrowColumn(arrow_column); - auto nullable_type = std::make_shared(std::move(nested_column.type)); - auto nullable_column = ColumnNullable::create(nested_column.column, nullmap_column); - return {std::move(nullable_column), std::move(nullable_type), column_name}; - } + std::string format_name; + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior; + bool allow_arrow_null_type; + bool skip_columns_with_unsupported_types; +}; +static ColumnWithTypeAndName readColumnFromArrowColumn( + const std::shared_ptr & arrow_column, + std::string column_name, + std::unordered_map dictionary_infos, + DataTypePtr type_hint, + bool is_nullable_column, + bool is_map_nested_column, + const ReadColumnFromArrowColumnSettings & settings); + +static ColumnWithTypeAndName readNonNullableColumnFromArrowColumn( + const std::shared_ptr & arrow_column, + std::string column_name, + std::unordered_map dictionary_infos, + DataTypePtr type_hint, + bool is_map_nested_column, + const ReadColumnFromArrowColumnSettings & settings) +{ switch (arrow_column->type()->id()) { case arrow::Type::STRING: @@ -790,7 +786,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::BOOL: return readColumnWithBooleanData(arrow_column, column_name); case arrow::Type::DATE32: - return readColumnWithDate32Data(arrow_column, column_name, type_hint, date_time_overflow_behavior); + return readColumnWithDate32Data(arrow_column, column_name, type_hint, settings.date_time_overflow_behavior); case arrow::Type::DATE64: return readColumnWithDate64Data(arrow_column, column_name); // ClickHouse writes Date as arrow UINT16 and DateTime as arrow UINT32, @@ -837,9 +833,16 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( key_type_hint = map_type_hint->getKeyType(); } } + auto arrow_nested_column = getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint, true); - if (skipped) + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, + column_name, + dictionary_infos, + nested_type_hint, + false /*is_nullable_column*/, + true /*is_map_nested_column*/, + settings); + if (!nested_column.column) return {}; auto offsets_column = readOffsetsFromArrowListColumn(arrow_column); @@ -866,7 +869,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( case arrow::Type::LIST: case arrow::Type::LARGE_LIST: { - bool is_large = arrow_column->type()->id() == arrow::Type::LARGE_LIST; + bool is_large_list = arrow_column->type()->id() == arrow::Type::LARGE_LIST; DataTypePtr nested_type_hint; if (type_hint) { @@ -874,12 +877,33 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( if (array_type_hint) nested_type_hint = array_type_hint->getNestedType(); } - auto arrow_nested_column = is_large ? getNestedArrowColumn(arrow_column) : getNestedArrowColumn(arrow_column); - auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); - if (skipped) + + bool is_nested_nullable_column = false; + if (is_large_list) + { + auto * arrow_large_list_type = assert_cast(arrow_column->type().get()); + is_nested_nullable_column = arrow_large_list_type->value_field()->nullable(); + } + else + { + auto * arrow_list_type = assert_cast(arrow_column->type().get()); + is_nested_nullable_column = arrow_list_type->value_field()->nullable(); + } + + auto arrow_nested_column = is_large_list ? getNestedArrowColumn(arrow_column) : getNestedArrowColumn(arrow_column); + auto nested_column = readColumnFromArrowColumn(arrow_nested_column, + column_name, + dictionary_infos, + nested_type_hint, + is_nested_nullable_column, + false /*is_map_nested_column*/, + settings); + if (!nested_column.column) return {}; - auto offsets_column = is_large ? readOffsetsFromArrowListColumn(arrow_column) : readOffsetsFromArrowListColumn(arrow_column); + + auto offsets_column = is_large_list ? readOffsetsFromArrowListColumn(arrow_column) : readOffsetsFromArrowListColumn(arrow_column); auto array_column = ColumnArray::create(nested_column.column, offsets_column); + DataTypePtr array_type; /// If type hint is Nested, we should return Nested type, /// because we differentiate Nested and simple Array(Tuple) @@ -913,11 +937,13 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( for (int i = 0; i != arrow_struct_type->num_fields(); ++i) { - auto field_name = arrow_struct_type->field(i)->name(); + const auto & field = arrow_struct_type->field(i); + const auto & field_name = field->name(); + DataTypePtr nested_type_hint; if (tuple_type_hint) { - if (tuple_type_hint->haveExplicitNames() && !is_map_nested) + if (tuple_type_hint->haveExplicitNames() && !is_map_nested_column) { auto pos = tuple_type_hint->tryGetPositionByName(field_name); if (pos) @@ -926,13 +952,21 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( else if (size_t(i) < tuple_type_hint->getElements().size()) nested_type_hint = tuple_type_hint->getElement(i); } + auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - auto element = readColumnFromArrowColumn(nested_arrow_column, field_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior, nested_type_hint); - if (skipped) + auto column_with_type_and_name = readColumnFromArrowColumn(nested_arrow_column, + field_name, + dictionary_infos, + nested_type_hint, + field->nullable(), + false /*is_map_nested_column*/, + settings); + if (!column_with_type_and_name.column) return {}; - tuple_elements.emplace_back(std::move(element.column)); - tuple_types.emplace_back(std::move(element.type)); - tuple_names.emplace_back(std::move(element.name)); + + tuple_elements.emplace_back(std::move(column_with_type_and_name.column)); + tuple_types.emplace_back(std::move(column_with_type_and_name.type)); + tuple_names.emplace_back(std::move(column_with_type_and_name.name)); } auto tuple_column = ColumnTuple::create(std::move(tuple_elements)); @@ -953,8 +987,19 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( arrow::DictionaryArray & dict_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); dict_array.emplace_back(dict_chunk.dictionary()); } + auto arrow_dict_column = std::make_shared(dict_array); - auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, date_time_overflow_behavior); + auto dict_column = readColumnFromArrowColumn(arrow_dict_column, + column_name, + dictionary_infos, + nullptr /*nested_type_hint*/, + false /*is_nullable_column*/, + false /*is_map_nested_column*/, + settings); + + if (!dict_column.column) + return {}; + for (size_t i = 0; i != dict_column.column->size(); ++i) { if (dict_column.column->isDefaultAt(i)) @@ -963,6 +1008,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( break; } } + auto lc_type = std::make_shared(is_lc_nullable ? makeNullable(dict_column.type) : dict_column.type); auto tmp_lc_column = lc_type->createColumn(); auto tmp_dict_column = IColumn::mutate(assert_cast(tmp_lc_column.get())->getDictionaryPtr()); @@ -1002,7 +1048,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( // TODO: read UUID as a string? case arrow::Type::NA: { - if (allow_null_type) + if (settings.allow_arrow_null_type) { auto type = std::make_shared(); auto column = ColumnNothing::create(arrow_column->length()); @@ -1012,11 +1058,8 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( } default: { - if (skip_columns_with_unsupported_types) - { - skipped = true; + if (settings.skip_columns_with_unsupported_types) return {}; - } throw Exception( ErrorCodes::UNKNOWN_TYPE, @@ -1024,14 +1067,59 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( "If it happens during schema inference and you want to skip columns with " "unsupported types, you can enable setting input_format_{}" "_skip_columns_with_unsupported_types_in_schema_inference", - format_name, + settings.format_name, arrow_column->type()->name(), column_name, - boost::algorithm::to_lower_copy(format_name)); + boost::algorithm::to_lower_copy(settings.format_name)); } } } +static ColumnWithTypeAndName readColumnFromArrowColumn( + const std::shared_ptr & arrow_column, + std::string column_name, + std::unordered_map dictionary_infos, + DataTypePtr type_hint, + bool is_nullable_column, + bool is_map_nested_column, + const ReadColumnFromArrowColumnSettings & settings) +{ + bool read_as_nullable_column = arrow_column->null_count() || is_nullable_column || (type_hint && type_hint->isNullable()); + if (read_as_nullable_column && + arrow_column->type()->id() != arrow::Type::LIST && + arrow_column->type()->id() != arrow::Type::LARGE_LIST && + arrow_column->type()->id() != arrow::Type::MAP && + arrow_column->type()->id() != arrow::Type::STRUCT && + arrow_column->type()->id() != arrow::Type::DICTIONARY) + { + DataTypePtr nested_type_hint; + if (type_hint) + nested_type_hint = removeNullable(type_hint); + + auto nested_column = readNonNullableColumnFromArrowColumn(arrow_column, + column_name, + dictionary_infos, + nested_type_hint, + is_map_nested_column, + settings); + + if (!nested_column.column) + return {}; + + auto nullmap_column = readByteMapFromArrowColumn(arrow_column); + auto nullable_type = std::make_shared(std::move(nested_column.type)); + auto nullable_column = ColumnNullable::create(nested_column.column, nullmap_column); + + return {std::move(nullable_column), std::move(nullable_type), column_name}; + } + + return readNonNullableColumnFromArrowColumn(arrow_column, + column_name, + dictionary_infos, + type_hint, + is_map_nested_column, + settings); +} // Creating CH header by arrow schema. Will be useful in task about inserting // data from file without knowing table structure. @@ -1042,44 +1130,56 @@ static void checkStatus(const arrow::Status & status, const String & column_name throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()}; } +/// Create empty arrow column using specified field +static std::shared_ptr createArrowColumn(const std::shared_ptr & field, const String & format_name) +{ + arrow::MemoryPool * pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); + checkStatus(status, field->name(), format_name); + + std::shared_ptr arrow_array; + status = array_builder->Finish(&arrow_array); + checkStatus(status, field->name(), format_name); + + return std::make_shared(arrow::ArrayVector{arrow_array}); +} Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( - const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types, const Block * hint_header, bool ignore_case) + const arrow::Schema & schema, + const std::string & format_name, + bool skip_columns_with_unsupported_types) { + ReadColumnFromArrowColumnSettings settings + { + .format_name = format_name, + .date_time_overflow_behavior = FormatSettings::DateTimeOverflowBehavior::Ignore, + .allow_arrow_null_type = false, + .skip_columns_with_unsupported_types = skip_columns_with_unsupported_types + }; + ColumnsWithTypeAndName sample_columns; - std::unordered_set nested_table_names; - if (hint_header) - nested_table_names = Nested::getAllTableNames(*hint_header, ignore_case); for (const auto & field : schema.fields()) { - if (hint_header && !hint_header->has(field->name(), ignore_case) - && !nested_table_names.contains(ignore_case ? boost::to_lower_copy(field->name()) : field->name())) - continue; - /// Create empty arrow column by it's type and convert it to ClickHouse column. - arrow::MemoryPool * pool = arrow::default_memory_pool(); - std::unique_ptr array_builder; - arrow::Status status = MakeBuilder(pool, field->type(), &array_builder); - checkStatus(status, field->name(), format_name); + auto arrow_column = createArrowColumn(field, format_name); - std::shared_ptr arrow_array; - status = array_builder->Finish(&arrow_array); - checkStatus(status, field->name(), format_name); - - arrow::ArrayVector array_vector = {arrow_array}; - auto arrow_column = std::make_shared(array_vector); std::unordered_map dict_infos; - bool skipped = false; - bool allow_null_type = false; - if (hint_header && hint_header->has(field->name()) && hint_header->getByName(field->name()).type->isNullable()) - allow_null_type = true; - ColumnWithTypeAndName sample_column = readColumnFromArrowColumn( - arrow_column, field->name(), format_name, false, dict_infos, allow_null_type, skip_columns_with_unsupported_types, skipped); - if (!skipped) + + auto sample_column = readColumnFromArrowColumn( + arrow_column, + field->name(), + dict_infos, + nullptr /*nested_type_hint*/, + field->nullable() /*is_nullable_column*/, + false /*is_map_nested_column*/, + settings); + + if (sample_column.column) sample_columns.emplace_back(std::move(sample_column)); } + return Block(std::move(sample_columns)); } @@ -1101,30 +1201,43 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( { } -void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values) +Chunk ArrowColumnToCHColumn::arrowTableToCHChunk(const std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values) { - NameToColumnPtr name_to_column_ptr; + NameToArrowColumn name_to_arrow_column; + for (auto column_name : table->ColumnNames()) { - std::shared_ptr arrow_column = table->GetColumnByName(column_name); + auto arrow_column = table->GetColumnByName(column_name); if (!arrow_column) throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column '{}' is duplicated", column_name); + auto arrow_field = table->schema()->GetFieldByName(column_name); + if (case_insensitive_matching) boost::to_lower(column_name); - name_to_column_ptr[std::move(column_name)] = arrow_column; + + name_to_arrow_column[std::move(column_name)] = {std::move(arrow_column), std::move(arrow_field)}; } - arrowColumnsToCHChunk(res, name_to_column_ptr, num_rows, block_missing_values); + return arrowColumnsToCHChunk(name_to_arrow_column, num_rows, block_missing_values); } -void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows, BlockMissingValues * block_missing_values) +Chunk ArrowColumnToCHColumn::arrowColumnsToCHChunk(const NameToArrowColumn & name_to_arrow_column, size_t num_rows, BlockMissingValues * block_missing_values) { - Columns columns_list; - columns_list.reserve(header.columns()); + ReadColumnFromArrowColumnSettings settings + { + .format_name = format_name, + .date_time_overflow_behavior = date_time_overflow_behavior, + .allow_arrow_null_type = true, + .skip_columns_with_unsupported_types = false + }; + + Columns columns; + columns.reserve(header.columns()); + std::unordered_map>> nested_tables; - bool skipped = false; - for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + + for (size_t column_i = 0, header_columns = header.columns(); column_i < header_columns; ++column_i) { const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); @@ -1133,15 +1246,17 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & boost::to_lower(search_column_name); ColumnWithTypeAndName column; - if (!name_to_column_ptr.contains(search_column_name)) + if (!name_to_arrow_column.contains(search_column_name)) { bool read_from_nested = false; + /// Check if it's a subcolumn from some struct. String nested_table_name = Nested::extractTableName(header_column.name); String search_nested_table_name = nested_table_name; if (case_insensitive_matching) boost::to_lower(search_nested_table_name); - if (name_to_column_ptr.contains(search_nested_table_name)) + + if (name_to_arrow_column.contains(search_nested_table_name)) { if (!nested_tables.contains(search_nested_table_name)) { @@ -1153,10 +1268,19 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & } auto nested_table_type = Nested::collect(nested_columns).front().type; - std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; - ColumnsWithTypeAndName cols = { - readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, - skipped, date_time_overflow_behavior, nested_table_type)}; + const auto & arrow_column = name_to_arrow_column.find(search_nested_table_name)->second; + + ColumnsWithTypeAndName cols = + { + readColumnFromArrowColumn(arrow_column.column, + nested_table_name, + dictionary_infos, + nested_table_type, + arrow_column.field->nullable() /*is_nullable_column*/, + false /*is_map_nested_column*/, + settings) + }; + BlockPtr block_ptr = std::make_shared(cols); auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; @@ -1180,7 +1304,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & column.name = header_column.name; column.type = header_column.type; column.column = header_column.column->cloneResized(num_rows); - columns_list.push_back(std::move(column.column)); + columns.push_back(std::move(column.column)); if (block_missing_values) block_missing_values->setBits(column_i, num_rows); continue; @@ -1189,9 +1313,14 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & } else { - auto arrow_column = name_to_column_ptr[search_column_name]; - column = readColumnFromArrowColumn( - arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, date_time_overflow_behavior, header_column.type); + const auto & arrow_column = name_to_arrow_column.find(search_column_name)->second; + column = readColumnFromArrowColumn(arrow_column.column, + header_column.name, + dictionary_infos, + header_column.type, + arrow_column.field->nullable(), + false /*is_map_nested_column*/, + settings); } if (null_as_default) @@ -1216,10 +1345,10 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & } column.type = header_column.type; - columns_list.push_back(std::move(column.column)); + columns.push_back(std::move(column.column)); } - res.setColumns(columns_list, num_rows); + return Chunk(std::move(columns), num_rows); } } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 079e0374917..27e9afdf763 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -19,8 +19,6 @@ class Chunk; class ArrowColumnToCHColumn { public: - using NameToColumnPtr = std::unordered_map>; - ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, @@ -30,18 +28,13 @@ public: bool case_insensitive_matching_ = false, bool is_stream_ = false); - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); + Chunk arrowTableToCHChunk(const std::shared_ptr & table, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); - void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); - - /// Transform arrow schema to ClickHouse header. If hint_header is provided, - /// we will skip columns in schema that are not in hint_header. + /// Transform arrow schema to ClickHouse header static Block arrowSchemaToCHHeader( const arrow::Schema & schema, const std::string & format_name, - bool skip_columns_with_unsupported_types = false, - const Block * hint_header = nullptr, - bool ignore_case = false); + bool skip_columns_with_unsupported_types = false); struct DictionaryInfo { @@ -52,6 +45,16 @@ public: private: + struct ArrowColumn + { + std::shared_ptr column; + std::shared_ptr field; + }; + + using NameToArrowColumn = std::unordered_map; + + Chunk arrowColumnsToCHChunk(const NameToArrowColumn & name_to_arrow_column, size_t num_rows, BlockMissingValues * block_missing_values); + const Block & header; const std::string format_name; /// If false, throw exception if some columns in header not exists in arrow table. diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index a41eacf26b7..aa83b87b2d2 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -71,12 +71,10 @@ Chunk ORCBlockInputFormat::read() approx_bytes_read_for_chunk = file_reader->GetRawORCReader()->getStripe(stripe_current)->getDataLength(); ++stripe_current; - Chunk res; /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows, block_missing_values_ptr); - return res; + return arrow_column_to_ch_column->arrowTableToCHChunk(table, num_rows, block_missing_values_ptr); } void ORCBlockInputFormat::resetParser() diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 62e576d4953..d41cb3447de 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -601,7 +601,7 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &res.block_missing_values : nullptr; - row_group_batch.arrow_column_to_ch_column->arrowTableToCHChunk(res.chunk, *tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); + res.chunk = row_group_batch.arrow_column_to_ch_column->arrowTableToCHChunk(*tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); lock.lock(); diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index 3584f137225..14a912a180d 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -282,11 +282,10 @@ struct DeltaLakeMetadataParser::Impl format_settings.date_time_overflow_behavior, /* case_insensitive_column_matching */false); - Chunk res; std::shared_ptr table; THROW_ARROW_NOT_OK(reader->ReadTable(&table)); - column_reader.arrowTableToCHChunk(res, table, reader->parquet_reader()->metadata()->num_rows()); + Chunk res = column_reader.arrowTableToCHChunk(table, reader->parquet_reader()->metadata()->num_rows()); const auto & res_columns = res.getColumns(); if (res_columns.size() != 2) From 1e05d9ed3f4a960ca6a219514fd56fa13c644efc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 7 Apr 2024 20:36:02 +0300 Subject: [PATCH 172/243] Added tests --- .../03036_parquet_arrow_nullable.reference | 40 ++++++++++++ .../03036_parquet_arrow_nullable.sh | 63 +++++++++++++++++++ 2 files changed, 103 insertions(+) create mode 100644 tests/queries/0_stateless/03036_parquet_arrow_nullable.reference create mode 100755 tests/queries/0_stateless/03036_parquet_arrow_nullable.sh diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference new file mode 100644 index 00000000000..8820bb7cb9f --- /dev/null +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.reference @@ -0,0 +1,40 @@ +Parquet +a UInt64 +a_nullable Nullable(UInt64) +Arrow +a UInt64 +a_nullable Nullable(UInt64) +Parquet +b Array(Nullable(UInt64)) +b_nullable Array(Nullable(UInt64)) +Arrow +b Array(Nullable(UInt64)) +b_nullable Array(Nullable(UInt64)) +Parquet +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +Arrow +c Tuple(\n a UInt64,\n b String) +c_nullable Tuple(\n a Nullable(UInt64),\n b Nullable(String)) +Parquet +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +Arrow +d Tuple(\n a UInt64,\n b Tuple(\n a UInt64,\n b String),\n d_nullable Tuple(\n a UInt64,\n b Tuple(\n a Nullable(UInt64),\n b Nullable(String)))) +Parquet +e Map(UInt64, Nullable(String)) +e_nullable Map(UInt64, Nullable(String)) +Arrow +e Map(UInt64, Nullable(String)) +e_nullable Map(UInt64, Nullable(String)) +Parquet +f Map(UInt64, Map(UInt64, Nullable(String))) +f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +Arrow +f Map(UInt64, Map(UInt64, Nullable(String))) +f_nullables Map(UInt64, Map(UInt64, Nullable(String))) +Parquet +g String +g_nullable Nullable(String) +Arrow +g LowCardinality(String) +g_nullable LowCardinality(String) diff --git a/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh new file mode 100755 index 00000000000..bdd641e2b94 --- /dev/null +++ b/tests/queries/0_stateless/03036_parquet_arrow_nullable.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.data + +formats="Parquet Arrow" + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, a_nullable Nullable(UInt64)', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('b Array(UInt64), b_nullable Array(Nullable(UInt64))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('c Tuple(a UInt64, b String), c_nullable Tuple(a Nullable(UInt64), b Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('d Tuple(a UInt64, b Tuple(a UInt64, b String), d_nullable Tuple(a UInt64, b Tuple(a Nullable(UInt64), b Nullable(String))))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('e Map(UInt64, String), e_nullable Map(UInt64, Nullable(String))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('f Map(UInt64, Map(UInt64, String)), f_nullables Map(UInt64, Map(UInt64, Nullable(String)))', 42) limit 10 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +for format in $formats +do + echo $format + $CLICKHOUSE_LOCAL -q "select * from generateRandom('g LowCardinality(String), g_nullable LowCardinality(Nullable(String))', 42) limit 10 settings output_format_arrow_low_cardinality_as_dictionary=1, allow_suspicious_low_cardinality_types=1 format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE') SETTINGS schema_inference_make_columns_nullable = 0" +done + +rm $DATA_FILE + From 373bf4968408e41392f5dc7e6791fc61143939f9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 7 Apr 2024 20:05:41 +0200 Subject: [PATCH 173/243] fix --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- .../queries/0_stateless/01069_window_view_proc_tumble_watch.py | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index f82f5b07910..04c26053dba 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1710,7 +1710,7 @@ void StorageWindowView::throwIfWindowViewIsDisabled(ContextPtr local_context) co { if (disabled_due_to_analyzer || (local_context && local_context->getSettingsRef().allow_experimental_analyzer)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Experimental WINDOW VIEW feature is not supported " - "with new infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); + "in the current infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); } void registerStorageWindowView(StorageFactory & factory) diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py index eb31b2ccbcf..21c2e831afc 100755 --- a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -28,6 +28,8 @@ with client(name="client1>", log=log) as client1, client( client1.expect(prompt) client2.send("SET allow_experimental_window_view = 1") client2.expect(prompt) + client2.send("SET allow_experimental_analyzer = 0") + client2.expect(prompt) client1.send("CREATE DATABASE 01069_window_view_proc_tumble_watch") client1.expect(prompt) From 4b5db7357caf36ca797293ef1ec81df7c492e3cb Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Sun, 7 Apr 2024 20:48:59 +0300 Subject: [PATCH 174/243] docs: predefined query handler example fix --- docs/en/interfaces/http.md | 18 ++++++++++-------- docs/ru/interfaces/http.md | 18 ++++++++++-------- docs/zh/interfaces/http.md | 21 ++++++++++++--------- 3 files changed, 32 insertions(+), 25 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 4eeb19cefcf..bba5cde16f1 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -507,16 +507,18 @@ Example: ``` xml - [^/]+)(/(?P[^/]+))?]]> + [^/]+)]]> GET TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> + [^/]+)]]> predefined_query_handler - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} + + SELECT name, value FROM system.settings + WHERE name IN ({name_1:String}, {name_2:String}) + @@ -524,13 +526,13 @@ Example: ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' -1 -max_final_threads 2 +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_final_threads' 'http://localhost:8123/query_param_with_url/max_threads?max_threads=1&max_final_threads=2' +max_final_threads 2 +max_threads 1 ``` :::note -In one `predefined_query_handler` only supports one `query` of an insert type. +In one `predefined_query_handler` only one `query` is supported. ::: ### dynamic_query_handler {#dynamic_query_handler} diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index be8cfbdda6c..5f11f1b430b 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -434,16 +434,18 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` xml - [^/]+)(/(?P[^/]+))?]]> + [^/]+)]]> GET TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> + [^/]+)]]> predefined_query_handler - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} + + SELECT name, value FROM system.settings + WHERE name IN ({name_1:String}, {name_2:String}) + @@ -451,13 +453,13 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' -1 -max_final_threads 2 +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_final_threads' 'http://localhost:8123/query_param_with_url/max_threads?max_threads=1&max_final_threads=2' +max_final_threads 2 +max_threads 1 ``` :::note Предупреждение -В одном `predefined_query_handler` поддерживается только один запрос типа `INSERT`. +В одном `predefined_query_handler` поддерживается только один запрос. ::: ### dynamic_query_handler {#dynamic_query_handler} diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index 84ca5ed0c47..f55cf41936f 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -427,29 +427,32 @@ $ curl -v 'http://localhost:8123/predefined_query' ``` xml - [^/]+)(/(?P[^/]+))?]]> - GET + [^/]+)]]> + GET TEST_HEADER_VALUE - [^/]+)(/(?P[^/]+))?]]> + [^/]+)]]> predefined_query_handler - SELECT value FROM system.settings WHERE name = {name_1:String} - SELECT name, value FROM system.settings WHERE name = {name_2:String} + + SELECT name, value FROM system.settings + WHERE name IN ({name_1:String}, {name_2:String}) + + ``` ``` bash -$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost:8123/query_param_with_url/1/max_threads/max_final_threads?max_threads=1&max_final_threads=2' -1 -max_final_threads 2 +$ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_final_threads' 'http://localhost:8123/query_param_with_url/max_threads?max_threads=1&max_final_threads=2' +max_final_threads 2 +max_threads 1 ``` :::warning -在一个`predefined_query_handler`中,只支持insert类型的一个`查询`。 +在一个`predefined_query_handler`中,只支持的一个`查询`。 ::: ### 动态查询 {#dynamic_query_handler} From f5e9a09d69ea0d1f961464e866c77a73c5c0e82e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 Apr 2024 05:20:09 +0200 Subject: [PATCH 175/243] Remove support for INSERT WATCH query --- src/Client/ClientBase.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 14 +++----------- src/Interpreters/executeQuery.cpp | 11 ----------- src/Parsers/ASTInsertQuery.cpp | 7 +------ src/Parsers/ASTInsertQuery.h | 2 -- src/Parsers/ParserInsertQuery.cpp | 15 +-------------- 6 files changed, 6 insertions(+), 45 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7a3192d1d9c..8107bd94394 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1964,7 +1964,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. - if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert_with_inlined_data) + if (insert && (!insert->select || input_function) && !is_async_insert_with_inlined_data) { if (input_function && insert->format.empty()) throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()"); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index fc58f7b5098..35ff65c2335 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -340,13 +340,10 @@ bool InterpreterInsertQuery::shouldAddSquashingFroStorage(const StoragePtr & tab { auto context_ptr = getContext(); const Settings & settings = context_ptr->getSettingsRef(); - const ASTInsertQuery * query = nullptr; - if (query_ptr) - query = query_ptr->as(); /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - return !(settings.distributed_foreground_insert && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch); + return !(settings.distributed_foreground_insert && table->isRemote()) && !async_insert && !no_squash; } Chain InterpreterInsertQuery::buildPreSinkChain( @@ -429,7 +426,7 @@ BlockIO InterpreterInsertQuery::execute() std::vector presink_chains; std::vector sink_chains; - if (!distributed_pipeline || query.watch) + if (!distributed_pipeline) { /// Number of streams works like this: /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever @@ -560,11 +557,6 @@ BlockIO InterpreterInsertQuery::execute() } } } - else if (query.watch) - { - InterpreterWatchQuery interpreter_watch{ query.watch, getContext() }; - pipeline = interpreter_watch.buildQueryPipeline(); - } ThreadGroupPtr running_group; if (current_thread) @@ -591,7 +583,7 @@ BlockIO InterpreterInsertQuery::execute() { res.pipeline = std::move(*distributed_pipeline); } - else if (query.select || query.watch) + else if (query.select) { const auto & header = presink_chains.at(0).getInputHeader(); auto actions_dag = ActionsDAG::makeConvertingActions( diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index ea2f69bd2b1..96a9c8d8c8e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -644,15 +644,6 @@ void logExceptionBeforeStart( } } -static void setQuerySpecificSettings(ASTPtr & ast, ContextMutablePtr context) -{ - if (auto * ast_insert_into = ast->as()) - { - if (ast_insert_into->watch) - context->setSetting("output_format_enable_streaming", 1); - } -} - void validateAnalyzerSettings(ASTPtr ast, bool context_value) { if (ast->as()) @@ -898,8 +889,6 @@ static std::tuple executeQueryImpl( if (auto * insert_query = ast->as()) insert_query->tail = istr; - setQuerySpecificSettings(ast, context); - /// There is an option of probabilistic logging of queries. /// If it is used - do the random sampling and "collapse" the settings. /// It allows to consistently log queries with all the subqueries in distributed query processing diff --git a/src/Parsers/ASTInsertQuery.cpp b/src/Parsers/ASTInsertQuery.cpp index 72a569fe047..8e3458539f3 100644 --- a/src/Parsers/ASTInsertQuery.cpp +++ b/src/Parsers/ASTInsertQuery.cpp @@ -123,13 +123,8 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s settings.ostr << delim; select->formatImpl(settings, state, frame); } - else if (watch) - { - settings.ostr << delim; - watch->formatImpl(settings, state, frame); - } - if (!select && !watch) + if (!select) { if (!format.empty()) { diff --git a/src/Parsers/ASTInsertQuery.h b/src/Parsers/ASTInsertQuery.h index b0f444ed755..aeab0f148be 100644 --- a/src/Parsers/ASTInsertQuery.h +++ b/src/Parsers/ASTInsertQuery.h @@ -24,7 +24,6 @@ public: ASTPtr settings_ast; ASTPtr select; - ASTPtr watch; ASTPtr infile; ASTPtr compression; @@ -63,7 +62,6 @@ public: if (partition_by) { res->partition_by = partition_by->clone(); res->children.push_back(res->partition_by); } if (settings_ast) { res->settings_ast = settings_ast->clone(); res->children.push_back(res->settings_ast); } if (select) { res->select = select->clone(); res->children.push_back(res->select); } - if (watch) { res->watch = watch->clone(); res->children.push_back(res->watch); } if (infile) { res->infile = infile->clone(); res->children.push_back(res->infile); } if (compression) { res->compression = compression->clone(); res->children.push_back(res->compression); } diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index d1171dd4815..9373e6a1c93 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -36,7 +36,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_format(Keyword::FORMAT); ParserKeyword s_settings(Keyword::SETTINGS); ParserKeyword s_select(Keyword::SELECT); - ParserKeyword s_watch(Keyword::WATCH); ParserKeyword s_partition_by(Keyword::PARTITION_BY); ParserKeyword s_with(Keyword::WITH); ParserToken s_lparen(TokenType::OpeningRoundBracket); @@ -56,7 +55,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr columns; ASTPtr format; ASTPtr select; - ASTPtr watch; ASTPtr table_function; ASTPtr settings_ast; ASTPtr partition_by_expr; @@ -143,7 +141,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) String format_str; Pos before_values = pos; - /// VALUES or FORMAT or SELECT or WITH or WATCH. + /// VALUES or FORMAT or SELECT or WITH. /// After FROM INFILE we expect FORMAT, SELECT, WITH or nothing. if (!infile && s_values.ignore(pos, expected)) { @@ -175,14 +173,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) tryGetIdentifierNameInto(format, format_str); } - else if (!infile && s_watch.ignore(pos, expected)) - { - /// If WATCH is defined, return to position before WATCH and parse - /// rest of query as WATCH query. - pos = before_values; - ParserWatchQuery watch_p; - watch_p.parse(pos, watch, expected); - } else if (!infile) { /// If all previous conditions were false and it's not FROM INFILE, query is incorrect @@ -286,7 +276,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->columns = columns; query->format = std::move(format_str); query->select = select; - query->watch = watch; query->settings_ast = settings_ast; query->data = data != end ? data : nullptr; query->end = end; @@ -295,8 +284,6 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->children.push_back(columns); if (select) query->children.push_back(select); - if (watch) - query->children.push_back(watch); if (settings_ast) query->children.push_back(settings_ast); From 057893c3107a3e14f4ec2d9c54a2664129125de4 Mon Sep 17 00:00:00 2001 From: peter279k Date: Mon, 8 Apr 2024 14:32:46 +0800 Subject: [PATCH 176/243] Add checksum to validate the downloaded archive --- docs/en/getting-started/example-datasets/menus.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 32fe62865d4..5a35c1d45bc 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -18,6 +18,9 @@ Run the command: ```bash wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz +# Option: Validate the checksum +md5sum 2021_08_01_07_01_17_data.tgz +# Checksum should be equal to: db6126724de939a5481e3160a2d67d15 ``` Replace the link to the up to date link from http://menus.nypl.org/data if needed. From 064acacd93a7de86cd66bf551905b9ff365a9eef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 Apr 2024 12:43:10 +0200 Subject: [PATCH 177/243] Update test --- .../0_stateless/02263_format_insert_settings.reference | 10 ---------- .../0_stateless/02263_format_insert_settings.sh | 2 -- 2 files changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 2bba75f6788..ea8b78faf8c 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -21,10 +21,6 @@ INSERT INTO foo FORMAT Values INSERT INTO foo SELECT 1 [oneline] insert into foo select 1 INSERT INTO foo SELECT 1 -[multi] insert into foo watch bar -INSERT INTO foo WATCH bar -[oneline] insert into foo watch bar -INSERT INTO foo WATCH bar [multi] insert into foo format tsv INSERT INTO foo FORMAT tsv [oneline] insert into foo format tsv @@ -41,12 +37,6 @@ SETTINGS max_threads = 1 SELECT 1 [oneline] insert into foo settings max_threads=1 select 1 INSERT INTO foo SETTINGS max_threads = 1 SELECT 1 -[multi] insert into foo settings max_threads=1 watch bar -INSERT INTO foo -SETTINGS max_threads = 1 -WATCH bar -[oneline] insert into foo settings max_threads=1 watch bar -INSERT INTO foo SETTINGS max_threads = 1 WATCH bar [multi] insert into foo settings max_threads=1 format tsv INSERT INTO foo SETTINGS max_threads = 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 49aa56d6c0a..808ab23ee59 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -40,12 +40,10 @@ $CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' run_format_both 'insert into foo select 1' -run_format_both 'insert into foo watch bar' run_format_both 'insert into foo format tsv' run_format_both 'insert into foo settings max_threads=1 values' run_format_both 'insert into foo settings max_threads=1 select 1' -run_format_both 'insert into foo settings max_threads=1 watch bar' run_format_both 'insert into foo settings max_threads=1 format tsv' run_format_both 'insert into foo select 1 settings max_threads=1' run_format_both 'insert into foo settings max_threads=1 select 1 settings max_threads=1' From 444016fb3ee585cae98df9f16285b9c0fff6577a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 11:20:30 +0000 Subject: [PATCH 178/243] CI: fix unittest issue --- tests/ci/ci.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 36e9b183805..f60c40f5860 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -773,6 +773,7 @@ class CiOptions: not pr_info.is_pr() and not debug_message ): # if commit_message is provided it's test/debug scenario - do not return # CI options can be configured in PRs only + # if debug_message is provided - it's a test return res message = debug_message or GitRunner(set_cwd_to_git_root=True).run( f"{GIT_PREFIX} log {pr_info.sha} --format=%B -n 1" @@ -790,9 +791,9 @@ class CiOptions: print(f"CI tags from PR body: [{matches_pr}]") matches = list(set(matches + matches_pr)) - if "do not test" in pr_info.labels: - # do_not_test could be set in GH labels - res.do_not_test = True + if "do not test" in pr_info.labels: + # do_not_test could be set in GH labels + res.do_not_test = True for match in matches: if match.startswith("job_"): From 094f94882c972a798ace493ca9a7019255a64f7b Mon Sep 17 00:00:00 2001 From: Ilya Andreev <18560147+andreev-io@users.noreply.github.com> Date: Mon, 8 Apr 2024 11:35:03 +0100 Subject: [PATCH 179/243] Fix a typo in the documentation of the ALTER TABLE ... MODIFY QUERY statement --- docs/en/sql-reference/statements/alter/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index 59045afdeb6..e063b27424e 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -8,7 +8,7 @@ sidebar_label: VIEW You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process. -This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause. +This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underlying storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause. **Example with TO table** From 83d1f1a8769d3be8d78f48db82873b9438ac87f4 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 11:51:59 +0000 Subject: [PATCH 180/243] CI: fix for docs only pr --- tests/ci/ci.py | 7 ++++--- tests/ci/pr_info.py | 1 + 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 36e9b183805..f60c40f5860 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -773,6 +773,7 @@ class CiOptions: not pr_info.is_pr() and not debug_message ): # if commit_message is provided it's test/debug scenario - do not return # CI options can be configured in PRs only + # if debug_message is provided - it's a test return res message = debug_message or GitRunner(set_cwd_to_git_root=True).run( f"{GIT_PREFIX} log {pr_info.sha} --format=%B -n 1" @@ -790,9 +791,9 @@ class CiOptions: print(f"CI tags from PR body: [{matches_pr}]") matches = list(set(matches + matches_pr)) - if "do not test" in pr_info.labels: - # do_not_test could be set in GH labels - res.do_not_test = True + if "do not test" in pr_info.labels: + # do_not_test could be set in GH labels + res.do_not_test = True for match in matches: if match.startswith("job_"): diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index ddf59c49e1f..204284785c9 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -26,6 +26,7 @@ NeedsDataType = Dict[str, Dict[str, Union[str, Dict[str, str]]]] DIFF_IN_DOCUMENTATION_EXT = [ ".html", ".md", + ".mdx", ".yml", ".txt", ".css", From 259d50c57b6227b2a078effcef8de19cd23c346f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Apr 2024 14:36:07 +0200 Subject: [PATCH 181/243] Add more comments --- src/Interpreters/Cache/EvictionCandidates.cpp | 22 ++++-- src/Interpreters/Cache/EvictionCandidates.h | 8 +- src/Interpreters/Cache/FileCache.cpp | 77 ++++++++++++------- src/Interpreters/Cache/FileCacheFactory.cpp | 8 +- .../Cache/LRUFileCachePriority.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 72 +++++++++-------- 6 files changed, 112 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/Cache/EvictionCandidates.cpp b/src/Interpreters/Cache/EvictionCandidates.cpp index f9f9bdfe662..d20ae77d720 100644 --- a/src/Interpreters/Cache/EvictionCandidates.cpp +++ b/src/Interpreters/Cache/EvictionCandidates.cpp @@ -17,6 +17,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +EvictionCandidates::EvictionCandidates() + : log(getLogger("EvictionCandidates")) +{ +} + EvictionCandidates::~EvictionCandidates() { /// Here `queue_entries_to_invalidate` contains queue entries @@ -64,8 +69,11 @@ void EvictionCandidates::add( void EvictionCandidates::removeQueueEntries(const CachePriorityGuard::Lock & lock) { - auto log = getLogger("EvictionCandidates"); + /// Remove queue entries of eviction candidates. + /// This will release space we consider to be hold for them. + LOG_TEST(log, "Will remove {} eviction candidates", size()); + for (const auto & [key, key_candidates] : candidates) { for (const auto & candidate : key_candidates.candidates) @@ -87,6 +95,7 @@ void EvictionCandidates::evict() auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::FilesystemCacheEvictMicroseconds); + /// If queue entries are already removed, then nothing to invalidate. if (!removed_queue_entries) queue_entries_to_invalidate.reserve(candidates_size); @@ -184,6 +193,12 @@ void EvictionCandidates::finalize( on_finalize.clear(); } +bool EvictionCandidates::needFinalize() const +{ + /// Do we need to call finalize()? + return !on_finalize.empty() || !queue_entries_to_invalidate.empty(); +} + void EvictionCandidates::setSpaceHolder( size_t size, size_t elements, @@ -196,9 +211,4 @@ void EvictionCandidates::setSpaceHolder( hold_space = std::make_unique(size, elements, priority, lock); } -void EvictionCandidates::insert(EvictionCandidates && other, const CachePriorityGuard::Lock &) -{ - candidates.insert(make_move_iterator(other.candidates.begin()), make_move_iterator(other.candidates.end())); -} - } diff --git a/src/Interpreters/Cache/EvictionCandidates.h b/src/Interpreters/Cache/EvictionCandidates.h index baacbc0cfae..0dcc6bc0dda 100644 --- a/src/Interpreters/Cache/EvictionCandidates.h +++ b/src/Interpreters/Cache/EvictionCandidates.h @@ -9,7 +9,7 @@ class EvictionCandidates : private boost::noncopyable public: using FinalizeEvictionFunc = std::function; - EvictionCandidates() = default; + EvictionCandidates(); ~EvictionCandidates(); void add( @@ -17,8 +17,6 @@ public: LockedKey & locked_key, const CachePriorityGuard::Lock &); - void insert(EvictionCandidates && other, const CachePriorityGuard::Lock &); - void evict(); void removeQueueEntries(const CachePriorityGuard::Lock &); @@ -29,6 +27,8 @@ public: FileCacheQueryLimit::QueryContext * query_context, const CachePriorityGuard::Lock &); + bool needFinalize() const; + size_t size() const { return candidates_size; } auto begin() const { return candidates.begin(); } @@ -57,6 +57,8 @@ private: bool removed_queue_entries = false; IFileCachePriority::HoldSpacePtr hold_space; + + LoggerPtr log; }; using EvictionCandidatesPtr = std::unique_ptr; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 12ea2c178bc..29f2ebeca55 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1389,7 +1389,18 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, || new_settings.max_elements != actual_settings.max_elements) { EvictionCandidates eviction_candidates; - bool limits_satisfied = false; + bool modified_size_limit = false; + + /// In order to not block cache for the duration of cache resize, + /// we do: + /// a. Take a cache lock. + /// 1. Collect eviction candidates, + /// 2. Remove queue entries of eviction candidates. + /// This will release space we consider to be hold for them, + /// so that we can safely modify size limits. + /// 3. Modify size limits of cache. + /// b. Release a cache lock. + /// 1. Do actual eviction from filesystem. { cache_is_being_resized.store(true, std::memory_order_relaxed); SCOPE_EXIT({ @@ -1399,38 +1410,45 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, auto cache_lock = lockCache(); FileCacheReserveStat stat; - limits_satisfied = main_priority->collectCandidatesForEviction( - new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, stat, eviction_candidates, cache_lock); - - eviction_candidates.removeQueueEntries(cache_lock); - - if (limits_satisfied) + if (main_priority->collectCandidatesForEviction( + new_settings.max_size, new_settings.max_elements, 0/* max_candidates_to_evict */, + stat, eviction_candidates, cache_lock)) { + /// Remove only queue entries of eviction candidates. + eviction_candidates.removeQueueEntries(cache_lock); + /// Note that (in-memory) metadata about corresponding file segments + /// (e.g. file segment info in CacheMetadata) will be removed + /// only after eviction from filesystem. This is needed to avoid + /// a race on removal of file from filesystsem and + /// addition of the same file as part of a newly cached file segment. + + /// Modify cache size limits. + /// From this point cache eviction will follow them. main_priority->modifySizeLimits( - new_settings.max_size, new_settings.max_elements, new_settings.slru_size_ratio, cache_lock); + new_settings.max_size, new_settings.max_elements, + new_settings.slru_size_ratio, cache_lock); + + modified_size_limit = true; } - else + } + + if (modified_size_limit) + { + try { - LOG_WARNING(log, "Unable to modify size limit from {} to {}, " - "elements limit from {} to {}", - actual_settings.max_size, new_settings.max_size, - actual_settings.max_elements, new_settings.max_elements); + /// Do actual eviction from filesystem. + eviction_candidates.evict(); + } + catch (...) + { + if (eviction_candidates.needFinalize()) + eviction_candidates.finalize(nullptr, lockCache()); + throw; } - } - try - { - eviction_candidates.evict(); - } - catch (...) - { - auto cache_lock = lockCache(); - eviction_candidates.finalize(nullptr, cache_lock); - throw; - } + if (eviction_candidates.needFinalize()) + eviction_candidates.finalize(nullptr, lockCache()); - if (limits_satisfied) - { LOG_INFO(log, "Changed max_size from {} to {}, max_elements from {} to {}", actual_settings.max_size, new_settings.max_size, actual_settings.max_elements, new_settings.max_elements); @@ -1438,6 +1456,13 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, actual_settings.max_size = new_settings.max_size; actual_settings.max_elements = new_settings.max_elements; } + else + { + LOG_WARNING(log, "Unable to modify size limit from {} to {}, " + "elements limit from {} to {}", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); + } } if (new_settings.max_file_segment_size != actual_settings.max_file_segment_size) diff --git a/src/Interpreters/Cache/FileCacheFactory.cpp b/src/Interpreters/Cache/FileCacheFactory.cpp index 747b31bff64..a7a5834f03d 100644 --- a/src/Interpreters/Cache/FileCacheFactory.cpp +++ b/src/Interpreters/Cache/FileCacheFactory.cpp @@ -142,10 +142,8 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig caches_by_name_copy = caches_by_name; } - auto * log = &Poco::Logger::get("FileCacheFactory"); - std::unordered_set checked_paths; - for (const auto & [cache_name, cache_info] : caches_by_name_copy) + for (const auto & [_, cache_info] : caches_by_name_copy) { if (cache_info->config_path.empty() || checked_paths.contains(cache_info->config_path)) continue; @@ -158,10 +156,12 @@ void FileCacheFactory::updateSettingsFromConfig(const Poco::Util::AbstractConfig FileCacheSettings old_settings = cache_info->getSettings(); if (old_settings == new_settings) { - LOG_TRACE(log, "No settings changes for cache: {}", cache_name); continue; } + /// FIXME: registerDiskCache modifies `path` setting of FileCacheSettings if path is relative. + /// This can lead to calling applySettingsIfPossible even though nothing changed, which is avoidable. + // LOG_TRACE(log, "Will apply settings changes for cache {}. " // "Settings changes: {} (new settings: {}, old_settings: {})", // cache_name, fmt::join(new_settings.getSettingsDiff(old_settings), ", "), diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index e859529f5e7..1a2040f9ed2 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -280,7 +280,7 @@ bool LRUFileCachePriority::collectCandidatesForEviction( auto can_fit = [&] { - return canFit(size, 1, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock); + return canFit(size, elements, stat.total_stat.releasable_size, stat.total_stat.releasable_count, lock); }; iterateForEviction(res, stat, can_fit, lock); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 631c1aa2ae6..2cbd56ba0bc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -941,49 +941,47 @@ KeyMetadata::iterator LockedKey::removeFileSegmentImpl( file_segment->detach(segment_lock, *this); + try { - try + const auto path = key_metadata->getFileSegmentPath(*file_segment); + if (file_segment->segment_kind == FileSegmentKind::Temporary) { - const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (file_segment->segment_kind == FileSegmentKind::Temporary) - { - /// FIXME: For temporary file segment the requirement is not as strong because - /// the implementation of "temporary data in cache" creates files in advance. - if (fs::exists(path)) - fs::remove(path); - } - else if (file_segment->downloaded_size == 0) - { - chassert(!fs::exists(path)); - } - else if (fs::exists(path)) - { + /// FIXME: For temporary file segment the requirement is not as strong because + /// the implementation of "temporary data in cache" creates files in advance. + if (fs::exists(path)) fs::remove(path); - - /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. - int flags = file_segment->getFlagsForLocalRead(); - /// Files are created with flags from file_segment->getFlagsForLocalRead() - /// plus optionally O_DIRECT is added, depends on query setting, so remove both. - OpenedFileCache::instance().remove(path, flags); - OpenedFileCache::instance().remove(path, flags | O_DIRECT); - - LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path); - } - else if (!can_be_broken) - { -#ifdef ABORT_ON_LOGICAL_ERROR - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); -#else - LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}", - path, getKey(), file_segment->offset()); -#endif - } } - catch (...) + else if (file_segment->downloaded_size == 0) { - tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); + chassert(!fs::exists(path)); } + else if (fs::exists(path)) + { + fs::remove(path); + + /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. + int flags = file_segment->getFlagsForLocalRead(); + /// Files are created with flags from file_segment->getFlagsForLocalRead() + /// plus optionally O_DIRECT is added, depends on query setting, so remove both. + OpenedFileCache::instance().remove(path, flags); + OpenedFileCache::instance().remove(path, flags | O_DIRECT); + + LOG_TEST(key_metadata->logger(), "Removed file segment at path: {}", path); + } + else if (!can_be_broken) + { +#ifdef ABORT_ON_LOGICAL_ERROR + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); +#else + LOG_WARNING(key_metadata->logger(), "Expected path {} to exist, while removing {}:{}", + path, getKey(), file_segment->offset()); +#endif + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); } return key_metadata->erase(it); From b6eef6137823a92c6a6ba601bb4c879b2dee30fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 5 Apr 2024 17:10:01 +0200 Subject: [PATCH 182/243] Fix one phony case --- contrib/avro-cmake/CMakeLists.txt | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index 63b3854eef9..c99c7dd4624 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -62,9 +62,12 @@ target_link_libraries (_avrocpp PRIVATE ch_contrib::snappy) # create a symlink to include headers with set(AVRO_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") -ADD_CUSTOM_TARGET(avro_symlink_headers ALL - COMMAND ${CMAKE_COMMAND} -E make_directory "${AVRO_INCLUDE_DIR}" - COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVRO_INCLUDE_DIR}/avro" +ADD_CUSTOM_COMMAND(OUTPUT "${AVRO_INCLUDE_DIR}" + COMMAND ${CMAKE_COMMAND} -E make_directory "${AVRO_INCLUDE_DIR}" + COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVRO_INCLUDE_DIR}/avro" + DEPENDS "${AVROCPP_ROOT_DIR}/api" ) +ADD_CUSTOM_TARGET(avro_symlink_headers ALL + DEPENDS "${AVRO_INCLUDE_DIR}") add_dependencies(_avrocpp avro_symlink_headers) target_include_directories(_avrocpp SYSTEM BEFORE PUBLIC "${AVRO_INCLUDE_DIR}") From 31cd71f8f2e78623af980d14ab0b29b3969757da Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Apr 2024 14:39:11 +0200 Subject: [PATCH 183/243] Rename test --- ...ence => 03032_dynamically_resize_filesystem_cache_2.reference} | 0 ...hardcore.sh => 03032_dynamically_resize_filesystem_cache_2.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03032_dynamically_resize_filesystem_cache_hardcore.reference => 03032_dynamically_resize_filesystem_cache_2.reference} (100%) rename tests/queries/0_stateless/{03032_dynamically_resize_filesystem_cache_hardcore.sh => 03032_dynamically_resize_filesystem_cache_2.sh} (100%) diff --git a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.reference similarity index 100% rename from tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.reference rename to tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.reference diff --git a/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh b/tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh similarity index 100% rename from tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_hardcore.sh rename to tests/queries/0_stateless/03032_dynamically_resize_filesystem_cache_2.sh From e4b0ca5d836e14fada7592777a0443914bfbaa47 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 12:59:20 +0000 Subject: [PATCH 184/243] Fix filter pushdown from additional_table_filters in Merge engine in analyzer --- src/Planner/PlannerJoinTree.cpp | 3 ++- src/Storages/StorageDummy.h | 6 ++++++ ...03033_analyzer_merge_engine_filter_push_down.reference | 3 +++ .../03033_analyzer_merge_engine_filter_push_down.sql | 8 ++++++++ 4 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.reference create mode 100644 tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index d2f37ff1ad4..534080f1739 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -814,7 +814,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); - if (storage->supportsPrewhere() && optimize_move_to_prewhere) + auto supported_prewhere_columns = storage->supportedPrewhereColumns(); + if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name))) { if (!prewhere_info) prewhere_info = std::make_shared(); diff --git a/src/Storages/StorageDummy.h b/src/Storages/StorageDummy.h index e9d8f90f755..ae9bf2483e1 100644 --- a/src/Storages/StorageDummy.h +++ b/src/Storages/StorageDummy.h @@ -19,6 +19,12 @@ public: bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } bool supportsPrewhere() const override { return true; } + + std::optional supportedPrewhereColumns() const override + { + return original_storage_snapshot ? original_storage_snapshot->storage.supportedPrewhereColumns() : std::nullopt; + } + bool supportsSubcolumns() const override { return true; } bool supportsDynamicSubcolumns() const override { return true; } bool canMoveConditionsToPrewhere() const override diff --git a/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.reference b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.reference new file mode 100644 index 00000000000..86a00059854 --- /dev/null +++ b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.reference @@ -0,0 +1,3 @@ +UInt32 1 +UInt32 2 +UInt32 3 diff --git a/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql new file mode 100644 index 00000000000..9be1152bbbf --- /dev/null +++ b/tests/queries/0_stateless/03033_analyzer_merge_engine_filter_push_down.sql @@ -0,0 +1,8 @@ +set allow_suspicious_low_cardinality_types=1; +drop table if exists test; +create table test (`x` LowCardinality(Nullable(UInt32)), `y` String) engine = MergeTree order by tuple(); +insert into test values (1, 'a'), (2, 'bb'), (3, 'ccc'), (4, 'dddd'); +create table m_table (x UInt32, y String) engine = Merge(currentDatabase(), 'test*'); +select toTypeName(x), x FROM m_table SETTINGS additional_table_filters = {'m_table':'x != 4'}, optimize_move_to_prewhere=1, allow_experimental_analyzer=1; +drop table test; + From 5e1c1b6b94d920fbfb361c0cf606728f730e149a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 13:41:44 +0000 Subject: [PATCH 185/243] CI: test merge queue --- tests/ci/ci.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index f60c40f5860..c2962c5b40e 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -318,7 +318,7 @@ class CiCache: self.update() if self.cache_data_fetched: - # there are no record w/o underling data - no need to fetch + # there are no records without fetched data - no need to fetch return self # clean up From 44d3612d77032e2b104296840690154e53d8f073 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Apr 2024 15:52:38 +0200 Subject: [PATCH 186/243] Review suggestion --- src/Interpreters/Cache/FileCache.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 29f2ebeca55..be452e43bed 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1458,10 +1458,13 @@ void FileCache::applySettingsIfPossible(const FileCacheSettings & new_settings, } else { - LOG_WARNING(log, "Unable to modify size limit from {} to {}, " - "elements limit from {} to {}", - actual_settings.max_size, new_settings.max_size, - actual_settings.max_elements, new_settings.max_elements); + LOG_WARNING( + log, "Unable to modify size limit from {} to {}, elements limit from {} to {}. " + "`max_size` and `max_elements` settings will remain inconsistent with config.xml. " + "Next attempt to update them will happen on the next config reload. " + "You can trigger it with SYSTEM RELOAD CONFIG.", + actual_settings.max_size, new_settings.max_size, + actual_settings.max_elements, new_settings.max_elements); } } From f3dc77ee00008f640b4d1f47445a223bbe286000 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 14:15:49 +0000 Subject: [PATCH 187/243] disable autofix for merge queue --- tests/ci/style_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 373fa7b316f..4580f007606 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -131,6 +131,11 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) pr_info = PRInfo() + + if pr_info.is_merge_queue() and args.push: + print("Auto style fix will be disabled for Merge Queue workflow") + args.push = False + run_cpp_check = True run_shell_check = True run_python_check = True From 82b2adef97a87de683c1a20ec696c03216416a23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Apr 2024 14:50:13 +0000 Subject: [PATCH 188/243] Fix GLOBAL IN table queries with analyzer. --- src/Planner/CollectSets.cpp | 52 +++++++++++-------- src/Planner/CollectSets.h | 4 ++ src/Storages/buildQueryTreeForShard.cpp | 7 ++- .../test_cluster_all_replicas/test.py | 11 ++++ 4 files changed, 49 insertions(+), 25 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index e150b8a5956..37502828f63 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -24,6 +24,34 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } +QueryTreeNodePtr makeExecutableSubqueryForIn(const QueryTreeNodePtr & in_second_argument, const ContextPtr & context) +{ + auto subquery_to_execute = in_second_argument; + if (auto * table_node = in_second_argument->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + size_t columns_to_select_size = columns_to_select.size(); + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + auto subquery_for_table = std::make_shared(Context::createCopy(context)); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + subquery_to_execute = std::move(subquery_for_table); + } + + return subquery_to_execute; +} + namespace { @@ -88,29 +116,7 @@ public: if (sets.findSubquery(set_key)) return; - auto subquery_to_execute = in_second_argument; - if (auto * table_node = in_second_argument->as()) - { - auto storage_snapshot = table_node->getStorageSnapshot(); - auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); - size_t columns_to_select_size = columns_to_select.size(); - auto column_nodes_to_select = std::make_shared(); - column_nodes_to_select->getNodes().reserve(columns_to_select_size); - NamesAndTypes projection_columns; - projection_columns.reserve(columns_to_select_size); - for (auto & column : columns_to_select) - { - column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); - projection_columns.emplace_back(column.name, column.type); - } - auto subquery_for_table = std::make_shared(Context::createCopy(planner_context.getQueryContext())); - subquery_for_table->setIsSubquery(true); - subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); - subquery_for_table->getJoinTree() = std::move(subquery_to_execute); - subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); - subquery_to_execute = std::move(subquery_for_table); - } - + auto subquery_to_execute = makeExecutableSubqueryForIn(in_second_argument, planner_context.getQueryContext()); sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings); } else diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 5f9f7a5a466..e4168c7dd49 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -14,4 +14,8 @@ struct SelectQueryOptions; */ void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); +/// Build subqiery which we execute for IN function. +/// It is needed to support `IN table` case. +QueryTreeNodePtr makeExecutableSubqueryForIn(const QueryTreeNodePtr & in_second_argument, const ContextPtr & context); + } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 5284f52a7e4..09e48a93df4 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -361,10 +362,12 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex { auto & in_function_subquery_node = in_function_node->getArguments().getNodes().at(1); auto in_function_node_type = in_function_subquery_node->getNodeType(); - if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION) + if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION && in_function_node_type != QueryTreeNodeType::TABLE) continue; - auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, + auto subquery_to_execute = makeExecutableSubqueryForIn(in_function_subquery_node, planner_context->getQueryContext()); + + auto temporary_table_expression_node = executeSubqueryNode(subquery_to_execute, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); diff --git a/tests/integration/test_cluster_all_replicas/test.py b/tests/integration/test_cluster_all_replicas/test.py index eb406de6a8d..15f3f36f74e 100644 --- a/tests/integration/test_cluster_all_replicas/test.py +++ b/tests/integration/test_cluster_all_replicas/test.py @@ -42,6 +42,17 @@ def test_cluster(start_cluster): ) +def test_global_in(start_cluster): + + node1.query("CREATE TABLE u(uid Int16) ENGINE=Log as select 0"); + + assert set( + node1.query( + """SELECT hostName(), * FROM clusterAllReplicas("one_shard_two_nodes", system.one) where dummy GLOBAL IN u""" + ).splitlines() + ) == {"node1\t0", "node2\t0"} + + @pytest.mark.parametrize( "cluster", [ From bcccbe2b19bd50429ce6de1f4e0619c0677fa4a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 Apr 2024 14:54:36 +0000 Subject: [PATCH 189/243] Fixing typo --- src/Planner/CollectSets.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index e4168c7dd49..0ee006f3320 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -14,7 +14,7 @@ struct SelectQueryOptions; */ void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); -/// Build subqiery which we execute for IN function. +/// Build subquery which we execute for IN function. /// It is needed to support `IN table` case. QueryTreeNodePtr makeExecutableSubqueryForIn(const QueryTreeNodePtr & in_second_argument, const ContextPtr & context); From 7fcfbffd0cf215e87d36aa515d832577256ce2e3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Apr 2024 15:01:11 +0000 Subject: [PATCH 190/243] Automatic style fix --- tests/integration/test_cluster_all_replicas/test.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/test_cluster_all_replicas/test.py b/tests/integration/test_cluster_all_replicas/test.py index 15f3f36f74e..59b41ca87af 100644 --- a/tests/integration/test_cluster_all_replicas/test.py +++ b/tests/integration/test_cluster_all_replicas/test.py @@ -43,8 +43,7 @@ def test_cluster(start_cluster): def test_global_in(start_cluster): - - node1.query("CREATE TABLE u(uid Int16) ENGINE=Log as select 0"); + node1.query("CREATE TABLE u(uid Int16) ENGINE=Log as select 0") assert set( node1.query( From 603824748d7bec40d8dc7b30a33a988e214c3328 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 15:03:13 +0000 Subject: [PATCH 191/243] CI: disable finish check for mq --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ff0adee1443..74ce8452de8 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -157,7 +157,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !failure() && !cancelled() && github.event_name != 'merge_group' }} needs: [Tests_1, Tests_2] runs-on: [self-hosted, style-checker] steps: From c69d8e18f245b67057f9c77c1c5e06352c805529 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Apr 2024 15:38:15 +0000 Subject: [PATCH 192/243] Dont use constant database name --- ...unknown_identifier_materialized_column.sql | 10 ++++---- .../0_stateless/03053_analyzer_join_alias.sql | 16 ++++++------- ...same_table_name_in_different_databases.sql | 24 +++++++++---------- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql index 276e4845831..938f270b9e4 100644 --- a/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql +++ b/tests/queries/0_stateless/03049_unknown_identifier_materialized_column.sql @@ -1,14 +1,14 @@ -- https://github.com/ClickHouse/ClickHouse/issues/54317 SET allow_experimental_analyzer=1; -DROP DATABASE IF EXISTS 03049_database; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -CREATE DATABASE 03049_database; -USE 03049_database; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; +USE {CLICKHOUSE_DATABASE:Identifier}; CREATE TABLE l (y String) Engine Memory; CREATE TABLE r (d Date, y String, ty UInt16 MATERIALIZED toYear(d)) Engine Memory; select * from l L left join r R on L.y = R.y where R.ty >= 2019; select * from l left join r on l.y = r.y where r.ty >= 2019; -select * from 03049_database.l left join 03049_database.r on l.y = r.y where r.ty >= 2019; +select * from {CLICKHOUSE_DATABASE:Identifier}.l left join {CLICKHOUSE_DATABASE:Identifier}.r on l.y = r.y where r.ty >= 2019; -DROP DATABASE IF EXISTS 03049_database; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/03053_analyzer_join_alias.sql b/tests/queries/0_stateless/03053_analyzer_join_alias.sql index ef51ec73026..894b8af7c6f 100644 --- a/tests/queries/0_stateless/03053_analyzer_join_alias.sql +++ b/tests/queries/0_stateless/03053_analyzer_join_alias.sql @@ -1,9 +1,9 @@ -- https://github.com/ClickHouse/ClickHouse/issues/23104 SET allow_experimental_analyzer=1; -DROP DATABASE IF EXISTS test_03053; -CREATE DATABASE test_03053; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; -CREATE TABLE test_03053.base +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.base ( `id` UInt64, `id2` UInt64, @@ -14,7 +14,7 @@ ENGINE=MergeTree() PARTITION BY d ORDER BY (id,id2,d); -CREATE TABLE test_03053.derived1 +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.derived1 ( `id1` UInt64, `d1` UInt64, @@ -24,7 +24,7 @@ ENGINE = MergeTree() PARTITION BY d1 ORDER BY (id1, d1); -CREATE TABLE test_03053.derived2 +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.derived2 ( `id2` UInt64, `d2` UInt64, @@ -39,6 +39,6 @@ SELECT derived2.id2 AS `derived2.id2`, derived2.value2 AS `derived2.value2`, derived1.value1 AS `derived1.value1` -FROM test_03053.base AS base -LEFT JOIN test_03053.derived2 AS derived2 ON base.id2 = derived2.id2 -LEFT JOIN test_03053.derived1 AS derived1 ON base.id = derived1.id1; +FROM {CLICKHOUSE_DATABASE:Identifier}.base AS base +LEFT JOIN {CLICKHOUSE_DATABASE:Identifier}.derived2 AS derived2 ON base.id2 = derived2.id2 +LEFT JOIN {CLICKHOUSE_DATABASE:Identifier}.derived1 AS derived1 ON base.id = derived1.id1; diff --git a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql index 03ad9c97d94..10d18324c3c 100644 --- a/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql +++ b/tests/queries/0_stateless/03092_analyzer_same_table_name_in_different_databases.sql @@ -1,18 +1,18 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61947 SET allow_experimental_analyzer=1; -DROP DATABASE IF EXISTS d3; -DROP DATABASE IF EXISTS d4; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE d3; -CREATE DATABASE d4; -CREATE TABLE d3.`1-1` (field Int8) ENGINE = Memory; -CREATE TABLE d4.`2-1` (field Int8) ENGINE = Memory; -CREATE TABLE d4.`3-1` (field Int8) ENGINE = Memory; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.`2-1` (field Int8) ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.`3-1` (field Int8) ENGINE = Memory; -INSERT INTO d3.`1-1` VALUES (1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.`1-1` VALUES (1); -SELECT d3.`1-1`.* -FROM d3.`1-1` -LEFT JOIN d4.`2-1` ON d3.`1-1`.field = d4.`2-1`.field -LEFT JOIN d4.`3-1` ON d4.`2-1`.field = d4.`3-1`.field; +SELECT {CLICKHOUSE_DATABASE:Identifier}.`1-1`.* +FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` +LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`2-1` ON {CLICKHOUSE_DATABASE:Identifier}.`1-1`.field = {CLICKHOUSE_DATABASE_1:Identifier}.`2-1`.field +LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`3-1` ON {CLICKHOUSE_DATABASE_1:Identifier}.`2-1`.field = {CLICKHOUSE_DATABASE_1:Identifier}.`3-1`.field; From baa62cdaeeb23aba770efe6368bba6ec97cf6214 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 8 Apr 2024 16:09:47 +0000 Subject: [PATCH 193/243] CI: no CI Running status for MQ --- tests/ci/run_check.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 6187656983e..435a5f726f2 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -201,14 +201,17 @@ def main(): ci_report_url = create_ci_report(pr_info, []) print("::notice ::Can run") - post_commit_status( - commit, - PENDING, - ci_report_url, - description, - CI_STATUS_NAME, - pr_info, - ) + + if not pr_info.is_merge_queue(): + # we need clean CI status for MQ to merge (no pending statuses) + post_commit_status( + commit, + PENDING, + ci_report_url, + description, + CI_STATUS_NAME, + pr_info, + ) if __name__ == "__main__": From 39c6188a2c0b7014136e1d9d9f16c684741fb0cb Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 16:38:19 +0000 Subject: [PATCH 194/243] Fix logical error 'numbers_storage.step != UInt64{0}' --- src/TableFunctions/TableFunctionNumbers.cpp | 4 ++++ .../03037_zero_step_in_numbers_table_function.reference | 0 .../0_stateless/03037_zero_step_in_numbers_table_function.sql | 2 ++ 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.reference create mode 100644 tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.sql diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 2989eb5fbef..16f56eab981 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int BAD_ARGUMENTS; } namespace @@ -78,6 +79,9 @@ StoragePtr TableFunctionNumbers::executeImpl( UInt64 length = arguments.size() >= 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); UInt64 step = arguments.size() == 3 ? evaluateArgument(context, arguments[2]) : 1; + if (!step) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function {} requires step to be a positive number", getName()); + auto res = std::make_shared( StorageID(getDatabaseName(), table_name), multithreaded, std::string{"number"}, length, offset, step); res->startup(); diff --git a/tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.reference b/tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.sql b/tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.sql new file mode 100644 index 00000000000..08fafd6ddfa --- /dev/null +++ b/tests/queries/0_stateless/03037_zero_step_in_numbers_table_function.sql @@ -0,0 +1,2 @@ +select * from numbers(1, 10, 0); -- {serverError BAD_ARGUMENTS} + From b318091528eed9db6d04d25bae115d24d3b82eb8 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 8 Apr 2024 17:17:04 +0000 Subject: [PATCH 195/243] Don't check overflow in dotProduct in undefined sanitizer --- src/Functions/array/arrayDotProduct.cpp | 4 ++-- .../queries/0_stateless/03037_dot_product_overflow.reference | 1 + tests/queries/0_stateless/03037_dot_product_overflow.sql | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03037_dot_product_overflow.reference create mode 100644 tests/queries/0_stateless/03037_dot_product_overflow.sql diff --git a/src/Functions/array/arrayDotProduct.cpp b/src/Functions/array/arrayDotProduct.cpp index 783843a89d5..4551140acc3 100644 --- a/src/Functions/array/arrayDotProduct.cpp +++ b/src/Functions/array/arrayDotProduct.cpp @@ -66,13 +66,13 @@ struct DotProduct }; template - static void accumulate(State & state, Type x, Type y) + static NO_SANITIZE_UNDEFINED void accumulate(State & state, Type x, Type y) { state.sum += x * y; } template - static void combine(State & state, const State & other_state) + static NO_SANITIZE_UNDEFINED void combine(State & state, const State & other_state) { state.sum += other_state.sum; } diff --git a/tests/queries/0_stateless/03037_dot_product_overflow.reference b/tests/queries/0_stateless/03037_dot_product_overflow.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03037_dot_product_overflow.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03037_dot_product_overflow.sql b/tests/queries/0_stateless/03037_dot_product_overflow.sql new file mode 100644 index 00000000000..94d5eba6255 --- /dev/null +++ b/tests/queries/0_stateless/03037_dot_product_overflow.sql @@ -0,0 +1,2 @@ +select ignore(dotProduct(materialize([9223372036854775807, 1]), materialize([-3, 1]))); + From 75aff7fc1aed1c56cfd406466a99d5739895f0c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 8 Apr 2024 15:50:10 +0200 Subject: [PATCH 196/243] Remove the code --- contrib/avro-cmake/CMakeLists.txt | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index c99c7dd4624..96f740b6dd2 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -59,15 +59,3 @@ target_link_libraries (_avrocpp PRIVATE boost::headers_only boost::iostreams) target_compile_definitions (_avrocpp PUBLIC SNAPPY_CODEC_AVAILABLE) target_include_directories (_avrocpp PRIVATE ${SNAPPY_INCLUDE_DIR}) target_link_libraries (_avrocpp PRIVATE ch_contrib::snappy) - -# create a symlink to include headers with -set(AVRO_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") -ADD_CUSTOM_COMMAND(OUTPUT "${AVRO_INCLUDE_DIR}" - COMMAND ${CMAKE_COMMAND} -E make_directory "${AVRO_INCLUDE_DIR}" - COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVRO_INCLUDE_DIR}/avro" - DEPENDS "${AVROCPP_ROOT_DIR}/api" -) -ADD_CUSTOM_TARGET(avro_symlink_headers ALL - DEPENDS "${AVRO_INCLUDE_DIR}") -add_dependencies(_avrocpp avro_symlink_headers) -target_include_directories(_avrocpp SYSTEM BEFORE PUBLIC "${AVRO_INCLUDE_DIR}") From 75e46f865c1d445c23d55e918f1bb83669250b46 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Apr 2024 19:19:18 +0200 Subject: [PATCH 197/243] Avoid uncaught exception for onFault handler onFault() is called from the std::thread, and it should catch all exceptions, otherwise you can unrelated fatal errors.
stacktrace 2024.04.08 13:15:29.526847 [ 2067427 ] {} BaseDaemon: (version 24.2.2.71 (official build), build id: 57F857DCFE8BA6838F6463E4665CD700852BFF0E, git hash: 9293d361e72be9f6ccfd444d504e2137b2e837cf) (from thread 2118603) Terminate called for uncaught exception: 2024.04.08 13:15:29.526904 [ 2067427 ] {} BaseDaemon: Code: 210. DB::NetException: I/O error: Broken pipe, while writing to socket (10.61.7.253:9000 -> 10.101.53.134:46036). (NETWORK_ERROR), Stack trace (when copying this message, always include the lines below): 2024.04.08 13:15:29.526983 [ 2067427 ] {} BaseDaemon: 2024.04.08 13:15:29.527042 [ 2067427 ] {} BaseDaemon: 0. ./build_docker/./src/Common/Exception.cpp:96: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000cf5af1b 2024.04.08 13:15:29.527061 [ 2067427 ] {} BaseDaemon: 1. ./contrib/llvm-project/libcxx/include/string:1499: DB::NetException::NetException(int, FormatStringHelperImpl::type, std::type_identity::type, std::type_identity::type>, String&&, String&&, String&&) @ 0x000000000d07dfe1 2024.04.08 13:15:29.527082 [ 2067427 ] {} BaseDaemon: 2. ./build_docker/./src/IO/WriteBufferFromPocoSocket.cpp:0: DB::WriteBufferFromPocoSocket::nextImpl() @ 0x000000000d07e97e 2024.04.08 13:15:29.527125 [ 2067427 ] {} BaseDaemon: 3. ./src/IO/WriteBuffer.h:65: DB::TCPHandler::sendLogs() @ 0x0000000012fd31c1 2024.04.08 13:15:29.527144 [ 2067427 ] {} BaseDaemon: 4. ./contrib/llvm-project/libcxx/include/atomic:958: void std::__function::__policy_invoker::__call_impl>(std::__function::__policy_storage const*) @ 0x0000000012fdcc6e 2024.04.08 13:15:29.527163 [ 2067427 ] {} BaseDaemon: 5. ./contrib/llvm-project/libcxx/include/__functional/function.h:0: ? @ 0x000000000d25c65b 2024.04.08 13:15:29.527182 [ 2067427 ] {} BaseDaemon: 6. ./build_docker/./src/Daemon/BaseDaemon.cpp:286: void* std::__thread_proxy[abi:v15000]>, SignalListener::run()::'lambda'()>>(void*) @ 0x000000000d25e775 2024.04.08 13:15:29.527191 [ 2067427 ] {} BaseDaemon: 7. ? @ 0x00007f0fe4906609 2024.04.08 13:15:29.527211 [ 2067427 ] {} BaseDaemon: 8. ? @ 0x00007f0fe482b353 2024.04.08 13:15:29.534235 [ 2118604 ] {} BaseDaemon: ########## Short fault info ############ 2024.04.08 13:15:29.534347 [ 2118604 ] {} BaseDaemon: (version 24.2.2.71 (official build), build id: 57F857DCFE8BA6838F6463E4665CD700852BFF0E, git hash: 9293d361e72be9f6ccfd444d504e2137b2e837cf) (from thread 2118603) Received signal 6 2024.04.08 13:15:29.534476 [ 2118604 ] {} BaseDaemon: Signal description: Aborted 2024.04.08 13:15:29.534484 [ 2118604 ] {} BaseDaemon: 2024.04.08 13:15:29.534510 [ 2118604 ] {} BaseDaemon: Stack trace: 0x00007f0fe474f00b 0x00007f0fe472e859 0x000000000d24f72e 0x0000000017d15be3 0x0000000017d15818 0x0000000012fd350d 0x0000000012fdcc6e 0x000000000d25c65b 0x000000000d25e775 0x00007f0fe4906609 0x00007f0fe482b353 2024.04.08 13:15:29.534531 [ 2118604 ] {} BaseDaemon: ######################################## 2024.04.08 13:15:29.534609 [ 2118604 ] {} BaseDaemon: (version 24.2.2.71 (official build), build id: 57F857DCFE8BA6838F6463E4665CD700852BFF0E, git hash: 9293d361e72be9f6ccfd444d504e2137b2e837cf) (from thread 2118603) (no query) Received signal Aborted (6) 2024.04.08 13:15:29.534638 [ 2118604 ] {} BaseDaemon: 2024.04.08 13:15:29.534663 [ 2118604 ] {} BaseDaemon: Stack trace: 0x00007f0fe474f00b 0x00007f0fe472e859 0x000000000d24f72e 0x0000000017d15be3 0x0000000017d15818 0x0000000012fd350d 0x0000000012fdcc6e 0x000000000d25c65b 0x000000000d25e775 0x00007f0fe4906609 0x00007f0fe482b353 2024.04.08 13:15:29.534711 [ 2118604 ] {} BaseDaemon: 2. ? @ 0x00007f0fe474f00b 2024.04.08 13:15:29.534728 [ 2118604 ] {} BaseDaemon: 3. ? @ 0x00007f0fe472e859 2024.04.08 13:15:29.613230 [ 2118604 ] {} BaseDaemon: 4.0. inlined from ./contrib/llvm-project/libcxxabi/src/cxa_exception.cpp:670: __cxa_decrement_exception_refcount 2024.04.08 13:15:29.613283 [ 2118604 ] {} BaseDaemon: 4.1. inlined from ./contrib/llvm-project/libcxx/src/support/runtime/exception_pointer_cxxabi.ipp:17: ~exception_ptr 2024.04.08 13:15:29.613311 [ 2118604 ] {} BaseDaemon: 4. ./build_docker/./src/Daemon/BaseDaemon.cpp:591: terminate_handler() @ 0x000000000d24f72e 2024.04.08 13:15:29.617590 [ 2118604 ] {} BaseDaemon: 5. ./build_docker/./contrib/llvm-project/libcxxabi/src/cxa_handlers.cpp:61: std::__terminate(void (*)()) @ 0x0000000017d15be3 2024.04.08 13:15:29.619575 [ 2118604 ] {} BaseDaemon: 6. ./build_docker/./contrib/llvm-project/libcxxabi/src/cxa_exception.cpp:0: __cxa_rethrow @ 0x0000000017d15818 2024.04.08 13:15:30.104097 [ 2118604 ] {} BaseDaemon: 7.0. inlined from ./src/IO/WriteBuffer.h:0: DB::WriteBuffer::next() 2024.04.08 13:15:30.104331 [ 2118604 ] {} BaseDaemon: 7.1. inlined from ./build_docker/./src/Server/TCPHandler.cpp:2225: DB::TCPHandler::sendLogData(DB::Block const&) 2024.04.08 13:15:30.104408 [ 2118604 ] {} BaseDaemon: 7. ./build_docker/./src/Server/TCPHandler.cpp:2303: DB::TCPHandler::sendLogs() @ 0x0000000012fd350d 2024.04.08 13:15:30.217481 [ 2118604 ] {} BaseDaemon: 8.0. inlined from ./contrib/llvm-project/libcxx/include/atomic:958: double std::__cxx_atomic_load[abi:v15000](std::__cxx_atomic_base_impl const*, std::memory_order) 2024.04.08 13:15:30.217579 [ 2118604 ] {} BaseDaemon: 8.1. inlined from ./contrib/llvm-project/libcxx/include/atomic:1588: std::__atomic_base::load[abi:v15000](std::memory_order) const 2024.04.08 13:15:30.217617 [ 2118604 ] {} BaseDaemon: 8.2. inlined from ./src/Common/ThreadFuzzer.cpp:407: pthread_mutex_unlock 2024.04.08 13:15:30.217644 [ 2118604 ] {} BaseDaemon: 8.3. inlined from ./contrib/llvm-project/libcxx/include/__threading_support:314: std::__libcpp_mutex_unlock[abi:v15000](pthread_mutex_t*) 2024.04.08 13:15:30.217676 [ 2118604 ] {} BaseDaemon: 8.4. inlined from ./contrib/llvm-project/libcxx/src/mutex.cpp:52: std::mutex::unlock() 2024.04.08 13:15:30.217699 [ 2118604 ] {} BaseDaemon: 8.5. inlined from ./contrib/llvm-project/libcxx/include/__mutex_base:100: ~lock_guard 2024.04.08 13:15:30.217747 [ 2118604 ] {} BaseDaemon: 8.6. inlined from ./build_docker/./src/Server/TCPHandler.cpp:392: operator() 2024.04.08 13:15:30.217776 [ 2118604 ] {} BaseDaemon: 8.7. inlined from ./contrib/llvm-project/libcxx/include/__functional/invoke.h:394: ? 2024.04.08 13:15:30.217796 [ 2118604 ] {} BaseDaemon: 8.8. inlined from ./contrib/llvm-project/libcxx/include/__functional/invoke.h:479: ? 2024.04.08 13:15:30.217859 [ 2118604 ] {} BaseDaemon: 8.9. inlined from ./contrib/llvm-project/libcxx/include/__functional/function.h:235: ? 2024.04.08 13:15:30.217878 [ 2118604 ] {} BaseDaemon: 8. ./contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x0000000012fdcc6e 2024.04.08 13:15:30.240809 [ 2118604 ] {} BaseDaemon: 9. ./contrib/llvm-project/libcxx/include/__functional/function.h:0: ? @ 0x000000000d25c65b 2024.04.08 13:15:30.283617 [ 2118604 ] {} BaseDaemon: 10.0. inlined from ./build_docker/./src/Daemon/BaseDaemon.cpp:286: operator() 2024.04.08 13:15:30.283686 [ 2118604 ] {} BaseDaemon: 10.1. inlined from ./contrib/llvm-project/libcxx/include/__functional/invoke.h:394: ? 2024.04.08 13:15:30.283725 [ 2118604 ] {} BaseDaemon: 10.2. inlined from ./contrib/llvm-project/libcxx/include/thread:284: void std::__thread_execute[abi:v15000]>, SignalListener::run()::'lambda'()>(std::tuple>, SignalListener::run()::'lambda'()>&, std::__tuple_indices<>) 2024.04.08 13:15:30.283755 [ 2118604 ] {} BaseDaemon: 10. ./contrib/llvm-project/libcxx/include/thread:295: void* std::__thread_proxy[abi:v15000]>, SignalListener::run()::'lambda'()>>(void*) @ 0x000000000d25e775 2024.04.08 13:15:30.283799 [ 2118604 ] {} BaseDaemon: 11. ? @ 0x00007f0fe4906609 2024.04.08 13:15:30.283821 [ 2118604 ] {} BaseDaemon: 12. ? @ 0x00007f0fe482b353 2024.04.08 13:15:30.574588 [ 2118604 ] {} BaseDaemon: Integrity check of the executable successfully passed (checksum: 3485110FABDB0C94202BD684999A9814) 2024.04.08 13:15:30.574704 [ 2118604 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues
v2: fatal logging Signed-off-by: Azat Khuzhin --- src/Daemon/BaseDaemon.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index cc22db3969c..019ad8e716b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -332,6 +332,7 @@ private: const std::vector & thread_frame_pointers, UInt32 thread_num, ThreadStatus * thread_ptr) const + try { ThreadStatus thread_status; @@ -543,6 +544,11 @@ private: fatal_error_printed.test_and_set(); } + catch (...) + { + PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); + LOG_FATAL(getLogger(__PRETTY_FUNCTION__), message); + } }; From 5db0df0bc1ff7be9e39dac189c7bfe2dfb6179ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 Apr 2024 21:48:36 +0300 Subject: [PATCH 198/243] Update BaseDaemon.cpp --- src/Daemon/BaseDaemon.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 019ad8e716b..592ca4e55d5 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -520,7 +520,7 @@ private: } } - /// ClickHouse Keeper does not link to some part of Settings. + /// ClickHouse Keeper does not link to some parts of Settings. #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD /// List changed settings. if (!query_id.empty()) @@ -538,7 +538,7 @@ private: } #endif - /// When everything is done, we will try to send these error messages to client. + /// When everything is done, we will try to send these error messages to the client. if (thread_ptr) thread_ptr->onFatalError(); @@ -546,6 +546,7 @@ private: } catch (...) { + /// onFault is called from the std::thread, and it should catch all exceptions; otherwise, you can get unrelated fatal errors. PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); LOG_FATAL(getLogger(__PRETTY_FUNCTION__), message); } From 33eba42c13c6894b05972df9588b49a78051d5f0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 20:50:41 +0200 Subject: [PATCH 199/243] Document wyHash64 --- .../sql-reference/functions/hash-functions.md | 38 ++++++++++++++++++- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 90c7d8c2206..ab97e43c624 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1077,9 +1077,7 @@ Result: ## wordShingleSimHashUTF8 -Splits a UTF-8 string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case sensitive. -Can be used for detection of semi-duplicate strings with [bitHammingDistance](/docs/en/sql-reference/functions/bit-functions.md/#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. **Syntax** @@ -1153,6 +1151,42 @@ Result: └────────────┘ ``` +## wyHash64 + +Produces a 64-bit [wyHash64](https://github.com/wangyi-fudan/wyhash) hash value. + +**Syntax** + +```sql +wyHash64(string) +``` + +**Arguments** + +- `string` — String. [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT wyHash64('ClickHouse') AS Hash; +``` + +Result: + +```response +┌─────────────────Hash─┐ +│ 12336419557878201794 │ +└──────────────────────┘ +``` + ## ngramMinHash Splits a ASCII string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive. From 9a08f1ddd6f414097a6bf00f1159924d4a14b4e4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 8 Apr 2024 20:53:19 +0200 Subject: [PATCH 200/243] Update StorageFileLog.cpp --- src/Storages/FileLog/StorageFileLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7b0cfdf6a6c..ba6528b6e4c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -152,7 +152,7 @@ StorageFileLog::StorageFileLog( if (!fileOrSymlinkPathStartsWith(path, getContext()->getUserFilesPath())) { - if (LoadingStrictnessLevel::ATTACH <= mode) + if (LoadingStrictnessLevel::SECONDARY_CREATE <= mode) { LOG_ERROR(log, "The absolute data path should be inside `user_files_path`({})", getContext()->getUserFilesPath()); return; From 3f91ece54b140906f050c0d6fc11563b81e48364 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 20:54:42 +0200 Subject: [PATCH 201/243] Remove blank space from wordShingleSimHashUTF8 --- docs/en/sql-reference/functions/hash-functions.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index ab97e43c624..9bfaaae5463 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1077,8 +1077,6 @@ Result: ## wordShingleSimHashUTF8 - - **Syntax** ```sql From 1b9a72e374f0194d3ab856f5cc0cd491a65af9a4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 20:56:30 +0200 Subject: [PATCH 202/243] Add back accidently removed description --- docs/en/sql-reference/functions/hash-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 9bfaaae5463..902e5ab9baf 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1077,6 +1077,10 @@ Result: ## wordShingleSimHashUTF8 +Splits a UTF-8 string into parts (shingles) of `shinglesize` words and returns the word shingle `simhash`. Is case sensitive. + +Can be used for detection of semi-duplicate strings with [bitHammingDistance](/docs/en/sql-reference/functions/bit-functions.md/#bithammingdistance). The smaller is the [Hamming Distance](https://en.wikipedia.org/wiki/Hamming_distance) of the calculated `simhashes` of two strings, the more likely these strings are the same. + **Syntax** ```sql From 2280fdeec1d41fdeb7a09459577312de8dc70bec Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Apr 2024 19:16:47 +0000 Subject: [PATCH 203/243] Empty commit From b138b1e103d6ccba62620b849931a9a607e9a42b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 8 Apr 2024 19:18:44 +0000 Subject: [PATCH 204/243] Empty commit From 208722a5133caac4f8d1e54afd0d301a1ff0d2de Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 21:55:27 +0200 Subject: [PATCH 205/243] Fix spelling mistake --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 8aff8b7e930..9b457880991 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -13,7 +13,7 @@ Case-insensitive search follows the lowercase-uppercase rules of the English lan `I` whereas in the Turkish language it is `İ` - results for languages other than English may be unexpected. ::: -Functions in this section also assume that the searched string (refered to in this section as `haystack`) and the search string (refered to in this section as `needle`) are single-byte encoded text. If this assumption is +Functions in this section also assume that the searched string (referred to in this section as `haystack`) and the search string (referred to in this section as `needle`) are single-byte encoded text. If this assumption is violated, no exception is thrown and results are undefined. Search with UTF-8 encoded strings is usually provided by separate function variants. Likewise, if a UTF-8 function variant is used and the input strings are not UTF-8 encoded text, no exception is thrown and the results are undefined. Note that no automatic Unicode normalization is performed, however you can use the From 2c41bcb25aae59f4f7964ef112e0a9c426c65f27 Mon Sep 17 00:00:00 2001 From: Blargian Date: Mon, 8 Apr 2024 21:57:11 +0200 Subject: [PATCH 206/243] Add multiSearchXYZ functions to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 57a8e0d5840..ebc53845011 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -550,6 +550,17 @@ Mongodb mortonDecode mortonEncode MsgPack +multiSearchAllPositionsCaseInsensitive +multiSearchAllPositionsCaseInsensitiveUTF +multiSearchAnyCaseInsensitive +multiSearchAnyCaseInsensitiveUTF +multiSearchAnyUTF +multiSearchFirstIndexCaseInsensitive +multiSearchFirstIndexCaseInsensitiveUTF +multiSearchFirstIndexUTF +multiSearchFirstPositionCaseInsensitive +multiSearchFirstPositionCaseInsensitiveUTF +multiSearchFirstPositionUTF MultiPolygon Multiline Multiqueries From f20771542633e85b07f49674e861740f44d1fe3e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 8 Apr 2024 22:25:50 +0200 Subject: [PATCH 207/243] Update 03068_analyzer_distributed_join.sql --- tests/queries/0_stateless/03068_analyzer_distributed_join.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql index 61b1199dc44..542380feb7c 100644 --- a/tests/queries/0_stateless/03068_analyzer_distributed_join.sql +++ b/tests/queries/0_stateless/03068_analyzer_distributed_join.sql @@ -1,5 +1,5 @@ --- https://github.com/ClickHouse/ClickHouse/issues/6571 --- Tag: no-replicated-database +-- Tags: no-replicated-database +-- Closes: https://github.com/ClickHouse/ClickHouse/issues/6571 SET allow_experimental_analyzer=1; CREATE TABLE LINEITEM_shard ON CLUSTER test_shard_localhost From eac8852c75da47991fa85c0611e8ef53c7059474 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 9 Apr 2024 00:14:49 +0200 Subject: [PATCH 208/243] Less flaky tests --- ...8_analyzer_ambiguous_column_multi_call.sql | 12 +++---- ...same_table_name_in_different_databases.sql | 32 +++++++++---------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql index 4ca5005fa1d..e6f1ed81f91 100644 --- a/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql +++ b/tests/queries/0_stateless/03088_analyzer_ambiguous_column_multi_call.sql @@ -1,13 +1,13 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61014 SET allow_experimental_analyzer=1; -DROP DATABASE IF EXISTS test_03088; -create database test_03088; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +create database {CLICKHOUSE_DATABASE:Identifier}; -create table test_03088.a (i int) engine = Log(); +create table {CLICKHOUSE_DATABASE:Identifier}.a (i int) engine = Log(); select - test_03088.a.i + {CLICKHOUSE_DATABASE:Identifier}.a.i from - test_03088.a, - test_03088.a as x; + {CLICKHOUSE_DATABASE:Identifier}.a, + {CLICKHOUSE_DATABASE:Identifier}.a as x; diff --git a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql index 2185b5f450a..436f9395fc4 100644 --- a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql +++ b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql @@ -1,28 +1,28 @@ -- https://github.com/ClickHouse/ClickHouse/issues/61947 SET allow_experimental_analyzer=1; -DROP DATABASE IF EXISTS d1; -DROP DATABASE IF EXISTS d2; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE d1; -CREATE DATABASE d2; -CREATE TABLE d1.`1-1` (field Int8) ENGINE = Memory; -CREATE TABLE d2.`1-1` (field Int8) ENGINE = Memory; -CREATE TABLE d2.`2-1` (field Int8) ENGINE = Memory; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` (field Int8) ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.`2-1` (field Int8) ENGINE = Memory; -INSERT INTO d1.`1-1` VALUES (1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.`1-1` VALUES (1); SELECT * -FROM d1.`1-1` -LEFT JOIN d2.`1-1` ON d1.`1-1`.field = d2.`1-1`.field; +FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` +LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` ON {CLICKHOUSE_DATABASE:Identifier}.`1-1`.field = {CLICKHOUSE_DATABASE_1:Identifier}.`1-1`.field; SELECT ''; -SELECT 'using asterisk', d1.`1-1`.*, d2.`1-1`.* -FROM d1.`1-1` -LEFT JOIN d2.`1-1` USING field +SELECT 'using asterisk', {CLICKHOUSE_DATABASE:Identifier}.`1-1`.*, {CLICKHOUSE_DATABASE_1:Identifier}.`1-1`.* +FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` +LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` USING field UNION ALL -SELECT 'using field name', d1.`1-1`.field, d2.`1-1`.field -FROM d1.`1-1` -LEFT JOIN d2.`1-1` USING field +SELECT 'using field name', {CLICKHOUSE_DATABASE:Identifier}.`1-1`.field, {CLICKHOUSE_DATABASE_1:Identifier}.`1-1`.field +FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` +LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` USING field ORDER BY *; From f45acef1316dd1f15becd386e880471642765f44 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 9 Apr 2024 00:33:18 +0000 Subject: [PATCH 209/243] Support for a tiny feature in stateless tests image --- docker/test/stateless/run.sh | 53 ++++++++++++++++++++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index bac9d8df7a9..24b821f1d7d 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -41,6 +41,8 @@ source /utils.lib if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then echo "Azure is disabled" +elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + echo "Azure is disabled" else azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & fi @@ -137,6 +139,32 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi +if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \ + | sed "s|/var/lib/clickhouse/filesystem_caches/|/var/lib/clickhouse/filesystem_caches_1/|" \ + > /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp + mv /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server1/config.d/filesystem_caches_path.xml + + sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \ + | sed "s|/var/lib/clickhouse/filesystem_caches/|/var/lib/clickhouse/filesystem_caches_1/|" \ + > /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp + mv /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server1/config.d/filesystem_caches_path.xml + + mkdir -p /var/run/clickhouse-server1 + sudo chown clickhouse:clickhouse /var/run/clickhouse-server1 + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ + --pid-file /var/run/clickhouse-server1/clickhouse-server.pid \ + -- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --mysql_port 19004 --postgresql_port 19005 \ + --keeper_server.tcp_port 19181 --keeper_server.server_id 2 \ + --prometheus.port 19988 \ + --macros.replica r2 # It doesn't work :( + + MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) + MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) +fi # Wait for the server to start, but not for too long. for _ in {1..100} @@ -183,6 +211,10 @@ function run_tests() ADDITIONAL_OPTIONS+=('--s3-storage') fi + if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--shared-catalog') + fi + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') # Too many tests fail for DatabaseReplicated in parallel. @@ -264,6 +296,12 @@ do echo "$err" [[ "0" != "${#err}" ]] && failed_to_save_logs=1 fi + + if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 ) + echo "$err" + [[ "0" != "${#err}" ]] && failed_to_save_logs=1 + fi done # Stop server so we can safely read data with clickhouse-local. @@ -275,6 +313,10 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] sudo clickhouse stop --pid-path /var/run/clickhouse-server2 ||: fi +if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: +fi + rg -Fa "" /var/log/clickhouse-server/clickhouse-server.log ||: rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst & @@ -302,6 +344,10 @@ if [ $failed_to_save_logs -ne 0 ]; then clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||: fi + + if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: + fi done fi @@ -341,3 +387,10 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: tar -chf /test_output/coordination2.tar /var/lib/clickhouse2/coordination ||: fi + +if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then + rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: + zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||: + mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: + tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||: +fi From 0265ba502716d8536c85ea8914ba791f1d278c66 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 10:55:08 +0200 Subject: [PATCH 210/243] Add wyHash to aspell-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 8aa2a463c47..84c70d91879 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2760,6 +2760,7 @@ wordShingleSimHashUTF wordshingleMinHash writability wrt +wyHash xcode xeus xkcd From 6e90a197f8e88a2d0a74490e96455bc203365295 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 9 Apr 2024 12:02:11 +0200 Subject: [PATCH 211/243] Update 03091_analyzer_same_table_name_in_different_databases.sql --- ...03091_analyzer_same_table_name_in_different_databases.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql index 436f9395fc4..599275c66e8 100644 --- a/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql +++ b/tests/queries/0_stateless/03091_analyzer_same_table_name_in_different_databases.sql @@ -18,6 +18,8 @@ LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` ON {CLICKHOUSE_DATABASE:Ident SELECT ''; +SELECT * FROM +( SELECT 'using asterisk', {CLICKHOUSE_DATABASE:Identifier}.`1-1`.*, {CLICKHOUSE_DATABASE_1:Identifier}.`1-1`.* FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` USING field @@ -25,4 +27,5 @@ UNION ALL SELECT 'using field name', {CLICKHOUSE_DATABASE:Identifier}.`1-1`.field, {CLICKHOUSE_DATABASE_1:Identifier}.`1-1`.field FROM {CLICKHOUSE_DATABASE:Identifier}.`1-1` LEFT JOIN {CLICKHOUSE_DATABASE_1:Identifier}.`1-1` USING field -ORDER BY *; +) +ORDER BY ALL; From 0543fc3263dba3cf56e1445725f3589e43d91b21 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 9 Apr 2024 13:17:42 +0300 Subject: [PATCH 212/243] OptimizeGroupByInjectiveFunctionsPass remove unused constant --- .../OptimizeGroupByInjectiveFunctionsPass.cpp | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp index 61893202525..a30ad2a1590 100644 --- a/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.cpp @@ -12,24 +12,6 @@ namespace DB namespace { -const std::unordered_set possibly_injective_function_names -{ - "dictGet", - "dictGetString", - "dictGetUInt8", - "dictGetUInt16", - "dictGetUInt32", - "dictGetUInt64", - "dictGetInt8", - "dictGetInt16", - "dictGetInt32", - "dictGetInt64", - "dictGetFloat32", - "dictGetFloat64", - "dictGetDate", - "dictGetDateTime" -}; - class OptimizeGroupByInjectiveFunctionsVisitor : public InDepthQueryTreeVisitorWithContext { using Base = InDepthQueryTreeVisitorWithContext; From 4e344f6a9397d27a15af82023b182469eaeebd35 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 9 Apr 2024 10:25:41 +0000 Subject: [PATCH 213/243] remove ci status and reports for MQ case --- tests/ci/commit_status_helper.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index bda2db13991..56728c3d3ba 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -148,6 +148,11 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: """It adds or updates the comment status to all Pull Requests but for release one, so the method does nothing for simple pushes and pull requests with `release`/`release-lts` labels""" + + if pr_info.is_merge_queue(): + # skip report creation for the MQ + return + # to reduce number of parameters, the Github is constructed on the fly gh = Github() gh.__requester = commit._requester # type:ignore #pylint:disable=protected-access @@ -441,7 +446,9 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> or pr_info.release_pr or pr_info.number == 0 ) - if not_run: + + # FIXME: For now, always set mergeable check in the Merge Queue. It's required to pass MQ + if not_run and not pr_info.is_merge_queue(): # Let's avoid unnecessary work return From fe868ddf86ccaa60bccbe44afbfbe1e24ecf6c5d Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 12:31:35 +0200 Subject: [PATCH 214/243] Document uniqCombined64 and update uniqueCombined --- .../reference/uniqcombined.md | 19 +++-- .../reference/uniqcombined64.md | 76 ++++++++++++++++++- 2 files changed, 86 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md index 2f3efde859d..99a46c14a30 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -15,9 +15,9 @@ The `uniqCombined` function is a good choice for calculating the number of diffe **Arguments** -The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. +- `HLL_precision`: The base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each). +- `X`: A variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. -`HLL_precision` is the base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each). **Returned value** @@ -25,26 +25,29 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, ` **Implementation details** -Function: +The `uniqCombined` function: - Calculates a hash (64-bit hash for `String` and 32-bit otherwise) for all parameters in the aggregate, then uses it in calculations. - - Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table. - - For a small number of distinct elements, an array is used. When the set size is larger, a hash table is used. For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. - + - For a small number of distinct elements, an array is used. + - When the set size is larger, a hash table is used. + - For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. - Provides the result deterministically (it does not depend on the query processing order). :::note Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) ::: -Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined`: +Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined` function: - Consumes several times less memory. - Calculates with several times higher accuracy. - Usually has slightly lower performance. In some scenarios, `uniqCombined` can perform better than `uniq`, for example, with distributed queries that transmit a large number of aggregation states over the network. +**Example** + + + **See Also** - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md index 9f010da57f2..b6e09bcaae3 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md @@ -5,4 +5,78 @@ sidebar_position: 193 # uniqCombined64 -Same as [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined), but uses 64-bit hash for all data types. +Calculates the approximate number of different argument values. It is the same as [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined), but uses a 64-bit hash for all data types rather than just for the String data type. + +``` sql +uniqCombined64(HLL_precision)(x[, ...]) +``` + +**Parameters** + +- `HLL_precision`: The base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optionally, you can use the function as `uniqCombined64(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each). +- `X`: A variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types. + +**Returned value** + +- A number [UInt64](../../../sql-reference/data-types/int-uint.md)-type number. + +**Implementation details** + +The `uniqCombined64` function: +- Calculates a hash (64-bit hash for all data types) for all parameters in the aggregate, then uses it in calculations. +- Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table. + - For a small number of distinct elements, an array is used. + - When the set size is larger, a hash table is used. + - For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. +- Provides the result deterministically (it does not depend on the query processing order). + +:::note +Since it uses 64-bit hash for all types, the result does not suffer from very high error for cardinalities significantly larger than `UINT_MAX` like [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) does, which uses a 32-bit hash for non-`String` types. +::: + +Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined64` function: + +- Consumes several times less memory. +- Calculates with several times higher accuracy. + +**Example** + +In the example below `uniqCombined64` is run on `1e10` different numbers returning a very close approximation of the number of different argument values. + +Query: + +```sql +SELECT uniqCombined64(number) FROM numbers(1e10); +``` + +Result: + +```response +┌─uniqCombined64(number)─┐ +│ 9998568925 │ -- 10.00 billion +└────────────────────────┘ +``` + +By comparison the `uniqCombined` function returns a rather poor approximation for an input this size. + +Query: + +```sql +SELECT uniqCombined(number) FROM numbers(1e10); +``` + +Result: + +```response +┌─uniqCombined(number)─┐ +│ 5545308725 │ -- 5.55 billion +└──────────────────────┘ +``` + +**See Also** + +- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) +- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) +- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) +- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) +- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch) From af505aafbca25335b3f2a5659e681af373189231 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 12:34:53 +0200 Subject: [PATCH 215/243] Small fix --- .../aggregate-functions/reference/uniqcombined.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md index 99a46c14a30..4d47aa6621e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -29,13 +29,13 @@ The `uniqCombined` function: - Calculates a hash (64-bit hash for `String` and 32-bit otherwise) for all parameters in the aggregate, then uses it in calculations. - Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table. - - For a small number of distinct elements, an array is used. - - When the set size is larger, a hash table is used. - - For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. + - For a small number of distinct elements, an array is used. + - When the set size is larger, a hash table is used. + - For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory. - Provides the result deterministically (it does not depend on the query processing order). :::note -Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) +Since it uses a 32-bit hash for non-`String` types, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64). ::: Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined` function: From 7b3a973ee07d55ea6f8fe2cf6dfe2475e8cefc35 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 12:55:01 +0200 Subject: [PATCH 216/243] Add missing example to uniqCombined --- .../aggregate-functions/reference/uniqcombined.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md index 4d47aa6621e..18f44d2fcc4 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -46,7 +46,21 @@ Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq **Example** +Query: +```sql +SELECT uniqCombined(number) FROM numbers(1e6); +``` + +Result: + +```response +┌─uniqCombined(number)─┐ +│ 1001148 │ -- 1.00 million +└──────────────────────┘ +``` + +See the example section of [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) for an example of the difference between `uniqCombined` and `uniqCombined64` for much larger inputs. **See Also** From 35e1e5aff7df25401ab3db59399f5bd49ba0deea Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 9 Apr 2024 14:42:29 +0300 Subject: [PATCH 217/243] Perf script update path in documentation --- tests/performance/scripts/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/scripts/README.md b/tests/performance/scripts/README.md index 0a0580c62a0..1a15189fe86 100644 --- a/tests/performance/scripts/README.md +++ b/tests/performance/scripts/README.md @@ -130,7 +130,7 @@ More stages are available, e.g. restart servers or run the tests. See the code. #### Run a single test on the already configured servers ``` -docker/test/performance-comparison/perf.py --host=localhost --port=9000 --runs=1 tests/performance/logical_functions_small.xml +tests/performance/scripts/perf.py --host=localhost --port=9000 --runs=1 tests/performance/logical_functions_small.xml ``` #### Run all tests on some custom configuration From 55798dbdcf4222031afbf945f099d5f3bc111659 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 13:53:31 +0200 Subject: [PATCH 218/243] Add tupleIntDiv function --- .../functions/tuple-functions.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index b089de67e98..873065d226b 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -584,6 +584,59 @@ SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res └──────────────────────┘ ``` +## tupleIntDiv + +Does integer division of two tuples. + +**Syntax** + +```sql +tupleIntDiv(tuple1, tuple2) +``` + +**Parameters** + +- `tuple1`: tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `tuple2`: tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. + +**Returned value** + +- Tuple of the quotients of `tuple1` and `tuple2`. [Tuple](../data-types/tuple) of integer values. + +**Implementation details** + +- If either `tuple1` or `tuple2` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. + +**Examples** + +Query: + +``` sql +SELECT tupleIntDiv((15, 10, 5),(5, 5, 5)); +``` + +Result: + +``` text +┌─tupleIntDiv((15, 10, 5), (5, 5, 5))─┐ +│ (3,2,1) │ +└─────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT tupleIntDiv((15, 10, 5),(5.5, 5.5, 5.5)); +``` + +Result: + +``` text +┌─tupleIntDiv((15, 10, 5), (5.5, 5.5, 5.5))─┐ +│ (2,1,0) │ +└───────────────────────────────────────────┘ +``` + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From eed1a33e775599f7fa14dfdcdcb49f41939a9c1d Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 13:55:10 +0200 Subject: [PATCH 219/243] update wording on tupleIntDiv --- docs/en/sql-reference/functions/tuple-functions.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 873065d226b..7b626f8ab62 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -586,7 +586,7 @@ SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res ## tupleIntDiv -Does integer division of two tuples. +Does integer division of two tuples, returns a tuple of the quotients. **Syntax** @@ -637,6 +637,9 @@ Result: └───────────────────────────────────────────┘ ``` +## tupleIntDivByNumber + + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From 5857ecf8481b60e101c1550a073a2176c59b8f61 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 14:01:50 +0200 Subject: [PATCH 220/243] Add tupleIntDivByNumber --- .../functions/tuple-functions.md | 62 +++++++++++++++++-- 1 file changed, 56 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 7b626f8ab62..58613c6194b 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -586,26 +586,26 @@ SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res ## tupleIntDiv -Does integer division of two tuples, returns a tuple of the quotients. +Does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients. **Syntax** ```sql -tupleIntDiv(tuple1, tuple2) +tupleIntDiv(tuple_num, tuple_div) ``` **Parameters** -- `tuple1`: tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. -- `tuple2`: tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. **Returned value** -- Tuple of the quotients of `tuple1` and `tuple2`. [Tuple](../data-types/tuple) of integer values. +- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values. **Implementation details** -- If either `tuple1` or `tuple2` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. +- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. **Examples** @@ -639,6 +639,56 @@ Result: ## tupleIntDivByNumber +Does integer division of a tuple of numerators by a given denominator, and returns a tuple of the quotients. + +**Syntax** + +```sql +tupleIntDivByNumber(tuple_num, div) +``` + +**Parameters** + +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `div`: The divisor value. [Tuple](../data-types/tuple) of numeric type. + +**Returned value** + +- Tuple of the quotients of `tuple_num` and `div`. [Tuple](../data-types/tuple) of integer values. + +**Implementation details** + +- If either `tuple_num` or `div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. + +**Examples** + +Query: + +``` sql +SELECT tupleIntDivByNumber((15, 10, 5),5); +``` + +Result: + +``` text +┌─tupleIntDivByNumber((15, 10, 5), 5)─┐ +│ (3,2,1) │ +└─────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT tupleIntDivByNumber((15.2, 10.7, 5.5),5.8); +``` + +Result: + +``` text +┌─tupleIntDivByNumber((15.2, 10.7, 5.5), 5.8)─┐ +│ (2,1,0) │ +└─────────────────────────────────────────────┘ +``` ## Distance functions From c8fb88b261b4bf1bbab57fb82bf895d471308126 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 14:15:32 +0200 Subject: [PATCH 221/243] Add tupleIntDivOrZero --- .../functions/tuple-functions.md | 44 ++++++++++++++++++- 1 file changed, 42 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 58613c6194b..853514c9743 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -606,6 +606,7 @@ tupleIntDiv(tuple_num, tuple_div) **Implementation details** - If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. +- An error will be thrown for division by 0. **Examples** @@ -659,13 +660,14 @@ tupleIntDivByNumber(tuple_num, div) **Implementation details** - If either `tuple_num` or `div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor. +- An error will be thrown for division by 0. **Examples** Query: ``` sql -SELECT tupleIntDivByNumber((15, 10, 5),5); +SELECT tupleIntDivByNumber((15, 10, 5), 5); ``` Result: @@ -679,7 +681,7 @@ Result: Query: ``` sql -SELECT tupleIntDivByNumber((15.2, 10.7, 5.5),5.8); +SELECT tupleIntDivByNumber((15.2, 10.7, 5.5), 5.8); ``` Result: @@ -690,6 +692,44 @@ Result: └─────────────────────────────────────────────┘ ``` +## tupleIntDivOrZero + +Like [tupleIntDiv](#tupleintdiv) it does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients. Does not throw an error for 0 divisors, but rather returns the quotient as 0. + +**Syntax** + +```sql +tupleIntDivOrZero(tuple_num, tuple_div) +``` + +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. + +**Returned value** + +- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values. +- Returns 0 for quotients where the divisor is 0. + +**Implementation details** + +- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDiv](#tupleintdiv). + +**Examples** + +Query: + +``` sql +SELECT tupleIntDivOrZero((5, 10, 15),(0, 0, 0)); +``` + +Result: + +``` text +┌─tupleIntDivOrZero((5, 10, 15), (0, 0, 0))─┐ +│ (0,0,0) │ +└───────────────────────────────────────────┘ +``` + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From bba66eb05a95541915fd6e4eccd0beac26b16a3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 9 Apr 2024 12:30:07 +0000 Subject: [PATCH 222/243] Review fixes. --- src/Analyzer/Utils.cpp | 22 ++++++++++++++++ src/Analyzer/Utils.h | 3 +++ src/Planner/CollectSets.cpp | 34 +++---------------------- src/Planner/CollectSets.h | 4 --- src/Storages/buildQueryTreeForShard.cpp | 5 ++-- 5 files changed, 32 insertions(+), 36 deletions(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 8ccf95deadc..2882c4e0c02 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -760,4 +760,26 @@ QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_ty return function_node; } +QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context) +{ + const auto & storage_snapshot = table_node->as()->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + size_t columns_to_select_size = columns_to_select.size(); + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, table_node)); + projection_columns.emplace_back(column.name, column.type); + } + auto subquery_for_table = std::make_shared(Context::createCopy(context)); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(table_node); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + return subquery_for_table; +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 8e32ef0464c..1b4a7d5ef3c 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -105,4 +105,7 @@ NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node); /// Wrap node into `_CAST` function QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_type, ContextPtr context); +/// Build subquery which we execute for `IN table` function. +QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context); + } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 37502828f63..b1f2875210d 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -23,35 +23,6 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; } - -QueryTreeNodePtr makeExecutableSubqueryForIn(const QueryTreeNodePtr & in_second_argument, const ContextPtr & context) -{ - auto subquery_to_execute = in_second_argument; - if (auto * table_node = in_second_argument->as()) - { - auto storage_snapshot = table_node->getStorageSnapshot(); - auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); - size_t columns_to_select_size = columns_to_select.size(); - auto column_nodes_to_select = std::make_shared(); - column_nodes_to_select->getNodes().reserve(columns_to_select_size); - NamesAndTypes projection_columns; - projection_columns.reserve(columns_to_select_size); - for (auto & column : columns_to_select) - { - column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); - projection_columns.emplace_back(column.name, column.type); - } - auto subquery_for_table = std::make_shared(Context::createCopy(context)); - subquery_for_table->setIsSubquery(true); - subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); - subquery_for_table->getJoinTree() = std::move(subquery_to_execute); - subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); - subquery_to_execute = std::move(subquery_for_table); - } - - return subquery_to_execute; -} - namespace { @@ -116,7 +87,10 @@ public: if (sets.findSubquery(set_key)) return; - auto subquery_to_execute = makeExecutableSubqueryForIn(in_second_argument, planner_context.getQueryContext()); + auto subquery_to_execute = in_second_argument; + if (in_second_argument->as()) + subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(std::move(subquery_to_execute), planner_context.getQueryContext()); + sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings); } else diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 0ee006f3320..5f9f7a5a466 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -14,8 +14,4 @@ struct SelectQueryOptions; */ void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); -/// Build subquery which we execute for IN function. -/// It is needed to support `IN table` case. -QueryTreeNodePtr makeExecutableSubqueryForIn(const QueryTreeNodePtr & in_second_argument, const ContextPtr & context); - } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 09e48a93df4..5bbdbe487b0 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -365,7 +364,9 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION && in_function_node_type != QueryTreeNodeType::TABLE) continue; - auto subquery_to_execute = makeExecutableSubqueryForIn(in_function_subquery_node, planner_context->getQueryContext()); + auto subquery_to_execute = in_function_subquery_node; + if (subquery_to_execute->as()) + subquery_to_execute = buildSubqueryToReadColumnsFromTableExpression(std::move(subquery_to_execute), planner_context->getQueryContext()); auto temporary_table_expression_node = executeSubqueryNode(subquery_to_execute, planner_context->getMutableQueryContext(), From 92a027a2d0b5c92c7308925e314728588d8e52fa Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 15:01:27 +0200 Subject: [PATCH 223/243] Add OrZero variants --- .../functions/tuple-functions.md | 70 ++++++++++++++++--- 1 file changed, 62 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 853514c9743..2351cfd47d4 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -638,6 +638,44 @@ Result: └───────────────────────────────────────────┘ ``` +## tupleIntDivOrZero + +Like [tupleIntDiv](#tupleintdiv) it does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients. It does not throw an error for 0 divisors, but rather returns the quotient as 0. + +**Syntax** + +```sql +tupleIntDivOrZero(tuple_num, tuple_div) +``` + +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. + +**Returned value** + +- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values. +- Returns 0 for quotients where the divisor is 0. + +**Implementation details** + +- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDiv](#tupleintdiv). + +**Examples** + +Query: + +``` sql +SELECT tupleIntDivOrZero((5, 10, 15),(0, 0, 0)); +``` + +Result: + +``` text +┌─tupleIntDivOrZero((5, 10, 15), (0, 0, 0))─┐ +│ (0,0,0) │ +└───────────────────────────────────────────┘ +``` + ## tupleIntDivByNumber Does integer division of a tuple of numerators by a given denominator, and returns a tuple of the quotients. @@ -692,40 +730,56 @@ Result: └─────────────────────────────────────────────┘ ``` -## tupleIntDivOrZero +## tupleIntDivOrZeroByNumber -Like [tupleIntDiv](#tupleintdiv) it does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients. Does not throw an error for 0 divisors, but rather returns the quotient as 0. +Like [tupleIntDivByNumber](#tupleintdivbynumber) it does integer division of a tuple of numerators by a given denominator, and returns a tuple of the quotients. It does not throw an error for 0 divisors, but rather returns the quotient as 0. **Syntax** ```sql -tupleIntDivOrZero(tuple_num, tuple_div) +tupleIntDivOrZeroByNumber(tuple_num, div) ``` +**Parameters** + - `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. -- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type. +- `div`: The divisor value. [Tuple](../data-types/tuple) of numeric type. **Returned value** -- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values. +- Tuple of the quotients of `tuple_num` and `div`. [Tuple](../data-types/tuple) of integer values. - Returns 0 for quotients where the divisor is 0. **Implementation details** -- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDiv](#tupleintdiv). +- If either `tuple_num` or `div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDivByNumber](#tupleintdivbynumber). **Examples** Query: ``` sql -SELECT tupleIntDivOrZero((5, 10, 15),(0, 0, 0)); +SELECT tupleIntDivOrZeroByNumber((15, 10, 5),(5)); ``` Result: ``` text -┌─tupleIntDivOrZero((5, 10, 15), (0, 0, 0))─┐ +┌─tupleIntDivOrZeroByNumber((15, 10, 5), 5)─┐ +│ (3,2,1) │ +└───────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT tupleIntDivOrZeroByNumber((15, 10, 5),(0)) +``` + +Result: + +``` text +┌─tupleIntDivOrZeroByNumber((15, 10, 5), 0)─┐ │ (0,0,0) │ └───────────────────────────────────────────┘ ``` From 7d969dd9faad1226eb60876bff5aa0dcfad5bc93 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 15:19:31 +0200 Subject: [PATCH 224/243] Add tupleModulo tupleModuloByNumber --- .../functions/tuple-functions.md | 86 +++++++++++++++++-- 1 file changed, 79 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 2351cfd47d4..ba177ca3349 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -613,7 +613,7 @@ tupleIntDiv(tuple_num, tuple_div) Query: ``` sql -SELECT tupleIntDiv((15, 10, 5),(5, 5, 5)); +SELECT tupleIntDiv((15, 10, 5), (5, 5, 5)); ``` Result: @@ -627,7 +627,7 @@ Result: Query: ``` sql -SELECT tupleIntDiv((15, 10, 5),(5.5, 5.5, 5.5)); +SELECT tupleIntDiv((15, 10, 5), (5.5, 5.5, 5.5)); ``` Result: @@ -665,7 +665,7 @@ tupleIntDivOrZero(tuple_num, tuple_div) Query: ``` sql -SELECT tupleIntDivOrZero((5, 10, 15),(0, 0, 0)); +SELECT tupleIntDivOrZero((5, 10, 15), (0, 0, 0)); ``` Result: @@ -689,7 +689,7 @@ tupleIntDivByNumber(tuple_num, div) **Parameters** - `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. -- `div`: The divisor value. [Tuple](../data-types/tuple) of numeric type. +- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type. **Returned value** @@ -743,7 +743,7 @@ tupleIntDivOrZeroByNumber(tuple_num, div) **Parameters** - `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. -- `div`: The divisor value. [Tuple](../data-types/tuple) of numeric type. +- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type. **Returned value** @@ -759,7 +759,7 @@ tupleIntDivOrZeroByNumber(tuple_num, div) Query: ``` sql -SELECT tupleIntDivOrZeroByNumber((15, 10, 5),(5)); +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), (5)); ``` Result: @@ -773,7 +773,7 @@ Result: Query: ``` sql -SELECT tupleIntDivOrZeroByNumber((15, 10, 5),(0)) +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), (0)) ``` Result: @@ -784,6 +784,78 @@ Result: └───────────────────────────────────────────┘ ``` +## tupleModulo + +Returns a tuple of the moduli (remainders) of division operations of two tuples. + +**Syntax** + +```sql +tupleModulo(tuple_num, tuple_mod) +``` + +**Parameters** + +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `tuple_div`: Tuple of modulus values. [Tuple](../data-types/tuple) of numeric type. + +**Returned value** + +- Tuple of the remainders of division of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of non-zero integer values. +- An error is thrown for division by zero. + +**Examples** + +Query: + +``` sql +SELECT tupleModulo((15, 10, 5), (5, 3, 2)); +``` + +Result: + +``` text +┌─tupleModulo((15, 10, 5), (5, 3, 2))─┐ +│ (0,1,1) │ +└─────────────────────────────────────┘ +``` + +## tupleModuloByNumber + +Returns a tuple of the moduli (remainders) of division operations of a tuple and a given divisor. + +**Syntax** + +```sql +tupleModuloByNumber(tuple_num, div) +``` + +**Parameters** + +- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type. +- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type. + +**Returned value** + +- Tuple of the remainders of division of `tuple_num` and `div`. [Tuple](../data-types/tuple) of non-zero integer values. +- An error is thrown for division by zero. + +**Examples** + +Query: + +``` sql +SELECT tupleModuloByNumber((15, 10, 5), 2); +``` + +Result: + +``` text +┌─tupleModuloByNumber((15, 10, 5), 2)─┐ +│ (1,0,1) │ +└─────────────────────────────────────┘ +``` + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). From 5caa89a2b33efa86fab1bc6a6813e143c8f37f67 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 9 Apr 2024 15:36:34 +0200 Subject: [PATCH 225/243] Fix completion of available ClickHouse tools Now clickhouse --help/ch --help will print --help for clickhouse-local, let's use just "clickhouse help" to get help with list of available tools in clickhouse binary itself. Signed-off-by: Azat Khuzhin --- programs/bash-completion/completions/clickhouse | 2 +- programs/main.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/bash-completion/completions/clickhouse b/programs/bash-completion/completions/clickhouse index ff0a60c60be..3c895a66075 100644 --- a/programs/bash-completion/completions/clickhouse +++ b/programs/bash-completion/completions/clickhouse @@ -3,7 +3,7 @@ function _clickhouse_get_utils() { local cmd=$1 && shift - "$cmd" --help |& awk '/^clickhouse.*args/ { print $2 }' + "$cmd" help |& awk '/^clickhouse.*args/ { print $2 }' } function _complete_for_clickhouse_entrypoint_bin() diff --git a/programs/main.cpp b/programs/main.cpp index 7162a18d764..9ad8b016c82 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -487,7 +487,7 @@ int main(int argc_, char ** argv_) /// Interpret binary without argument or with arguments starts with dash /// ('-') as clickhouse-local for better usability: /// - /// clickhouse # dumps help + /// clickhouse help # dumps help /// clickhouse -q 'select 1' # use local /// clickhouse # spawn local /// clickhouse local # spawn local From e07a614006cd756e3e86b01d22e954ca83ef4143 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 15:52:59 +0200 Subject: [PATCH 226/243] Add missing tests for tupleIntXYZ and tupleModulo, tupleModuloByNumber --- .../03033_tupleIntXYZ_and_tupleModulo.reference | 9 +++++++++ .../03033_tupleIntXYZ_and_tupleModulo.sql | 13 +++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.reference create mode 100644 tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.sql diff --git a/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.reference b/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.reference new file mode 100644 index 00000000000..f0ce1649939 --- /dev/null +++ b/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.reference @@ -0,0 +1,9 @@ +(3,2,1) +(2,1,0) +(0,0,0) +(3,2,1) +(2,1,0) +(3,2,1) +(0,0,0) +(0,1,1) +(1,0,1) diff --git a/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.sql b/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.sql new file mode 100644 index 00000000000..2cb7e726a4b --- /dev/null +++ b/tests/queries/0_stateless/03033_tupleIntXYZ_and_tupleModulo.sql @@ -0,0 +1,13 @@ +SELECT tupleIntDiv((15, 10, 5), (0, 0, 0)); -- { serverError ILLEGAL_DIVISION } +SELECT tupleIntDiv((15, 10, 5), (5, 5, 5)); +SELECT tupleIntDiv((15, 10, 5), (5.5, 5.5, 5.5)); +SELECT tupleIntDivOrZero((5, 10, 15), (0, 0, 0)); -- no error thrown for zero divisors +SELECT tupleIntDivByNumber((15, 10, 5), 0); -- { serverError ILLEGAL_DIVISION } +SELECT tupleIntDivByNumber((15, 10, 5), 5); +SELECT tupleIntDivByNumber((15.2, 10.7, 5.5), 5.8); +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 5); +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 0); -- no error thrown for zero divisors +SELECT tupleModulo((15, 10, 5), (0, 3, 2)); -- { serverError ILLEGAL_DIVISION } +SELECT tupleModulo((15, 10, 5), (5, 3, 2)); +SELECT tupleModuloByNumber((15, 10, 5), 0); -- { serverError ILLEGAL_DIVISION } +SELECT tupleModuloByNumber((15, 10, 5), 2); \ No newline at end of file From de8d31685db079da0c4e734330a3d75a67a30c5a Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 16:26:35 +0200 Subject: [PATCH 227/243] Minor edit --- docs/en/sql-reference/functions/tuple-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index ba177ca3349..b3cec1206b8 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -759,7 +759,7 @@ tupleIntDivOrZeroByNumber(tuple_num, div) Query: ``` sql -SELECT tupleIntDivOrZeroByNumber((15, 10, 5), (5)); +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 5); ``` Result: @@ -773,7 +773,7 @@ Result: Query: ``` sql -SELECT tupleIntDivOrZeroByNumber((15, 10, 5), (0)) +SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 0) ``` Result: From d5014b2d0e3f19aba82cca1480146cec1772e7a0 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 16:44:07 +0200 Subject: [PATCH 228/243] Add missing L2SquaredNorm function --- .../functions/distance-functions.md | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/docs/en/sql-reference/functions/distance-functions.md b/docs/en/sql-reference/functions/distance-functions.md index e20c35c6b6f..eb991acc94b 100644 --- a/docs/en/sql-reference/functions/distance-functions.md +++ b/docs/en/sql-reference/functions/distance-functions.md @@ -82,6 +82,44 @@ Result: └──────────────────┘ ``` +## L2SquaredNorm + +Calculates the square root of the sum of the squares of the vector values (the [L2Norm](#l2norm)) squared. + +**Syntax** + +```sql +L2SquaredNorm(vector) +``` + +Alias: `normL2Squared`. + +***Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- L2-norm squared. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L2SquaredNorm((1, 2)); +``` + +Result: + +```text +┌─L2SquaredNorm((1, 2))─┐ +│ 5 │ +└───────────────────────┘ +``` + ## LinfNorm Calculates the maximum of absolute values of a vector. From 8369f8d8c18aea16a815a229f1bfd4af27a9f102 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 16:54:09 +0200 Subject: [PATCH 229/243] Add missing l2SquaredNorm function --- .../functions/distance-functions.md | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/docs/en/sql-reference/functions/distance-functions.md b/docs/en/sql-reference/functions/distance-functions.md index e20c35c6b6f..5f3514049c7 100644 --- a/docs/en/sql-reference/functions/distance-functions.md +++ b/docs/en/sql-reference/functions/distance-functions.md @@ -81,6 +81,43 @@ Result: │ 2.23606797749979 │ └──────────────────┘ ``` +## L2SquaredNorm + +Calculates the square root of the sum of the squares of the vector values (the [L2Norm](#l2norm)) squared. + +**Syntax** + +```sql +L2SquaredNorm(vector) +``` + +Alias: `normL2Squared`. + +***Arguments** + +- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). + +**Returned value** + +- L2-norm squared. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT L2SquaredNorm((1, 2)); +``` + +Result: + +```text +┌─L2SquaredNorm((1, 2))─┐ +│ 5 │ +└───────────────────────┘ +``` ## LinfNorm From 283fc115ba3c57a12fbacb8afd0af7cc332722f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 9 Apr 2024 15:17:26 +0000 Subject: [PATCH 230/243] Updating the test. --- tests/integration/test_cluster_all_replicas/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_cluster_all_replicas/test.py b/tests/integration/test_cluster_all_replicas/test.py index 59b41ca87af..d8bad180e1b 100644 --- a/tests/integration/test_cluster_all_replicas/test.py +++ b/tests/integration/test_cluster_all_replicas/test.py @@ -43,7 +43,8 @@ def test_cluster(start_cluster): def test_global_in(start_cluster): - node1.query("CREATE TABLE u(uid Int16) ENGINE=Log as select 0") + node1.query("DROP TABLE IF EXISTS u;") + node1.query("CREATE TABLE u(uid Int16) ENGINE=Memory as select 0") assert set( node1.query( From 7aac552fabfa9a2fe19ca6930000eeee395e8752 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 17:44:41 +0200 Subject: [PATCH 231/243] Add missing kostikConsistentHash --- .../sql-reference/functions/hash-functions.md | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 90c7d8c2206..5bfacd34e19 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -594,6 +594,45 @@ Calculates JumpConsistentHash form a UInt64. Accepts two arguments: a UInt64-type key and the number of buckets. Returns Int32. For more information, see the link: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) +## kostikConsistentHash + +An O(1) time and space consistent hash algorithm by Konstantin 'kostik' Oblakov. Previously `yandexConsistentHash`. + +**Syntax** + +```sql +kostikConsistentHash(input, n) +``` + +Alias: `yandexConsistentHash` (left for backwards compatibility sake). + +**Parameters** + +- `input`: A UInt64-type key [UInt64](/docs/en/sql-reference/data-types/int-uint.md). +- `n`: Number of buckets. [UInt16](/docs/en/sql-reference/data-types/int-uint.md). + +**Returned value** + +- A [UInt16](/docs/en/sql-reference/data-types/int-uint.md) data type hash value. + +**Implementation details** + +It is efficient only if n <= 32768. + +**Example** + +Query: + +```sql +SELECT kostikConsistentHash(16045690984833335023, 2); +``` + +```response +┌─kostikConsistentHash(16045690984833335023, 2)─┐ +│ 1 │ +└───────────────────────────────────────────────┘ +``` + ## murmurHash2_32, murmurHash2_64 Produces a [MurmurHash2](https://github.com/aappleby/smhasher) hash value. From 9419c0f7882f8a99cd15115fdbc4946d9fe0e91c Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 17:53:00 +0200 Subject: [PATCH 232/243] remove l2squared --- .../functions/distance-functions.md | 38 ------------------- 1 file changed, 38 deletions(-) diff --git a/docs/en/sql-reference/functions/distance-functions.md b/docs/en/sql-reference/functions/distance-functions.md index eb991acc94b..e20c35c6b6f 100644 --- a/docs/en/sql-reference/functions/distance-functions.md +++ b/docs/en/sql-reference/functions/distance-functions.md @@ -82,44 +82,6 @@ Result: └──────────────────┘ ``` -## L2SquaredNorm - -Calculates the square root of the sum of the squares of the vector values (the [L2Norm](#l2norm)) squared. - -**Syntax** - -```sql -L2SquaredNorm(vector) -``` - -Alias: `normL2Squared`. - -***Arguments** - -- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md). - -**Returned value** - -- L2-norm squared. - -Type: [Float](../../sql-reference/data-types/float.md). - -**Example** - -Query: - -```sql -SELECT L2SquaredNorm((1, 2)); -``` - -Result: - -```text -┌─L2SquaredNorm((1, 2))─┐ -│ 5 │ -└───────────────────────┘ -``` - ## LinfNorm Calculates the maximum of absolute values of a vector. From 98c1cc7747ba2a8afb18982666f550167a6557ee Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 9 Apr 2024 17:09:04 +0000 Subject: [PATCH 233/243] new gh runner version 2.315.0 --- tests/ci/worker/prepare-ci-ami.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 281dff5b1c2..effc224c2d5 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -9,7 +9,7 @@ set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.313.0 +export RUNNER_VERSION=2.315.0 export RUNNER_HOME=/home/ubuntu/actions-runner deb_arch() { From 077c57a4c9e56c387afffa1da2aaabe970fde305 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 20:14:50 +0200 Subject: [PATCH 234/243] Add functions and word moduli to aspell-dict --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8aa2a463c47..146b9d48607 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1935,6 +1935,7 @@ mmap mmapped modularization moduloOrZero +moduli mongodb monthName moscow @@ -2646,6 +2647,12 @@ tupleMultiplyByNumber tupleNegate tuplePlus tupleToNameValuePairs +tupleIntDiv +tupleIntDivByNumber +tupleIntDivOrZero +tupleIntDivOrZeroByNumber +tupleModulo +tupleModuloByNumber turbostat txt typename From 9a6b987b0088b6173412da57a12ec4f3d86e0234 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 20:16:59 +0200 Subject: [PATCH 235/243] Add kostikConstantHash related words to aspell-dict --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 8aa2a463c47..005a5a7c69b 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -452,6 +452,9 @@ Khanna KittenHouse Klickhouse Kolmogorov +Konstantin +kostik +kostikConsistentHash Korzeniewski Kubernetes LDAP @@ -655,6 +658,7 @@ OTLP OUTFILE ObjectId Observability +Oblakov Octonica Ok OnTime From 9b35c637c173bc64f15ae6978e66966bc2b17ab8 Mon Sep 17 00:00:00 2001 From: Blargian Date: Tue, 9 Apr 2024 20:19:47 +0200 Subject: [PATCH 236/243] Add SquaredNorm to aspell-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 8aa2a463c47..1da1373070c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -860,6 +860,7 @@ Soundex SpanKind Spearman's SquaredDistance +SquaredNorm StartTLS StartTime StartupSystemTables From de2a5f018f4891619fd74eb929b998652b615f83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 9 Apr 2024 20:40:40 +0200 Subject: [PATCH 237/243] Update CollectSets.cpp --- src/Planner/CollectSets.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index b1f2875210d..f00b1e6ab16 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; } + namespace { From b96543e57e39f021cd56d3936d93b9382a5c824d Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 10 Apr 2024 14:25:49 +0800 Subject: [PATCH 238/243] Add translateUTF8 function usage --- .../functions/string-replace-functions.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index c7bd16cad4a..ab39b064a8b 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -193,3 +193,33 @@ Result: ## translateUTF8 Like [translate](#translate) but assumes `s`, `from` and `to` are UTF-8 encoded strings. + +**Syntax** + +``` sql +translateUTF8(s, from, to) +``` + +**Parameters** + +- `s`: A string type [String](/docs/en/sql-reference/data-types/string.md). +- `from`: A string type [String](/docs/en/sql-reference/data-types/string.md). +- `to`: A string type [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- `s`: A string type [String](/docs/en/sql-reference/data-types/string.md). + +**Examples** + +Query: + +``` sql +SELECT translateUTF8('Hello, World!', 'delor', 'DELOR') AS res; +``` + +``` response +┌─res───────────┐ +│ HELLO, WORLD! │ +└───────────────┘ +``` From 01f3d57e4be00682ed68557cec2d111ad77ef860 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 10 Apr 2024 12:24:07 +0800 Subject: [PATCH 239/243] Add upper and upperUTF8 function usage --- .../functions/string-functions.md | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 573790f7ff7..9ca77eff0e0 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -260,8 +260,36 @@ Alias: `lcase` Converts the ASCII Latin symbols in a string to uppercase. +**Syntax** + +``` sql +upper(input) +``` + Alias: `ucase` +**Parameters** + +- `input`: A string type [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- A [String](/docs/en/sql-reference/data-types/string.md) data type value. + +**Examples** + +Query: + +``` sql +SELECT upper('value') as Upper; +``` + +``` response +┌─Upper─┐ +│ VALUE │ +└───────┘ +``` + ## lowerUTF8 Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. @@ -278,6 +306,34 @@ Does not detect the language, e.g. for Turkish the result might not be exactly c If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. +**Syntax** + +``` sql +upperUTF8(input) +``` + +**Parameters** + +- `input`: A string type [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- A [String](/docs/en/sql-reference/data-types/string.md) data type value. + +**Example** + +Query: + +``` sql +SELECT upperUTF8('value') as Upperutf8; +``` + +``` response +┌─Upperutf8─┐ +│ VALUE │ +└───────────┘ +``` + ## isValidUTF8 Returns 1, if the set of bytes constitutes valid UTF-8-encoded text, otherwise 0. From 7774a2a313585312527db34e168e56f5405cb157 Mon Sep 17 00:00:00 2001 From: loselarry Date: Wed, 10 Apr 2024 17:21:25 +0800 Subject: [PATCH 240/243] chore: fix some comments Signed-off-by: loselarry --- base/poco/Foundation/src/pcre_compile.c | 2 +- docs/en/operations/settings/merge-tree-settings.md | 2 +- docs/en/operations/settings/settings.md | 2 +- src/Functions/FunctionsDecimalArithmetics.h | 2 +- src/Functions/serverConstants.cpp | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/base/poco/Foundation/src/pcre_compile.c b/base/poco/Foundation/src/pcre_compile.c index 3a6fafe8d56..b5f5f9a8286 100644 --- a/base/poco/Foundation/src/pcre_compile.c +++ b/base/poco/Foundation/src/pcre_compile.c @@ -4835,7 +4835,7 @@ for (;; ptr++) If the class contains characters outside the 0-255 range, a different opcode is compiled. It may optionally have a bit map for characters < 256, - but those above are are explicitly listed afterwards. A flag byte tells + but those above are explicitly listed afterwards. A flag byte tells whether the bitmap is present, and whether this is a negated class or not. In JavaScript compatibility mode, an isolated ']' causes an error. In diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 3e411a51ff4..9327d52227f 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -287,7 +287,7 @@ Default value: 0 (seconds) ## remote_fs_execute_merges_on_single_replica_time_threshold -When this setting has a value greater than than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled. +When this setting has a value greater than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled. :::note Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e4e7be83f7d..3e38c22dd8f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5302,7 +5302,7 @@ SETTINGS(dictionary_use_async_executor=1, max_threads=8); ## storage_metadata_write_full_object_key {#storage_metadata_write_full_object_key} When set to `true` the metadata files are written with `VERSION_FULL_OBJECT_KEY` format version. With that format full object storage key names are written to the metadata files. -When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. +When set to `false` the metadata files are written with the previous format version, `VERSION_INLINE_DATA`. With that format only suffixes of object storage key names are written to the metadata files. The prefix for all of object storage key names is set in configurations files at `storage_configuration.disks` section. Default value: `false`. diff --git a/src/Functions/FunctionsDecimalArithmetics.h b/src/Functions/FunctionsDecimalArithmetics.h index 79e10d215a9..e26ad7362b3 100644 --- a/src/Functions/FunctionsDecimalArithmetics.h +++ b/src/Functions/FunctionsDecimalArithmetics.h @@ -280,7 +280,7 @@ public: /** At compile time, result is unknown. We only know the Scale (number of fractional digits) at runtime. Also nothing is known about size of whole part. - As in simple division/multiplication for decimals, we scale the result up, but is is explicit here and no downscale is performed. + As in simple division/multiplication for decimals, we scale the result up, but it is explicit here and no downscale is performed. It guarantees that result will have given scale and it can also be MANUALLY converted to other decimal types later. **/ if (scale > DecimalUtils::max_precision) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index fd8fb22455b..e7e423058f1 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -32,7 +32,7 @@ namespace #endif - /// Get the host name. Is is constant on single server, but is not constant in distributed queries. + /// Get the host name. It is constant on single server, but is not constant in distributed queries. class FunctionHostName : public FunctionConstantBase { public: From 1f48b97d23ad8dd03cbad66280db1def4c489b51 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 10 Apr 2024 09:56:54 +0000 Subject: [PATCH 241/243] Fix clang-tidy build --- src/Common/examples/parallel_aggregation.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/examples/parallel_aggregation.cpp b/src/Common/examples/parallel_aggregation.cpp index 20f5f1c5224..7094690a3a8 100644 --- a/src/Common/examples/parallel_aggregation.cpp +++ b/src/Common/examples/parallel_aggregation.cpp @@ -205,7 +205,7 @@ static void aggregate4(Map & local_map, MapTwoLevel & global_map, Mutex * mutexe else { size_t hash_value = global_map.hash(*it); - size_t bucket = global_map.getBucketFromHash(hash_value); + size_t bucket = MapTwoLevel::getBucketFromHash(hash_value); if (mutexes[bucket].try_lock()) { From 927c5ca1fcefd3091c9a9967da2f1e1bb1affc74 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 10 Apr 2024 18:09:02 +0800 Subject: [PATCH 242/243] Update example --- .../sql-reference/functions/string-replace-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index ab39b064a8b..60fe286de25 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -215,11 +215,11 @@ translateUTF8(s, from, to) Query: ``` sql -SELECT translateUTF8('Hello, World!', 'delor', 'DELOR') AS res; +SELECT translateUTF8('Münchener Straße', 'üß', 'us') AS res; ``` ``` response -┌─res───────────┐ -│ HELLO, WORLD! │ -└───────────────┘ +┌─res──────────────┐ +│ Munchener Strase │ +└──────────────────┘ ``` From 06bbf97cc057fc872d0b0749b0d349d21bbb0f72 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 10 Apr 2024 18:12:29 +0800 Subject: [PATCH 243/243] Update example --- docs/en/sql-reference/functions/string-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9ca77eff0e0..d4df3e0479a 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -325,12 +325,12 @@ upperUTF8(input) Query: ``` sql -SELECT upperUTF8('value') as Upperutf8; +SELECT upperUTF8('München') as Upperutf8; ``` ``` response ┌─Upperutf8─┐ -│ VALUE │ +│ MÜNCHEN │ └───────────┘ ```