From f9fc95563899634f7be163bb562c89f95266859d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Thu, 6 Jun 2024 17:21:58 +0300 Subject: [PATCH 001/433] ATTACH AS [NOT] REPLICATED --- src/Interpreters/InterpreterCreateQuery.cpp | 90 +++++++++++++++++++++ src/Interpreters/InterpreterCreateQuery.h | 2 + src/Parsers/ASTCreateQuery.cpp | 8 ++ src/Parsers/ASTCreateQuery.h | 2 + src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserCreateQuery.cpp | 25 +++++- 6 files changed, 124 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66936dc25d7..8397a3a0631 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1128,6 +1128,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) FunctionNameNormalizer::visit(query.get()); auto create_query = query->as(); + /// Set replicated or not replicated MergeTree engine in metadata and query + if (create.attach_as_replicated.has_value()) + convertMergeTreeTableIfPossible(create_query, database, create.attach_as_replicated.value()); + if (!create.is_dictionary && create_query.is_dictionary) throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot ATTACH TABLE {}.{}, it is a Dictionary", @@ -1780,6 +1784,11 @@ BlockIO InterpreterCreateQuery::execute() bool is_create_database = create.database && !create.table; if (!create.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { + if (create.attach_as_replicated.has_value()) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "ATTACH AS [NOT] REPLICATED is not supported for ON CLUSTER queries"); + auto on_cluster_version = getContext()->getSettingsRef().distributed_ddl_entry_format_version; if (is_create_database || on_cluster_version < DDLLogEntry::NORMALIZE_CREATE_ON_INITIATOR_VERSION) return executeQueryOnCluster(create); @@ -1942,6 +1951,87 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } +static void setMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr context, bool replicated) +{ + auto * storage = create_query.storage; + auto args = std::make_shared(); + auto engine = std::make_shared(); + String engine_name; + + if (replicated) + { + const auto & server_settings = context->getServerSettings(); + String replica_path = server_settings.default_replica_path; + String replica_name = server_settings.default_replica_name; + + args->children.push_back(std::make_shared(replica_path)); + args->children.push_back(std::make_shared(replica_name)); + + /// Add old engine's arguments + if (storage->engine->arguments) + { + for (size_t i = 0; i < storage->engine->arguments->children.size(); ++i) + args->children.push_back(storage->engine->arguments->children[i]->clone()); + } + + engine_name = "Replicated" + storage->engine->name; + } + else + { + /// Add old engine's arguments without first two + if (storage->engine->arguments) + { + for (size_t i = 2; i < storage->engine->arguments->children.size(); ++i) + args->children.push_back(storage->engine->arguments->children[i]->clone()); + } + + engine_name = storage->engine->name.substr(strlen("Replicated")); + } + + /// Set new engine for the old query + engine->name = engine_name; + engine->arguments = args; + create_query.storage->set(create_query.storage->engine, engine->clone()); +} + +void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & create, DatabasePtr database, bool to_replicated) +{ + /// Check engine can be changed + if (database->getEngineName() != "Atomic") + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Table engine conversion to replicated is supported only for Atomic databases"); + + if (!create.storage || !create.storage->engine || create.storage->engine->name.find("MergeTree") == std::string::npos) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Table engine conversion is supported only for MergeTree family engines"); + + String engine_name = create.storage->engine->name; + if (engine_name.starts_with("Replicated")) + { + if (to_replicated) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Table is already replicated"); + } + else if (!to_replicated) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Table is already not replicated"); + + /// Set new engine + setMergeTreeEngine(create, getContext(), to_replicated); + + /// Save new metadata + String table_metadata_path = database->getObjectMetadataPath(create.getTable()); + String table_metadata_tmp_path = table_metadata_path + ".tmp"; + String statement = DB::getObjectDefinitionFromCreateQuery(create.clone()); + { + WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(statement, out); + out.next(); + if (getContext()->getSettingsRef().fsync_metadata) + out.sync(); + out.close(); + } + fs::rename(table_metadata_tmp_path, table_metadata_path); +} + void registerInterpreterCreateQuery(InterpreterFactory & factory) { auto create_fn = [] (const InterpreterFactory::Arguments & args) diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index be4a10eaf1d..cad41693d5f 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -116,6 +116,8 @@ private: BlockIO executeQueryOnCluster(ASTCreateQuery & create); + void convertMergeTreeTableIfPossible(ASTCreateQuery & create, DatabasePtr database, bool to_replicated); + ASTPtr query_ptr; /// Skip safety threshold when loading tables. diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 3e5c6a9d86e..91162a37e9b 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -351,6 +351,14 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << quoteString(*attach_from_path); + if (attach_as_replicated.has_value()) + { + if (attach_as_replicated.value()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS REPLICATED" << (settings.hilite ? hilite_none : ""); + else + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS NOT REPLICATED" << (settings.hilite ? hilite_none : ""); + } + formatOnCluster(settings); } else diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 6fbf045915b..a252e64d4bb 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -132,6 +132,8 @@ public: std::optional attach_from_path = std::nullopt; + std::optional attach_as_replicated = std::nullopt; + bool replace_table{false}; bool create_or_replace{false}; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index f0cbe42da80..7f50a819121 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -400,6 +400,7 @@ namespace DB MR_MACROS(RENAME, "RENAME") \ MR_MACROS(REPLACE_PARTITION, "REPLACE PARTITION") \ MR_MACROS(REPLACE, "REPLACE") \ + MR_MACROS(REPLICATED, "REPLICATED") \ MR_MACROS(RESET_SETTING, "RESET SETTING") \ MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \ MR_MACROS(RESTORE, "RESTORE") \ diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 014dc7bd3bf..eae87a8a6b9 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -680,6 +680,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_from(Keyword::FROM); ParserKeyword s_on(Keyword::ON); + ParserKeyword s_as(Keyword::AS); + ParserKeyword s_not(Keyword::NOT); + ParserKeyword s_replicated(Keyword::REPLICATED); ParserToken s_dot(TokenType::Dot); ParserToken s_comma(TokenType::Comma); ParserToken s_lparen(TokenType::OpeningRoundBracket); @@ -734,11 +737,23 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!table_name_p.parse(pos, table, expected)) return false; - if (attach && s_from.ignore(pos, expected)) + std::optional attach_as_replicated = std::nullopt; + if (attach) { - ParserStringLiteral from_path_p; - if (!from_path_p.parse(pos, from_path, expected)) - return false; + if (s_from.ignore(pos, expected)) + { + ParserStringLiteral from_path_p; + if (!from_path_p.parse(pos, from_path, expected)) + return false; + } else if (s_as.ignore(pos, expected)) + { + if (s_not.ignore(pos, expected)) + attach_as_replicated = false; + if (!s_replicated.ignore(pos, expected)) + return false; + if (!attach_as_replicated.has_value()) + attach_as_replicated = true; + } } if (s_on.ignore(pos, expected)) @@ -765,6 +780,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->uuid = table_id->uuid; query->has_uuid = table_id->uuid != UUIDHelpers::Nil; + query->attach_as_replicated = attach_as_replicated; + if (query->database) query->children.push_back(query->database); if (query->table) From cb32322c67eb54de998cdb9a9bb321f2674823d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 17 Jun 2024 20:47:42 +0300 Subject: [PATCH 002/433] Reuse convert methods from DatabaseOrdinary --- src/Databases/DatabaseOrdinary.cpp | 74 +++++++++++++-------- src/Databases/DatabaseOrdinary.h | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 46 +------------ 3 files changed, 51 insertions(+), 71 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 10a8e06e8f0..5a5c458fbbf 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -68,46 +68,65 @@ void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLev throw Exception(ErrorCodes::LOGICAL_ERROR, "Not implemented"); } -static void setReplicatedEngine(ASTCreateQuery * create_query, ContextPtr context) +static void checkReplicaPathExists(ASTCreateQuery & create_query, ContextPtr local_context) { - auto * storage = create_query->storage; - - /// Get replicated engine - const auto & server_settings = context->getServerSettings(); - String replica_path = server_settings.default_replica_path; - String replica_name = server_settings.default_replica_name; - - /// Check that replica path doesn't exist Macros::MacroExpansionInfo info; - StorageID table_id = StorageID(create_query->getDatabase(), create_query->getTable(), create_query->uuid); + StorageID table_id = StorageID(create_query.getDatabase(), create_query.getTable(), create_query.uuid); info.table_id = table_id; info.expand_special_macros_only = false; - String zookeeper_path = context->getMacros()->expand(replica_path, info); - if (context->getZooKeeper()->exists(zookeeper_path)) + const auto & server_settings = local_context->getServerSettings(); + String replica_path = server_settings.default_replica_path; + String zookeeper_path = local_context->getMacros()->expand(replica_path, info); + if (local_context->getZooKeeper()->exists(zookeeper_path)) throw Exception( ErrorCodes::UNEXPECTED_NODE_IN_ZOOKEEPER, "Found existing ZooKeeper path {} while trying to convert table {} to replicated. Table will not be converted.", zookeeper_path, backQuote(table_id.getFullTableName()) ); +} +void DatabaseOrdinary::setReplicatedMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr local_context, bool replicated) +{ + auto * storage = create_query.storage; auto args = std::make_shared(); - args->children.push_back(std::make_shared(replica_path)); - args->children.push_back(std::make_shared(replica_name)); + auto engine = std::make_shared(); + String engine_name; - /// Add old engine's arguments - if (storage->engine->arguments) + if (replicated) { - for (size_t i = 0; i < storage->engine->arguments->children.size(); ++i) - args->children.push_back(storage->engine->arguments->children[i]->clone()); + const auto & server_settings = local_context->getServerSettings(); + String replica_path = server_settings.default_replica_path; + String replica_name = server_settings.default_replica_name; + + args->children.push_back(std::make_shared(replica_path)); + args->children.push_back(std::make_shared(replica_name)); + + /// Add old engine's arguments + if (storage->engine->arguments) + { + for (size_t i = 0; i < storage->engine->arguments->children.size(); ++i) + args->children.push_back(storage->engine->arguments->children[i]->clone()); + } + + engine_name = "Replicated" + storage->engine->name; + } + else + { + /// Add old engine's arguments without first two + if (storage->engine->arguments) + { + for (size_t i = 2; i < storage->engine->arguments->children.size(); ++i) + args->children.push_back(storage->engine->arguments->children[i]->clone()); + } + + engine_name = storage->engine->name.substr(strlen("Replicated")); } - auto engine = std::make_shared(); - engine->name = "Replicated" + storage->engine->name; - engine->arguments = args; - /// Set new engine for the old query - create_query->storage->set(create_query->storage->engine, engine->clone()); + engine->name = engine_name; + engine->arguments = args; + create_query.storage->set(create_query.storage->engine, engine->clone()); } String DatabaseOrdinary::getConvertToReplicatedFlagPath(const String & name, const StoragePolicyPtr storage_policy, bool tableStarted) @@ -135,15 +154,15 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu fs::path file_path(file_name); fs::path full_path = path / file_path; - auto * create_query = ast->as(); + auto & create_query = ast->as(); - if (!create_query->storage || !create_query->storage->engine->name.ends_with("MergeTree") || create_query->storage->engine->name.starts_with("Replicated") || create_query->storage->engine->name.starts_with("Shared")) + if (!create_query.storage || !create_query.storage->engine->name.ends_with("MergeTree") || create_query.storage->engine->name.starts_with("Replicated") || create_query.storage->engine->name.starts_with("Shared")) return; /// Get table's storage policy MergeTreeSettings default_settings = getContext()->getMergeTreeSettings(); auto policy = getContext()->getStoragePolicy(default_settings.storage_policy); - if (auto * query_settings = create_query->storage->settings) + if (auto * query_settings = create_query.storage->settings) if (Field * policy_setting = query_settings->changes.tryGet("storage_policy")) policy = getContext()->getStoragePolicy(policy_setting->safeGet()); @@ -158,7 +177,8 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu LOG_INFO(log, "Found {} flag for table {}. Will try to change it's engine in metadata to replicated.", CONVERT_TO_REPLICATED_FLAG_NAME, backQuote(qualified_name.getFullName())); - setReplicatedEngine(create_query, getContext()); + checkReplicaPathExists(create_query, getContext()); + setReplicatedMergeTreeEngine(create_query, getContext(), true); /// Write changes to metadata String table_metadata_path = full_path; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index ef00ac8fdfa..0e13e1b775a 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -66,6 +66,8 @@ public: Strings getNamesOfPermanentlyDetachedTables() const override { return permanently_detached_tables; } + static void setReplicatedMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr context, bool replicated); + protected: virtual void commitAlterTable( const StorageID & table_id, diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8397a3a0631..52da39ee75c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #include #include #include @@ -1951,49 +1952,6 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } -static void setMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr context, bool replicated) -{ - auto * storage = create_query.storage; - auto args = std::make_shared(); - auto engine = std::make_shared(); - String engine_name; - - if (replicated) - { - const auto & server_settings = context->getServerSettings(); - String replica_path = server_settings.default_replica_path; - String replica_name = server_settings.default_replica_name; - - args->children.push_back(std::make_shared(replica_path)); - args->children.push_back(std::make_shared(replica_name)); - - /// Add old engine's arguments - if (storage->engine->arguments) - { - for (size_t i = 0; i < storage->engine->arguments->children.size(); ++i) - args->children.push_back(storage->engine->arguments->children[i]->clone()); - } - - engine_name = "Replicated" + storage->engine->name; - } - else - { - /// Add old engine's arguments without first two - if (storage->engine->arguments) - { - for (size_t i = 2; i < storage->engine->arguments->children.size(); ++i) - args->children.push_back(storage->engine->arguments->children[i]->clone()); - } - - engine_name = storage->engine->name.substr(strlen("Replicated")); - } - - /// Set new engine for the old query - engine->name = engine_name; - engine->arguments = args; - create_query.storage->set(create_query.storage->engine, engine->clone()); -} - void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & create, DatabasePtr database, bool to_replicated) { /// Check engine can be changed @@ -2015,7 +1973,7 @@ void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & cr throw Exception(ErrorCodes::INCORRECT_QUERY, "Table is already not replicated"); /// Set new engine - setMergeTreeEngine(create, getContext(), to_replicated); + DatabaseOrdinary::setReplicatedMergeTreeEngine(create, getContext(), to_replicated); /// Save new metadata String table_metadata_path = database->getObjectMetadataPath(create.getTable()); From 8d868101920bd234b6a1a4e97b5892c8dc53f55a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 17 Jun 2024 20:58:39 +0300 Subject: [PATCH 003/433] Basic tests --- ...3167_attach_as_replicated_errors.reference | 5 + .../03167_attach_as_replicated_errors.sh | 39 ++++++++ ..._as_replicated_materialized_view.reference | 7 ++ ...attach_as_replicated_materialized_view.sql | 41 ++++++++ .../03169_attach_as_replicated.reference | 14 +++ .../0_stateless/03169_attach_as_replicated.sh | 97 +++++++++++++++++++ 6 files changed, 203 insertions(+) create mode 100644 tests/queries/0_stateless/03167_attach_as_replicated_errors.reference create mode 100755 tests/queries/0_stateless/03167_attach_as_replicated_errors.sh create mode 100644 tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.reference create mode 100644 tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql create mode 100644 tests/queries/0_stateless/03169_attach_as_replicated.reference create mode 100755 tests/queries/0_stateless/03169_attach_as_replicated.sh diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference new file mode 100644 index 00000000000..627e1097cda --- /dev/null +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference @@ -0,0 +1,5 @@ +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh new file mode 100755 index 00000000000..fc749773eb0 --- /dev/null +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +# Tags: zookeeper, no-replicated-database, no-ordinary-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +ORDINARY_DB="ordinary_$CLICKHOUSE_DATABASE" +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " + CREATE DATABASE $ORDINARY_DB ENGINE = Ordinary; + + CREATE TABLE mt ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A; + DETACH TABLE mt; + CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/$ORDINARY_DB/rmt/s1', 'r1') PARTITION BY toYYYYMM(D) ORDER BY A; + DETACH TABLE rmt; + CREATE TABLE log ( A Int64, D Date, S String ) ENGINE Log; + DETACH TABLE log; + CREATE TABLE $ORDINARY_DB.mt ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A; + DETACH TABLE $ORDINARY_DB.mt; +" + +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS NOT REPLICATED" 2>&1)" \ + | grep -c 'Table is already not replicated' +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE rmt AS REPLICATED" 2>&1)" \ + | grep -c 'Table is already replicated' +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE log AS REPLICATED" 2>&1)" \ + | grep -c 'Table engine conversion is supported only for MergeTree family engines' +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS REPLICATED ON CLUSTER test_shard_localhost" 2>&1)" \ + | grep -c 'ATTACH AS \[NOT\] REPLICATED is not supported for ON CLUSTER queries' +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE $ORDINARY_DB.mt AS REPLICATED" 2>&1)" \ + | grep -c 'Table engine conversion to replicated is supported only for Atomic databases' + + +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " + ATTACH TABLE $ORDINARY_DB.mt; + DROP TABLE $ORDINARY_DB.mt; + DROP DATABASE $ORDINARY_DB; +" diff --git a/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.reference b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.reference new file mode 100644 index 00000000000..59fcb9dba0b --- /dev/null +++ b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.reference @@ -0,0 +1,7 @@ +6 +3 +hourly_data ReplicatedMergeTree +monthly_aggregated_data ReplicatedAggregatingMergeTree +monthly_aggregated_data_mv MaterializedView +12 +6 diff --git a/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql new file mode 100644 index 00000000000..7acd63ff714 --- /dev/null +++ b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql @@ -0,0 +1,41 @@ +CREATE TABLE hourly_data + (domain_name String, event_time DateTime, count_views UInt64) + ENGINE = MergeTree ORDER BY (domain_name, event_time); + +CREATE TABLE monthly_aggregated_data + (domain_name String, month Date, sumCountViews AggregateFunction(sum, UInt64)) + ENGINE = AggregatingMergeTree ORDER BY (domain_name, month); + + +CREATE MATERIALIZED VIEW monthly_aggregated_data_mv + TO monthly_aggregated_data + AS + SELECT + toDate(toStartOfMonth(event_time)) AS month, + domain_name, + sumState(count_views) AS sumCountViews + FROM hourly_data + GROUP BY + domain_name, + month; + +INSERT INTO hourly_data (domain_name, event_time, count_views) + VALUES ('clickhouse.com', '2019-01-01 10:00:00', 1), ('clickhouse.com', '2019-02-02 00:00:00', 2), ('clickhouse.com', '2019-02-01 00:00:00', 3); + +SELECT sumMerge(sumCountViews) as sumCountViews FROM monthly_aggregated_data_mv; +SELECT count() FROM hourly_data; + +DETACH TABLE hourly_data; +DETACH TABLE monthly_aggregated_data; +ATTACH TABLE hourly_data AS REPLICATED; +ATTACH TABLE monthly_aggregated_data AS REPLICATED; +SYSTEM RESTORE REPLICA hourly_data; +SYSTEM RESTORE REPLICA monthly_aggregated_data; + +SELECT name, engine FROM system.tables WHERE database=currentDatabase(); + +INSERT INTO hourly_data (domain_name, event_time, count_views) + VALUES ('clickhouse.com', '2019-01-01 10:00:00', 1), ('clickhouse.com', '2019-02-02 00:00:00', 2), ('clickhouse.com', '2019-02-01 00:00:00', 3); + +SELECT sumMerge(sumCountViews) as sumCountViews FROM monthly_aggregated_data_mv; +SELECT count() FROM hourly_data; diff --git a/tests/queries/0_stateless/03169_attach_as_replicated.reference b/tests/queries/0_stateless/03169_attach_as_replicated.reference new file mode 100644 index 00000000000..2db4b1c77cd --- /dev/null +++ b/tests/queries/0_stateless/03169_attach_as_replicated.reference @@ -0,0 +1,14 @@ +1 +1 +1 +1 + +1 +1 +1 +1 + +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03169_attach_as_replicated.sh b/tests/queries/0_stateless/03169_attach_as_replicated.sh new file mode 100755 index 00000000000..7a7bc00cfe7 --- /dev/null +++ b/tests/queries/0_stateless/03169_attach_as_replicated.sh @@ -0,0 +1,97 @@ +#!/usr/bin/env bash +# Tags: zookeeper, no-replicated-database, no-ordinary-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Create tables +${CLICKHOUSE_CLIENT} -n -q " + CREATE TABLE mt ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A; + INSERT INTO mt SELECT number, today(), '' FROM numbers(1e6); + INSERT INTO mt SELECT number, today()-60, '' FROM numbers(1e5); + + CREATE TABLE replacing ( A Int64, D Date, S String ) ENGINE ReplacingMergeTree() PARTITION BY toYYYYMM(D) ORDER BY A; + INSERT INTO replacing SELECT number, today(), '' FROM numbers(1e6); + INSERT INTO replacing SELECT number, today()-60, '' FROM numbers(1e5); + + CREATE TABLE replacing_ver ( A Int64, D Date, S String ) ENGINE = ReplacingMergeTree(D) PARTITION BY toYYYYMM(D) ORDER BY A; + CREATE TABLE collapsing_ver ( ID UInt64, Sign Int8, Version UInt8 ) ENGINE = VersionedCollapsingMergeTree(Sign, Version) ORDER BY ID; +" + +# Convert tables +$CLICKHOUSE_CLIENT -n -q " + DETACH TABLE mt; + DETACH TABLE replacing; + DETACH TABLE replacing_ver; + DETACH TABLE collapsing_ver; + + ATTACH TABLE mt AS REPLICATED; + ATTACH TABLE replacing AS REPLICATED; + ATTACH TABLE replacing_ver AS REPLICATED; + ATTACH TABLE collapsing_ver AS REPLICATED; + + SYSTEM RESTORE REPLICA mt; + SYSTEM RESTORE REPLICA replacing; + SYSTEM RESTORE REPLICA replacing_ver; + SYSTEM RESTORE REPLICA collapsing_ver; +" + +# Check tables +${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'mt'" \ +| grep -c "ReplicatedMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\') PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing'" \ +| grep -c "ReplicatedReplacingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\') PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing_ver'" \ +| grep -c "ReplicatedReplacingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', D) PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'collapsing_ver'" \ +| grep -c "ReplicatedVersionedCollapsingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', Sign, Version) ORDER BY ID" +echo + +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='mt'" \ +| grep -c "0" +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='replacing'" \ +| grep -c "0" +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='replacing_ver'" \ +| grep -c "0" +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='collapsing_ver'" \ +| grep -c "0" +echo + +# Convert tables back +# Get zk paths +MT_ZK_PATH=$($CLICKHOUSE_CLIENT --query="SELECT zookeeper_path FROM system.replicas WHERE database=currentDatabase() AND table='mt'") +REPLACING_ZK_PATH=$($CLICKHOUSE_CLIENT --query="SELECT zookeeper_path FROM system.replicas WHERE database=currentDatabase() AND table='replacing'") +REPLACING_VER_ZK_PATH=$($CLICKHOUSE_CLIENT --query="SELECT zookeeper_path FROM system.replicas WHERE database=currentDatabase() AND table='replacing_ver'") +COLLAPSING_VER_ZK_PATH=$($CLICKHOUSE_CLIENT --query="SELECT zookeeper_path FROM system.replicas WHERE database=currentDatabase() AND table='collapsing_ver'") + +# Restored replica has no table_shared_id node in ZK +# DROP REPLICA will log warning while deleting this node +$CLICKHOUSE_CLIENT -n -q " + DETACH TABLE mt; + DETACH TABLE replacing; + DETACH TABLE replacing_ver; + DETACH TABLE collapsing_ver; + + ATTACH TABLE mt AS NOT REPLICATED; + ATTACH TABLE replacing AS NOT REPLICATED; + ATTACH TABLE replacing_ver AS NOT REPLICATED; + ATTACH TABLE collapsing_ver AS NOT REPLICATED; + + SET send_logs_level = 'fatal'; + SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$MT_ZK_PATH'; + SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$REPLACING_ZK_PATH'; + SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$REPLACING_VER_ZK_PATH'; + SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$COLLAPSING_VER_ZK_PATH'; + SET send_logs_level = 'trace'; +" + +# Check tables +${CLICKHOUSE_CLIENT} --echo --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'mt'" \ +| grep -c "MergeTree PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --echo --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing'" \ +| grep -c "ReplacingMergeTree PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --echo --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing_ver'" \ +| grep -c "ReplacingMergeTree(D) PARTITION BY toYYYYMM(D) ORDER BY A" +${CLICKHOUSE_CLIENT} --echo --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'collapsing_ver'" \ +| grep -c "VersionedCollapsingMergeTree(Sign, Version) ORDER BY ID" From 2f15a2c03eec3ec0bcb782f10578a78527af494a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 18 Jun 2024 02:01:07 +0300 Subject: [PATCH 004/433] Docs --- .../mergetree-family/replication.md | 8 +++- docs/en/sql-reference/statements/attach.md | 42 +++++++++++++++++++ 2 files changed, 48 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 65a5f58b166..385a26793e5 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -304,6 +304,8 @@ We use the term `MergeTree` to refer to all table engines in the `MergeTree fami If you had a `MergeTree` table that was manually replicated, you can convert it to a replicated table. You might need to do this if you have already collected a large amount of data in a `MergeTree` table and now you want to enable replication. +[ATTACH TABLE ... AS REPLICATED](/docs/en/sql-reference/statements/attach.md#attach-mergetree-table-as-replicatedmergetree) statement allows to attach detached `MergeTree` table as `ReplicatedMergeTree`. + `MergeTree` table can be automatically converted on server restart if `convert_to_replicated` flag is set at the table's data directory (`/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/` for `Atomic` database). Create empty `convert_to_replicated` file and the table will be loaded as replicated on next server restart. @@ -320,7 +322,7 @@ To create a converted table on other replicas, you will need to explicitly speci SELECT zookeeper_path FROM system.replicas WHERE table = 'table_name'; ``` -There is also a manual way to do this without server restart. +There is also a manual way to do this. If the data differs on various replicas, first sync it, or delete this data on all the replicas except one. @@ -330,7 +332,9 @@ Then run `ALTER TABLE ATTACH PARTITION` on one of the replicas to add these data ## Converting from ReplicatedMergeTree to MergeTree {#converting-from-replicatedmergetree-to-mergetree} -Create a MergeTree table with a different name. Move all the data from the directory with the `ReplicatedMergeTree` table data to the new table’s data directory. Then delete the `ReplicatedMergeTree` table and restart the server. +Use [ATTACH TABLE ... AS NOT REPLICATED](/docs/en/sql-reference/statements/attach.md#attach-mergetree-table-as-replicatedmergetree) statement to attach detached `ReplicatedMergeTree` table as `MergeTree` on a single server. + +Another way to do this involves server restart. Create a MergeTree table with a different name. Move all the data from the directory with the `ReplicatedMergeTree` table data to the new table’s data directory. Then delete the `ReplicatedMergeTree` table and restart the server. If you want to get rid of a `ReplicatedMergeTree` table without launching the server: diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index be322212f34..368f479005b 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -70,6 +70,48 @@ It is supported by the [Atomic](../../engines/database-engines/atomic.md) databa ATTACH TABLE name UUID '' (col1 Type1, ...) ``` +## Attach MergeTree table as ReplicatedMergeTree + +Allows to attach non-replicated MergeTree table as ReplicatedMergeTree. ReplicatedMergeTree table will be created with values of `default_replica_path` and `default_replica_name` settings. It is also possible to attach a replicated table as a regular MergeTree. + +Note that table's data in ZooKeeper is not affected in this query. This means you have to add metadata in ZooKeeper using `SYSTEM RESTORE REPLICA` or clear it with `SYSTEM DROP REPLICA ... FROM ZKPATH ...` after attach. + +If you are trying to add a replica to an existing ReplicatedMergeTree table, keep in mind that all the data in converted MergeTree table will be replicated as well. + +**Syntax** + +```sql +ATTACH TABLE [db.]name AS [NOT] REPLICATED +``` + +**Сonvert table to replicated** + +```sql +DETACH TABLE test; +ATTACH TABLE test AS REPLICATED; +SYSTEM RESTORE REPLICA test; +``` + +**Сonvert table to not replicated** + +Get ZooKeeper path and replica name for table: + +```sql +SELECT replica_name, zookeeper_path FROM system.replicas WHERE table='test'; +``` +Result: +``` +┌─replica_name─┬─zookeeper_path─────────────────────────────────────────────┐ +│ r1 │ /clickhouse/tables/401e6a1f-9bf2-41a3-a900-abb7e94dff98/s1 │ +└──────────────┴────────────────────────────────────────────────────────────┘ +``` +Attach table as not replicated and delete replica's data from ZooKeeper: +```sql +DETACH TABLE test; +ATTACH TABLE test AS NOT REPLICATED; +SYSTEM DROP REPLICA 'r1' FROM ZKPATH '/clickhouse/tables/401e6a1f-9bf2-41a3-a900-abb7e94dff98/s1'; +``` + ## Attach Existing Dictionary Attaches a previously detached dictionary. From 75b88042ea384d54070086076d0272f78b5052a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 18 Jun 2024 18:42:21 +0300 Subject: [PATCH 005/433] Test active replica exists --- .../03167_attach_as_replicated_errors.reference | 1 + .../0_stateless/03167_attach_as_replicated_errors.sh | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference index 627e1097cda..a9e2f17562a 100644 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference @@ -3,3 +3,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index fc749773eb0..5a1f8a2283e 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -37,3 +37,11 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " DROP TABLE $ORDINARY_DB.mt; DROP DATABASE $ORDINARY_DB; " + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_1 (id UInt32) ENGINE=MergeTree() ORDER BY id;" +UUID=$($CLICKHOUSE_CLIENT --query="SELECT uuid FROM system.tables WHERE database=currentDatabase() AND table='already_exists_1';") +${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_2 (id UInt32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$UUID/s1', 'r1') ORDER BY id;" +${CLICKHOUSE_CLIENT} -q "DETACH TABLE already_exists_1;" +echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="ATTACH TABLE already_exists_1 AS REPLICATED" 2>&1)" \ + | grep -c 'There already is an active replica with this replica path' +${CLICKHOUSE_CLIENT} -q "ATTACH TABLE already_exists_1 AS NOT REPLICATED;" From 52596a8e6b74d5239680c881a762f371eaf698df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 18 Jun 2024 20:05:06 +0300 Subject: [PATCH 006/433] Style fix --- docs/en/sql-reference/statements/attach.md | 2 +- .../03167_attach_as_replicated_errors.sh | 7 ++++--- .../0_stateless/03169_attach_as_replicated.sh | 13 +++++++++---- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 368f479005b..a21c03625ac 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -92,7 +92,7 @@ ATTACH TABLE test AS REPLICATED; SYSTEM RESTORE REPLICA test; ``` -**Сonvert table to not replicated** +**Convert table to not replicated** Get ZooKeeper path and replica name for table: diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index 5a1f8a2283e..225853763ff 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " CREATE TABLE mt ( A Int64, D Date, S String ) ENGINE MergeTree() PARTITION BY toYYYYMM(D) ORDER BY A; DETACH TABLE mt; - CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/$ORDINARY_DB/rmt/s1', 'r1') PARTITION BY toYYYYMM(D) ORDER BY A; + CREATE TABLE rmt ( A Int64, D Date, S String ) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/rmt/s1', 'r1') PARTITION BY toYYYYMM(D) ORDER BY A; DETACH TABLE rmt; CREATE TABLE log ( A Int64, D Date, S String ) ENGINE Log; DETACH TABLE log; @@ -38,9 +38,10 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " DROP DATABASE $ORDINARY_DB; " -${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_1 (id UInt32) ENGINE=MergeTree() ORDER BY id;" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_1 (id UInt32) ENGINE=MergeTree() ORDER BY id;" UUID=$($CLICKHOUSE_CLIENT --query="SELECT uuid FROM system.tables WHERE database=currentDatabase() AND table='already_exists_1';") -${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_2 (id UInt32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$UUID/s1', 'r1') ORDER BY id;" +ARGS_1="('/clickhouse/tables/$UUID/s1', 'r1')" # Suppress style check for zk path +${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_2 (id UInt32) ENGINE=ReplicatedMergeTree$ARGS_1 ORDER BY id;" ${CLICKHOUSE_CLIENT} -q "DETACH TABLE already_exists_1;" echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="ATTACH TABLE already_exists_1 AS REPLICATED" 2>&1)" \ | grep -c 'There already is an active replica with this replica path' diff --git a/tests/queries/0_stateless/03169_attach_as_replicated.sh b/tests/queries/0_stateless/03169_attach_as_replicated.sh index 7a7bc00cfe7..c1a97ff644e 100755 --- a/tests/queries/0_stateless/03169_attach_as_replicated.sh +++ b/tests/queries/0_stateless/03169_attach_as_replicated.sh @@ -38,14 +38,19 @@ $CLICKHOUSE_CLIENT -n -q " " # Check tables +# ARGS_i is used to suppress style check for not using {database} in ZK path +# ATTACH AS REPLICATED uses only default path, so there is nothing else we can do +ARGS_1="(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\')" ${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'mt'" \ -| grep -c "ReplicatedMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\') PARTITION BY toYYYYMM(D) ORDER BY A" +| grep -c "ReplicatedMergeTree$ARGS_1 PARTITION BY toYYYYMM(D) ORDER BY A" ${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing'" \ -| grep -c "ReplicatedReplacingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\') PARTITION BY toYYYYMM(D) ORDER BY A" +| grep -c "ReplicatedReplacingMergeTree$ARGS_1 PARTITION BY toYYYYMM(D) ORDER BY A" +ARGS_2="(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', D)" ${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'replacing_ver'" \ -| grep -c "ReplicatedReplacingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', D) PARTITION BY toYYYYMM(D) ORDER BY A" +| grep -c "ReplicatedReplacingMergeTree$ARGS_2 PARTITION BY toYYYYMM(D) ORDER BY A" +ARGS_3="(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', Sign, Version)" ${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE database=currentDatabase() AND name = 'collapsing_ver'" \ -| grep -c "ReplicatedVersionedCollapsingMergeTree(\\\\'/clickhouse/tables/{uuid}/{shard}\\\\', \\\\'{replica}\\\\', Sign, Version) ORDER BY ID" +| grep -c "ReplicatedVersionedCollapsingMergeTree$ARGS_3 ORDER BY ID" echo $CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='mt'" \ From 58e34a53cf2ddc7f976caf397f37bc799326c742 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Tue, 18 Jun 2024 20:16:01 +0300 Subject: [PATCH 007/433] Style fix 2 --- docs/en/sql-reference/statements/attach.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index a21c03625ac..ca90165f53b 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -84,7 +84,7 @@ If you are trying to add a replica to an existing ReplicatedMergeTree table, kee ATTACH TABLE [db.]name AS [NOT] REPLICATED ``` -**Сonvert table to replicated** +**Convert table to replicated** ```sql DETACH TABLE test; From 5eb2255417dab983779c9d209500e71d511efac4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Wed, 19 Jun 2024 02:17:00 +0300 Subject: [PATCH 008/433] Flaky tests fix --- .../03168_attach_as_replicated_materialized_view.sql | 1 + tests/queries/0_stateless/03169_attach_as_replicated.sh | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql index 7acd63ff714..7aeb3dedff3 100644 --- a/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql +++ b/tests/queries/0_stateless/03168_attach_as_replicated_materialized_view.sql @@ -1,3 +1,4 @@ +-- Tags: zookeeper, no-replicated-database, no-ordinary-database CREATE TABLE hourly_data (domain_name String, event_time DateTime, count_views UInt64) ENGINE = MergeTree ORDER BY (domain_name, event_time); diff --git a/tests/queries/0_stateless/03169_attach_as_replicated.sh b/tests/queries/0_stateless/03169_attach_as_replicated.sh index c1a97ff644e..878115e6a27 100755 --- a/tests/queries/0_stateless/03169_attach_as_replicated.sh +++ b/tests/queries/0_stateless/03169_attach_as_replicated.sh @@ -53,13 +53,13 @@ ${CLICKHOUSE_CLIENT} --query="SELECT engine_full FROM system.tables WHERE databa | grep -c "ReplicatedVersionedCollapsingMergeTree$ARGS_3 ORDER BY ID" echo -$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='mt'" \ +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE database=currentDatabase() AND table='mt'" \ | grep -c "0" -$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='replacing'" \ +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE database=currentDatabase() AND table='replacing'" \ | grep -c "0" -$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='replacing_ver'" \ +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE database=currentDatabase() AND table='replacing_ver'" \ | grep -c "0" -$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE table='collapsing_ver'" \ +$CLICKHOUSE_CLIENT --echo --query="SELECT is_readonly FROM system.replicas WHERE database=currentDatabase() AND table='collapsing_ver'" \ | grep -c "0" echo From db15f4733ec93cee60af0563b8c1020d67e3391c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Wed, 19 Jun 2024 02:31:12 +0300 Subject: [PATCH 009/433] Suppress Ordinary database warning in test --- tests/queries/0_stateless/03167_attach_as_replicated_errors.sh | 2 ++ tests/queries/0_stateless/03169_attach_as_replicated.sh | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index 225853763ff..a3f2d7cfbf8 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -5,6 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# Creation of a database with Ordinary engine emits a warning. +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="error" ORDINARY_DB="ordinary_$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " diff --git a/tests/queries/0_stateless/03169_attach_as_replicated.sh b/tests/queries/0_stateless/03169_attach_as_replicated.sh index 878115e6a27..15a36f9d783 100755 --- a/tests/queries/0_stateless/03169_attach_as_replicated.sh +++ b/tests/queries/0_stateless/03169_attach_as_replicated.sh @@ -83,7 +83,7 @@ $CLICKHOUSE_CLIENT -n -q " ATTACH TABLE replacing_ver AS NOT REPLICATED; ATTACH TABLE collapsing_ver AS NOT REPLICATED; - SET send_logs_level = 'fatal'; + SET send_logs_level = 'error'; SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$MT_ZK_PATH'; SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$REPLACING_ZK_PATH'; SYSTEM DROP REPLICA 'r1' FROM ZKPATH '$REPLACING_VER_ZK_PATH'; From 72a8097929b8bf14fd7e114ae541f484c68ebcf8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Thu, 20 Jun 2024 02:36:12 +0300 Subject: [PATCH 010/433] Fix too short exception message test (00002) --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- .../queries/0_stateless/03167_attach_as_replicated_errors.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 52da39ee75c..7a48df47146 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1967,10 +1967,10 @@ void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & cr if (engine_name.starts_with("Replicated")) { if (to_replicated) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Table is already replicated"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Can not attach table as replicated, table is already replicated"); } else if (!to_replicated) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Table is already not replicated"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Can not attach table as not replicated, table is already not replicated"); /// Set new engine DatabaseOrdinary::setReplicatedMergeTreeEngine(create, getContext(), to_replicated); diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index a3f2d7cfbf8..3d24cdc8dde 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -23,9 +23,9 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " " echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS NOT REPLICATED" 2>&1)" \ - | grep -c 'Table is already not replicated' + | grep -c 'Can not attach table as not replicated, table is already not replicated' echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE rmt AS REPLICATED" 2>&1)" \ - | grep -c 'Table is already replicated' + | grep -c 'Can not attach table as replicated, table is already replicated' echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE log AS REPLICATED" 2>&1)" \ | grep -c 'Table engine conversion is supported only for MergeTree family engines' echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS REPLICATED ON CLUSTER test_shard_localhost" 2>&1)" \ From aced8589f210903a78a2facf28ae80af2b34cdb2 Mon Sep 17 00:00:00 2001 From: Haydn Date: Tue, 23 Jul 2024 22:05:32 -0700 Subject: [PATCH 011/433] Resolved Conflict by rebase --- src/QueryPipeline/RemoteQueryExecutor.cpp | 47 +++++++++++++++++++++++ src/QueryPipeline/RemoteQueryExecutor.h | 6 +++ 2 files changed, 53 insertions(+) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b08f2002f64..ffc5092c134 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -914,4 +915,50 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } +void RemoteQueryExecutor::reconnect() +{ + // Step 1: Disconnect existing connections + if (connections) + { + connections->disconnect(); + } + + // Step 2: Attempt to re-establish connections + connections = create_connections(nullptr); + if (!connections) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to reconnect to any replica."); + } + + // Step 3: Resend the query if connections are re-established + sendQuery(); +} + +bool RemoteQueryExecutor::isConnectionAlive() const +{ + if (!connections) + { + return false; + } + + // Check the status of each connection in the MultiplexedConnections + for (const auto & connection : connections->getConnections()) + { + // A minimal way to check connection health could be a lightweight query + try + { + connection->sendReadTaskResponse("SELECT 1"); + connection->receivePacket(); // Expecting some form of acknowledgment + } + catch (const Exception &) + { + return false; + } + } + + return true; +} + + + } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 04a59cc3b7e..69820262756 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -218,10 +218,14 @@ public: IConnections & getConnections() { return *connections; } + bool needToSkipUnavailableShard() const bool needToSkipUnavailableShard() const; bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } + void reconnect(); + bool isConnectionAlive() const; + private: RemoteQueryExecutor( const String & query_, @@ -350,3 +354,5 @@ private: }; } + + From 7a51a5310ce6468c840564f566db06e68dad13ce Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 3 Aug 2024 11:02:38 -0700 Subject: [PATCH 012/433] [ClickHouse-66074] Implement SYSTEM LOAD PRIMARY KEY and related unit tests - rebase version --- src/Access/Common/AccessType.h | 1 + src/Access/tests/gtest_access_rights_ops.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 56 +++++++++++++++++++ src/Interpreters/InterpreterSystemQuery.h | 1 + src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserSystemQuery.cpp | 1 + src/QueryPipeline/RemoteQueryExecutor.cpp | 49 ---------------- src/QueryPipeline/RemoteQueryExecutor.h | 5 -- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++ src/Storages/MergeTree/MergeTreeData.h | 3 + .../01271_show_privileges.reference | 1 + .../03202_system_load_primary_key.reference | 3 + .../03202_system_load_primary_key.sql | 11 ++++ 15 files changed, 97 insertions(+), 54 deletions(-) create mode 100644 tests/queries/0_stateless/03202_system_load_primary_key.reference create mode 100644 tests/queries/0_stateless/03202_system_load_primary_key.sql diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index e9f24a8c685..bd78d3bdc1f 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -211,6 +211,7 @@ enum class AccessType : uint8_t M(SYSTEM_FAILPOINT, "SYSTEM ENABLE FAILPOINT, SYSTEM DISABLE FAILPOINT, SYSTEM WAIT FAILPOINT", GLOBAL, SYSTEM) \ M(SYSTEM_LISTEN, "SYSTEM START LISTEN, SYSTEM STOP LISTEN", GLOBAL, SYSTEM) \ M(SYSTEM_JEMALLOC, "SYSTEM JEMALLOC PURGE, SYSTEM JEMALLOC ENABLE PROFILE, SYSTEM JEMALLOC DISABLE PROFILE, SYSTEM JEMALLOC FLUSH PROFILE", GLOBAL, SYSTEM) \ + M(SYSTEM_LOAD_PRIMARY_KEY, "SYSTEM LOAD PRIMARY KEY", TABLE, SYSTEM) \ M(SYSTEM_UNLOAD_PRIMARY_KEY, "SYSTEM UNLOAD PRIMARY KEY", TABLE, SYSTEM) \ M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \ \ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 9fe9a3d8506..60fd43cca8d 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -55,6 +55,7 @@ TEST(AccessRights, Union) "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " + "SYSTEM LOAD PRIMARY KEY, SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " "GRANT SET DEFINER ON db1, GRANT NAMED COLLECTION ADMIN ON db1"); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c284acfa308..c0b3512c0a7 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -14,6 +14,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -787,6 +790,10 @@ BlockIO InterpreterSystemQuery::execute() unloadPrimaryKeys(); break; } + case Type::LOAD_PRIMARY_KEY: { + loadPrimaryKeys(); + break; + } #if USE_JEMALLOC case Type::JEMALLOC_PURGE: @@ -1164,6 +1171,47 @@ void InterpreterSystemQuery::waitLoadingParts() } } + +void InterpreterSystemQuery::loadPrimaryKeys() +{ + if (!table_id.empty()) + { + getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY, table_id.database_name, table_id.table_name); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + + if (auto * merge_tree = dynamic_cast(table.get())) + { + LOG_TRACE(log, "Loading primary keys for table {}", table_id.getFullTableName()); + merge_tree->loadPrimaryKeys(); + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Command LOAD PRIMARY KEY is supported only for MergeTree table, but got: {}", table->getName()); + } + } + else + { + getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); + LOG_TRACE(log, "Loading primary keys for all tables"); + + auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); // Get the appropriate thread pool + + for (auto & database : DatabaseCatalog::instance().getDatabases()) + { + for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) + { + if (auto * merge_tree = dynamic_cast(it->table().get())) + { + thread_pool.scheduleOrThrowOnError([merge_tree]() { merge_tree->loadPrimaryKeys(); }); + } + } + } + + thread_pool.wait(); + } +} + void InterpreterSystemQuery::unloadPrimaryKeys() { if (!table_id.empty()) @@ -1521,6 +1569,14 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); break; } + case Type::LOAD_PRIMARY_KEY: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_LOAD_PRIMARY_KEY); + else + required_access.emplace_back(AccessType::SYSTEM_LOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); + break; + } case Type::STOP_THREAD_FUZZER: case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 776dd7915f0..fa212144a9d 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -60,6 +60,7 @@ private: void syncReplica(ASTSystemQuery & query); void setReplicaReadiness(bool ready); void waitLoadingParts(); + void loadPrimaryKeys(); void unloadPrimaryKeys(); void syncReplicatedDatabase(ASTSystemQuery & query); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a730ea0ba3d..6937d923e08 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -174,6 +174,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::STOP_CLEANUP: case Type::START_CLEANUP: case Type::UNLOAD_PRIMARY_KEY: + case Type::LOAD_PRIMARY_KEY: { if (table) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 167e724dcee..17d55075fce 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -102,6 +102,7 @@ public: CANCEL_VIEW, TEST_VIEW, UNLOAD_PRIMARY_KEY, + LOAD_PRIMARY_KEY, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0545c3e5568..e6dd1156cea 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -324,6 +324,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: case Type::UNLOAD_PRIMARY_KEY: + case Type::LOAD_PRIMARY_KEY: { if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false)) return false; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 9d1519de0b9..b93ed43ffa3 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -925,54 +925,6 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } -<<<<<<< HEAD -void RemoteQueryExecutor::reconnect() -{ - // Step 1: Disconnect existing connections - if (connections) - { - connections->disconnect(); - } - - // Step 2: Attempt to re-establish connections - connections = create_connections(nullptr); - if (!connections) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to reconnect to any replica."); - } - - // Step 3: Resend the query if connections are re-established - sendQuery(); -} - -bool RemoteQueryExecutor::isConnectionAlive() const -{ - if (!connections) - { - return false; - } - - // Check the status of each connection in the MultiplexedConnections - for (const auto & connection : connections->getConnections()) - { - // A minimal way to check connection health could be a lightweight query - try - { - connection->sendReadTaskResponse("SELECT 1"); - connection->receivePacket(); // Expecting some form of acknowledgment - } - catch (const Exception &) - { - return false; - } - } - - return true; -} - - - -======= bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() { #if defined(OS_LINUX) @@ -1006,5 +958,4 @@ bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() return false; } ->>>>>>> upstream/master } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 062d64f9dca..f56ca480001 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -222,13 +222,8 @@ public: bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } -<<<<<<< HEAD - void reconnect(); - bool isConnectionAlive() const; -======= /// return true if parallel replica packet was processed bool processParallelReplicaPacketIfAny(); ->>>>>>> upstream/master private: RemoteQueryExecutor( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 85ef0472ce7..05c544aae3f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -65,6 +65,7 @@ enum class DataPartRemovalState : uint8_t class IMergeTreeDataPart : public std::enable_shared_from_this, public DataPartStorageHolder { public: + void loadIndexWithLock() const { std::scoped_lock lock(index_mutex); loadIndex(); } static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -371,6 +372,7 @@ public: Index getIndex() const; void setIndex(const Columns & cols_); void setIndex(Columns && cols_); + void loadIndex() const TSA_REQUIRES(index_mutex); void unloadIndex(); bool isIndexLoaded() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2e10f5a0227..a22016e73a2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8639,6 +8639,21 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad return true; } +void MergeTreeData::loadPrimaryKeys() +{ + DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting}; + + for (const auto &data_part : getDataParts(affordable_states)) + { + if (data_part->isProjectionPart()) + { + continue; + } + // Lock the mutex and call loadIndexWithLock + const_cast(*data_part).loadIndexWithLock(); + } +} + void MergeTreeData::unloadPrimaryKeys() { for (auto & part : getAllDataPartsVector()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6662df3db84..eb4573d684e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1094,6 +1094,9 @@ public: /// Unloads primary keys of all parts. void unloadPrimaryKeys(); + // Load primary keys of all parts + void loadPrimaryKeys(); + /// Unloads primary keys of outdated parts that are not used by any query. /// Returns the number of parts for which index was unloaded. size_t unloadPrimaryKeysOfOutdatedParts(); diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 17554f5c8a5..1b663ee04eb 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -160,6 +160,7 @@ SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM SYSTEM FAILPOINT ['SYSTEM ENABLE FAILPOINT','SYSTEM DISABLE FAILPOINT','SYSTEM WAIT FAILPOINT'] GLOBAL SYSTEM SYSTEM LISTEN ['SYSTEM START LISTEN','SYSTEM STOP LISTEN'] GLOBAL SYSTEM SYSTEM JEMALLOC ['SYSTEM JEMALLOC PURGE','SYSTEM JEMALLOC ENABLE PROFILE','SYSTEM JEMALLOC DISABLE PROFILE','SYSTEM JEMALLOC FLUSH PROFILE'] GLOBAL SYSTEM +SYSTEM LOAD PRIMARY KEY ['SYSTEM LOAD PRIMARY KEY'] TABLE SYSTEM SYSTEM UNLOAD PRIMARY KEY ['SYSTEM UNLOAD PRIMARY KEY'] TABLE SYSTEM SYSTEM [] \N ALL dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference new file mode 100644 index 00000000000..48905639abd --- /dev/null +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -0,0 +1,3 @@ +1 a +2 b +3 c \ No newline at end of file diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql new file mode 100644 index 00000000000..775d5336174 --- /dev/null +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -0,0 +1,11 @@ +-- Test to ensure SYSTEM LOAD PRIMARY KEY works as expected +CREATE TABLE test_load_primary_key (id Int32, value String) ENGINE = MergeTree() ORDER BY id; + +-- Inserting some data +INSERT INTO test_load_primary_key VALUES (1, 'a'), (2, 'b'), (3, 'c'); + +-- Load primary key +SYSTEM LOAD PRIMARY KEY test_load_primary_key; + +-- Select to verify the data is correctly loaded +SELECT * FROM test_load_primary_key ORDER BY id; From df7c58f0f1fceb5353633254edc432597b6de8d7 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 4 Aug 2024 09:57:51 -0700 Subject: [PATCH 013/433] Fixed Style Checks --- src/QueryPipeline/RemoteQueryExecutor.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.h | 3 --- 2 files changed, 4 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index f56ca480001..763073cdeaa 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -217,7 +217,6 @@ public: IConnections & getConnections() { return *connections; } - bool needToSkipUnavailableShard() const bool needToSkipUnavailableShard() const; bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 05c544aae3f..36a5896144e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -690,9 +690,6 @@ private: virtual void appendFilesOfIndexGranularity(Strings & files) const; - /// Loads the index file. - void loadIndex() const TSA_REQUIRES(index_mutex); - void appendFilesOfIndex(Strings & files) const; /// Load rows count for this part from disk (for the newer storage format version). From 0d739c2acfab6a6b2f09c5c4531cf5f2269a70b1 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 11 Aug 2024 10:39:53 -0700 Subject: [PATCH 014/433] Added Missing new line in 03202.reference --- .../queries/0_stateless/03202_system_load_primary_key.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 48905639abd..14c2e9dc054 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,3 +1,3 @@ 1 a 2 b -3 c \ No newline at end of file +3 c From 0bbcb796ca46eb70d3f4894bb6813d71d8463ee1 Mon Sep 17 00:00:00 2001 From: Haydn Date: Thu, 15 Aug 2024 20:01:12 -0700 Subject: [PATCH 015/433] Remove duplicate specifications in gtest_access_rights_ops --- src/Access/tests/gtest_access_rights_ops.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 60fd43cca8d..7f905086847 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -54,7 +54,6 @@ TEST(AccessRights, Union) "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM VIEWS, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM VIRTUAL PARTS UPDATE, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, " - "SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " "SYSTEM LOAD PRIMARY KEY, SYSTEM UNLOAD PRIMARY KEY, dictGet ON db1.*, GRANT TABLE ENGINE ON db1, " "GRANT SET DEFINER ON db1, GRANT NAMED COLLECTION ADMIN ON db1"); } From f351c48515ed7f39c8860613c88afe5ef08fe8aa Mon Sep 17 00:00:00 2001 From: Haydn Date: Fri, 16 Aug 2024 19:28:34 -0700 Subject: [PATCH 016/433] Resolved merge conflicts in RemoteQueryExecutor --- src/QueryPipeline/RemoteQueryExecutor.cpp | 49 ----------------------- src/QueryPipeline/RemoteQueryExecutor.h | 6 --- 2 files changed, 55 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 09a375aab6f..d42cf92ca16 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -925,54 +925,6 @@ bool RemoteQueryExecutor::needToSkipUnavailableShard() const return context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()); } -<<<<<<< HEAD -void RemoteQueryExecutor::reconnect() -{ - // Step 1: Disconnect existing connections - if (connections) - { - connections->disconnect(); - } - - // Step 2: Attempt to re-establish connections - connections = create_connections(nullptr); - if (!connections) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to reconnect to any replica."); - } - - // Step 3: Resend the query if connections are re-established - sendQuery(); -} - -bool RemoteQueryExecutor::isConnectionAlive() const -{ - if (!connections) - { - return false; - } - - // Check the status of each connection in the MultiplexedConnections - for (const auto & connection : connections->getConnections()) - { - // A minimal way to check connection health could be a lightweight query - try - { - connection->sendReadTaskResponse("SELECT 1"); - connection->receivePacket(); // Expecting some form of acknowledgment - } - catch (const Exception &) - { - return false; - } - } - - return true; -} - - - -======= bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() { #if defined(OS_LINUX) @@ -1006,5 +958,4 @@ bool RemoteQueryExecutor::processParallelReplicaPacketIfAny() return false; } ->>>>>>> upstream/master } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 062d64f9dca..763073cdeaa 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -217,18 +217,12 @@ public: IConnections & getConnections() { return *connections; } - bool needToSkipUnavailableShard() const bool needToSkipUnavailableShard() const; bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; } -<<<<<<< HEAD - void reconnect(); - bool isConnectionAlive() const; -======= /// return true if parallel replica packet was processed bool processParallelReplicaPacketIfAny(); ->>>>>>> upstream/master private: RemoteQueryExecutor( From 6e5a7fd64e5baf983323c7f7e915c7d928907090 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sat, 17 Aug 2024 21:45:30 -0700 Subject: [PATCH 017/433] Added Logic to avoid timeout issue --- src/Interpreters/InterpreterSystemQuery.cpp | 14 +++++++++----- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++++++++++++----- 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 16308db426d..82854d0bba3 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1190,9 +1190,7 @@ void InterpreterSystemQuery::loadPrimaryKeys() else { getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); - LOG_TRACE(log, "Loading primary keys for all tables"); - auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); // Get the appropriate thread pool for (auto & database : DatabaseCatalog::instance().getDatabases()) { @@ -1200,12 +1198,18 @@ void InterpreterSystemQuery::loadPrimaryKeys() { if (auto * merge_tree = dynamic_cast(it->table().get())) { - thread_pool.scheduleOrThrowOnError([merge_tree]() { merge_tree->loadPrimaryKeys(); }); + try + { + // Directly load the primary keys without thread pool + merge_tree->loadPrimaryKeys(); + } + catch (const Exception & ex) + { + LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); + } } } } - - thread_pool.wait(); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 733ea613af0..d5dbcaac8b5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,19 +8659,29 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { - DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting}; + // Define the states of parts that need to be processed + DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - for (const auto &data_part : getDataParts(affordable_states)) + for (const auto & data_part : getDataParts(affordable_states)) { + // Skip projection parts, as they do not need primary key loading if (data_part->isProjectionPart()) - { continue; + + try + { + // Load primary keys with a lock on the index to avoid concurrent access issues + const_cast(*data_part).loadIndexWithLock(); + } + catch (const Exception & ex) + { + // Handle any errors but allow the loop to continue + // (no logging, but this catch block prevents crashes) } - // Lock the mutex and call loadIndexWithLock - const_cast(*data_part).loadIndexWithLock(); } } + void MergeTreeData::unloadPrimaryKeys() { for (auto & part : getAllDataPartsVector()) From 4ee73b31f17cffd6ea5a65243431ffb3122a2ccf Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 18 Aug 2024 21:34:31 -0700 Subject: [PATCH 018/433] handle ex properly --- src/Storages/MergeTree/MergeTreeData.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d5dbcaac8b5..16cd338df5a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,24 +8659,33 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { + // Ensure thread-safety if needed by using a mutex lock + std::unique_lock lock(load_primary_key_mutex, std::defer_lock); + if (!lock.try_lock()) + return; + // Define the states of parts that need to be processed DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; + // Loop through the parts for (const auto & data_part : getDataParts(affordable_states)) { - // Skip projection parts, as they do not need primary key loading + // Skip projection parts if (data_part->isProjectionPart()) continue; try { - // Load primary keys with a lock on the index to avoid concurrent access issues + // Lock the index and load the primary key const_cast(*data_part).loadIndexWithLock(); + + // Log successful load (optional) + LOG_TEST(log, "Loaded primary key for part {}", data_part->name); } catch (const Exception & ex) { - // Handle any errors but allow the loop to continue - // (no logging, but this catch block prevents crashes) + // Log the error for diagnostics (optional) + LOG_ERROR(log, "Failed to load primary key for part {}: {}", data_part->name, ex.what()); } } } From ae9942f520c3ab4290650068850b721825f4b5e3 Mon Sep 17 00:00:00 2001 From: Haydn Date: Sun, 18 Aug 2024 22:02:24 -0700 Subject: [PATCH 019/433] Fixed loadPK logic --- src/Storages/MergeTree/MergeTreeData.cpp | 31 ++++++++++-------------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 16cd338df5a..e0be9a60c50 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,35 +8659,30 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { - // Ensure thread-safety if needed by using a mutex lock - std::unique_lock lock(load_primary_key_mutex, std::defer_lock); - if (!lock.try_lock()) - return; - // Define the states of parts that need to be processed DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - // Loop through the parts + // Thread pool to process loading in parallel + auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); + for (const auto & data_part : getDataParts(affordable_states)) { - // Skip projection parts + // Skip projection parts, as they do not need primary key loading if (data_part->isProjectionPart()) continue; - try + // Check if the index is already loaded to avoid redundant loading + if (!data_part->isIndexLoaded()) { - // Lock the index and load the primary key - const_cast(*data_part).loadIndexWithLock(); - - // Log successful load (optional) - LOG_TEST(log, "Loaded primary key for part {}", data_part->name); - } - catch (const Exception & ex) - { - // Log the error for diagnostics (optional) - LOG_ERROR(log, "Failed to load primary key for part {}: {}", data_part->name, ex.what()); + // Use thread pool to parallelize the work + thread_pool.scheduleOrThrowOnError([data_part] { + const_cast(*data_part).loadIndexWithLock(); + }); } } + + // Wait for all tasks to finish + thread_pool.wait(); } From 4d70e1d092a6eb94a62d2a36913629d5bd20b21e Mon Sep 17 00:00:00 2001 From: Haydn Date: Mon, 19 Aug 2024 10:10:20 -0700 Subject: [PATCH 020/433] Prevent data race when load PrimaryKeys --- src/Storages/MergeTree/MergeTreeData.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e0be9a60c50..2345ba303a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8665,6 +8665,9 @@ void MergeTreeData::loadPrimaryKeys() // Thread pool to process loading in parallel auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); + // Mutex to protect shared data access + std::mutex mutex; + for (const auto & data_part : getDataParts(affordable_states)) { // Skip projection parts, as they do not need primary key loading @@ -8675,7 +8678,9 @@ void MergeTreeData::loadPrimaryKeys() if (!data_part->isIndexLoaded()) { // Use thread pool to parallelize the work - thread_pool.scheduleOrThrowOnError([data_part] { + thread_pool.scheduleOrThrowOnError([&mutex, data_part] { + // Lock the mutex before loading index to prevent data races + std::lock_guard lock(mutex); const_cast(*data_part).loadIndexWithLock(); }); } From c728d6255834f6de34d0c2a4537cd525e660f132 Mon Sep 17 00:00:00 2001 From: Haydn Date: Mon, 19 Aug 2024 10:21:04 -0700 Subject: [PATCH 021/433] Removed duplicated #include statement to avoid unexpected werror --- src/Interpreters/InterpreterSystemQuery.cpp | 4 +--- src/QueryPipeline/RemoteQueryExecutor.h | 2 -- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 82854d0bba3..32580bcd081 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -14,9 +14,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -67,6 +64,7 @@ #include #include #include +#include #include #include #include diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 763073cdeaa..83f33607dbf 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -354,5 +354,3 @@ private: }; } - - From f2749ed83ef556a5bc41d4adda3ee5547d6130da Mon Sep 17 00:00:00 2001 From: Haydn Date: Tue, 3 Sep 2024 21:02:50 -0700 Subject: [PATCH 022/433] Added logic to avoid server died --- src/Interpreters/InterpreterSystemQuery.cpp | 44 +++++++++++++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 29 +++++++++----- 2 files changed, 57 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 32580bcd081..a6dd568aafa 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1177,40 +1177,70 @@ void InterpreterSystemQuery::loadPrimaryKeys() if (auto * merge_tree = dynamic_cast(table.get())) { LOG_TRACE(log, "Loading primary keys for table {}", table_id.getFullTableName()); - merge_tree->loadPrimaryKeys(); + try + { + merge_tree->loadPrimaryKeys(); + } + catch (const Exception & ex) + { + LOG_ERROR(log, "Failed to load primary keys for table {}: {}", table_id.getFullTableName(), ex.message()); + } } else { throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Command LOAD PRIMARY KEY is supported only for MergeTree table, but got: {}", table->getName()); + ErrorCodes::BAD_ARGUMENTS, "Command LOAD PRIMARY KEY is supported only for MergeTree tables, but got: {}", table->getName()); } } else { getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); + // Define a thread pool for parallel processing + auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); + // Cap the number of concurrent tasks based on thread pool availability + size_t max_tasks = std::min(thread_pool.maxConcurrency(), DatabaseCatalog::instance().getDatabases().size()); + + // Process databases and tables in parallel + size_t tasks_scheduled = 0; for (auto & database : DatabaseCatalog::instance().getDatabases()) { for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) { if (auto * merge_tree = dynamic_cast(it->table().get())) { - try + // Schedule task for thread pool + if (tasks_scheduled < max_tasks) { - // Directly load the primary keys without thread pool - merge_tree->loadPrimaryKeys(); + thread_pool.scheduleOrThrowOnError([merge_tree, log=log] { + try + { + merge_tree->loadPrimaryKeys(); // Calls the improved loadPrimaryKeys in MergeTreeData + } + catch (const Exception & ex) + { + LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); + } + }); + tasks_scheduled++; } - catch (const Exception & ex) + else { - LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); + // Wait for current tasks to finish before scheduling more + thread_pool.wait(); + tasks_scheduled = 0; // Reset task counter } } } } + + // Wait for all tasks to complete + thread_pool.wait(); } } + void InterpreterSystemQuery::unloadPrimaryKeys() { if (!table_id.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2345ba303a4..60605385561 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8665,8 +8665,11 @@ void MergeTreeData::loadPrimaryKeys() // Thread pool to process loading in parallel auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); - // Mutex to protect shared data access - std::mutex mutex; + // Limit the number of tasks to avoid overloading the thread pool + size_t max_parallel_tasks = std::min(thread_pool.maxConcurrency(), getDataParts(affordable_states).size()); + + // Keep track of scheduled tasks + size_t scheduled_tasks = 0; for (const auto & data_part : getDataParts(affordable_states)) { @@ -8677,16 +8680,24 @@ void MergeTreeData::loadPrimaryKeys() // Check if the index is already loaded to avoid redundant loading if (!data_part->isIndexLoaded()) { - // Use thread pool to parallelize the work - thread_pool.scheduleOrThrowOnError([&mutex, data_part] { - // Lock the mutex before loading index to prevent data races - std::lock_guard lock(mutex); - const_cast(*data_part).loadIndexWithLock(); - }); + // Use thread pool to parallelize the work, limiting to max_parallel_tasks + if (scheduled_tasks < max_parallel_tasks) + { + thread_pool.scheduleOrThrowOnError([data_part] { + const_cast(*data_part).loadIndexWithLock(); + }); + scheduled_tasks++; + } + else + { + // If task limit is reached, wait for the current batch to finish + thread_pool.wait(); + scheduled_tasks = 0; // Reset the task counter for the next batch + } } } - // Wait for all tasks to finish + // Ensure all remaining tasks finish thread_pool.wait(); } From 6e1f61adc904928dff0fe6287bd34284c87d7796 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 9 Sep 2024 01:16:02 +0300 Subject: [PATCH 023/433] Disable support for not short attach to replicated queries --- src/Databases/DatabaseOrdinary.cpp | 4 ++-- src/Databases/DatabaseOrdinary.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 5 ++++- src/Parsers/ParserCreateQuery.cpp | 1 + .../0_stateless/03167_attach_as_replicated_errors.reference | 1 + .../queries/0_stateless/03167_attach_as_replicated_errors.sh | 2 ++ 6 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 91fbf2131c9..451adafc4f5 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -89,7 +89,7 @@ static void checkReplicaPathExists(ASTCreateQuery & create_query, ContextPtr loc ); } -void DatabaseOrdinary::setReplicatedMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr local_context, bool replicated) +void DatabaseOrdinary::setMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr local_context, bool replicated) { auto * storage = create_query.storage; auto args = std::make_shared(); @@ -181,7 +181,7 @@ void DatabaseOrdinary::convertMergeTreeToReplicatedIfNeeded(ASTPtr ast, const Qu LOG_INFO(log, "Found {} flag for table {}. Will try to change it's engine in metadata to replicated.", CONVERT_TO_REPLICATED_FLAG_NAME, backQuote(qualified_name.getFullName())); checkReplicaPathExists(create_query, getContext()); - setReplicatedMergeTreeEngine(create_query, getContext(), true); + setMergeTreeEngine(create_query, getContext(), /*replicated*/ true); /// Write changes to metadata String table_metadata_path = full_path; diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index 3b9ea973a3f..b20e0261ea5 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -73,7 +73,7 @@ public: return permanently_detached_tables; } - static void setReplicatedMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr context, bool replicated); + static void setMergeTreeEngine(ASTCreateQuery & create_query, ContextPtr context, bool replicated); protected: virtual void commitAlterTable( diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bddb4365b9a..87436040e62 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1383,6 +1383,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } /// TODO throw exception if !create.attach_short_syntax && !create.attach_from_path && !internal + if (!create.attach_short_syntax && create.attach_as_replicated.has_value()) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Attaching table as [not] replicated is supported only for short attach queries"); if (create.attach_from_path) { @@ -2207,7 +2210,7 @@ void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & cr throw Exception(ErrorCodes::INCORRECT_QUERY, "Can not attach table as not replicated, table is already not replicated"); /// Set new engine - DatabaseOrdinary::setReplicatedMergeTreeEngine(create, getContext(), to_replicated); + DatabaseOrdinary::setMergeTreeEngine(create, getContext(), to_replicated); /// Save new metadata String table_metadata_path = database->getObjectMetadataPath(create.getTable()); diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 989df261203..ba19ec2bdc8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -858,6 +858,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = query; query->attach = attach; + query->attach_as_replicated = attach_as_replicated; query->replace_table = replace; query->create_or_replace = or_replace; query->if_not_exists = if_not_exists; diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference index a9e2f17562a..fcb49fa9945 100644 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.reference @@ -4,3 +4,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index 3d24cdc8dde..f456ab2e721 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -30,6 +30,8 @@ echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_log | grep -c 'Table engine conversion is supported only for MergeTree family engines' echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS REPLICATED ON CLUSTER test_shard_localhost" 2>&1)" \ | grep -c 'ATTACH AS \[NOT\] REPLICATED is not supported for ON CLUSTER queries' +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE mt AS REPLICATED ( A Int64, D Date, S String ) ENGINE MergeTree ORDER BY A" 2>&1)" \ + | grep -c 'Attaching table as \[not\] replicated is supported only for short attach queries' echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE $ORDINARY_DB.mt AS REPLICATED" 2>&1)" \ | grep -c 'Table engine conversion to replicated is supported only for Atomic databases' From 6d62adbfb6c506d319b8151e58a11c640d525338 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Mon, 9 Sep 2024 13:25:29 +0300 Subject: [PATCH 024/433] Suppress ordinary db warning in test --- tests/queries/0_stateless/03167_attach_as_replicated_errors.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index f456ab2e721..1d10e93ecc0 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh # Creation of a database with Ordinary engine emits a warning. -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="error" +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal ORDINARY_DB="ordinary_$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " From 89f1a81a395fc8e1cdb1bafd1ad4865079e7c005 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Thu, 12 Sep 2024 19:20:37 +0300 Subject: [PATCH 025/433] Suppress ordinary db warning in test 2 --- .../0_stateless/03167_attach_as_replicated_errors.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index 1d10e93ecc0..b8df052edb3 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -42,11 +42,11 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " DROP DATABASE $ORDINARY_DB; " -${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_1 (id UInt32) ENGINE=MergeTree() ORDER BY id;" -UUID=$($CLICKHOUSE_CLIENT --query="SELECT uuid FROM system.tables WHERE database=currentDatabase() AND table='already_exists_1';") +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -q "CREATE TABLE already_exists_1 (id UInt32) ENGINE=MergeTree() ORDER BY id;" +UUID=$($CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 --query="SELECT uuid FROM system.tables WHERE database=currentDatabase() AND table='already_exists_1';") ARGS_1="('/clickhouse/tables/$UUID/s1', 'r1')" # Suppress style check for zk path -${CLICKHOUSE_CLIENT} -q "CREATE TABLE already_exists_2 (id UInt32) ENGINE=ReplicatedMergeTree$ARGS_1 ORDER BY id;" -${CLICKHOUSE_CLIENT} -q "DETACH TABLE already_exists_1;" -echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="ATTACH TABLE already_exists_1 AS REPLICATED" 2>&1)" \ +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -q "CREATE TABLE already_exists_2 (id UInt32) ENGINE=ReplicatedMergeTree$ARGS_1 ORDER BY id;" +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -q "DETACH TABLE already_exists_1;" +echo "$(${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 --server_logs_file=/dev/null --query="ATTACH TABLE already_exists_1 AS REPLICATED" 2>&1)" \ | grep -c 'There already is an active replica with this replica path' -${CLICKHOUSE_CLIENT} -q "ATTACH TABLE already_exists_1 AS NOT REPLICATED;" +${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -q "ATTACH TABLE already_exists_1 AS NOT REPLICATED;" From 530a86e4622b158784d798deb49a30e99289ebda Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 14 Sep 2024 21:19:55 -0700 Subject: [PATCH 026/433] Change the logic in testing, swap the order of function specification and correct comment formats --- src/Parsers/ASTSystemQuery.cpp | 2 +- src/Parsers/ASTSystemQuery.h | 2 +- src/Parsers/ParserSystemQuery.cpp | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 22 +++++++++------- src/Storages/MergeTree/MergeTreeData.h | 6 ++--- .../03202_system_load_primary_key.reference | 3 +++ .../03202_system_load_primary_key.sql | 26 ++++++++++++++++++- 8 files changed, 45 insertions(+), 19 deletions(-) diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 370ebfbcdf0..2e8d3498d26 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -173,8 +173,8 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::START_PULLING_REPLICATION_LOG: case Type::STOP_CLEANUP: case Type::START_CLEANUP: - case Type::UNLOAD_PRIMARY_KEY: case Type::LOAD_PRIMARY_KEY: + case Type::UNLOAD_PRIMARY_KEY: { if (table) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 17d55075fce..ea3bbd899a9 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -101,8 +101,8 @@ public: STOP_VIEWS, CANCEL_VIEW, TEST_VIEW, - UNLOAD_PRIMARY_KEY, LOAD_PRIMARY_KEY, + UNLOAD_PRIMARY_KEY, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index c597b5ad557..c2fdda86f3c 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -323,8 +323,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & /// START/STOP DISTRIBUTED SENDS does not require table case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: - case Type::UNLOAD_PRIMARY_KEY: case Type::LOAD_PRIMARY_KEY: + case Type::UNLOAD_PRIMARY_KEY: { if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false)) return false; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index d42cf92ca16..6f8b3931803 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 60605385561..295791fc97a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,28 +8659,28 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { - // Define the states of parts that need to be processed + /// Define the states of parts that need to be processed DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - // Thread pool to process loading in parallel + /// Thread pool to process loading in parallel auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); - // Limit the number of tasks to avoid overloading the thread pool + /// Limit the number of tasks to avoid overloading the thread pool size_t max_parallel_tasks = std::min(thread_pool.maxConcurrency(), getDataParts(affordable_states).size()); - // Keep track of scheduled tasks + /// Keep track of scheduled tasks size_t scheduled_tasks = 0; for (const auto & data_part : getDataParts(affordable_states)) { - // Skip projection parts, as they do not need primary key loading + /// Skip projection parts, as they do not need primary key loading if (data_part->isProjectionPart()) continue; - // Check if the index is already loaded to avoid redundant loading + /// Check if the index is already loaded to avoid redundant loading if (!data_part->isIndexLoaded()) { - // Use thread pool to parallelize the work, limiting to max_parallel_tasks + /// Use thread pool to parallelize the work, limiting to max_parallel_tasks if (scheduled_tasks < max_parallel_tasks) { thread_pool.scheduleOrThrowOnError([data_part] { @@ -8690,14 +8690,16 @@ void MergeTreeData::loadPrimaryKeys() } else { - // If task limit is reached, wait for the current batch to finish + /// If task limit is reached, wait for the current batch to finish thread_pool.wait(); - scheduled_tasks = 0; // Reset the task counter for the next batch + + /// Reset the task counter for the next batch + scheduled_tasks = 0; } } } - // Ensure all remaining tasks finish + /// Ensure all remaining tasks finish thread_pool.wait(); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index eb4573d684e..19d18c03466 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1091,11 +1091,9 @@ public: static VirtualColumnsDescription createVirtuals(const StorageInMemoryMetadata & metadata); - /// Unloads primary keys of all parts. - void unloadPrimaryKeys(); - - // Load primary keys of all parts + /// Load/unload primary keys of all data parts void loadPrimaryKeys(); + void unloadPrimaryKeys(); /// Unloads primary keys of outdated parts that are not used by any query. /// Returns the number of parts for which index was unloaded. diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 14c2e9dc054..43e1806c146 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,3 +1,6 @@ +0 0 +0 0 1 a 2 b 3 c +0 0 diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 775d5336174..63b4e4111b9 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -1,11 +1,35 @@ --- Test to ensure SYSTEM LOAD PRIMARY KEY works as expected +-- Tags: no-parallel +DROP TABLE IF EXISTS test_load_primary_key; + +-- Create the test table CREATE TABLE test_load_primary_key (id Int32, value String) ENGINE = MergeTree() ORDER BY id; -- Inserting some data INSERT INTO test_load_primary_key VALUES (1, 'a'), (2, 'b'), (3, 'c'); +-- Check primary key memory before loading (this checks if it's not loaded yet) +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +FROM system.parts +WHERE database = currentDatabase() +AND table = 'test_load_primary_key'; + -- Load primary key SYSTEM LOAD PRIMARY KEY test_load_primary_key; +-- Check primary key memory after loading +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +FROM system.parts +WHERE database = currentDatabase() +AND table = 'test_load_primary_key'; + +-- Unload primary key +SYSTEM UNLOAD PRIMARY KEY test_load_primary_key; + +-- Check primary key memory after unloading +SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +FROM system.parts +WHERE database = currentDatabase() +AND table = 'test_load_primary_key'; + -- Select to verify the data is correctly loaded SELECT * FROM test_load_primary_key ORDER BY id; From d63d007d4490faec258bf46bf2cd6ddc3fe40fd2 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 14 Sep 2024 21:45:33 -0700 Subject: [PATCH 027/433] Fixed all issue brought up by rschu1ze --- src/Interpreters/InterpreterSystemQuery.cpp | 50 ++++++--------------- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 31 ++----------- 3 files changed, 19 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index a6dd568aafa..7d8fa2274e5 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1196,47 +1196,25 @@ void InterpreterSystemQuery::loadPrimaryKeys() { getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); - // Define a thread pool for parallel processing - auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); - - // Cap the number of concurrent tasks based on thread pool availability - size_t max_tasks = std::min(thread_pool.maxConcurrency(), DatabaseCatalog::instance().getDatabases().size()); - - // Process databases and tables in parallel - size_t tasks_scheduled = 0; + /// Process each database and table sequentially for (auto & database : DatabaseCatalog::instance().getDatabases()) { for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) { if (auto * merge_tree = dynamic_cast(it->table().get())) { - // Schedule task for thread pool - if (tasks_scheduled < max_tasks) + try { - thread_pool.scheduleOrThrowOnError([merge_tree, log=log] { - try - { - merge_tree->loadPrimaryKeys(); // Calls the improved loadPrimaryKeys in MergeTreeData - } - catch (const Exception & ex) - { - LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); - } - }); - tasks_scheduled++; + /// Directly call loadPrimaryKeys without concurrency + merge_tree->loadPrimaryKeys(); } - else + catch (const Exception & ex) { - // Wait for current tasks to finish before scheduling more - thread_pool.wait(); - tasks_scheduled = 0; // Reset task counter + LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); } } } } - - // Wait for all tasks to complete - thread_pool.wait(); } } @@ -1590,14 +1568,6 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_JEMALLOC); break; } - case Type::UNLOAD_PRIMARY_KEY: - { - if (!query.table) - required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); - else - required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); - break; - } case Type::LOAD_PRIMARY_KEY: { if (!query.table) @@ -1606,6 +1576,14 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_LOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); break; } + case Type::UNLOAD_PRIMARY_KEY: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); + else + required_access.emplace_back(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, query.getDatabase(), query.getTable()); + break; + } case Type::STOP_THREAD_FUZZER: case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 36a5896144e..6254311e4d6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -65,7 +65,6 @@ enum class DataPartRemovalState : uint8_t class IMergeTreeDataPart : public std::enable_shared_from_this, public DataPartStorageHolder { public: - void loadIndexWithLock() const { std::scoped_lock lock(index_mutex); loadIndex(); } static constexpr auto DATA_FILE_EXTENSION = ".bin"; using Checksums = MergeTreeDataPartChecksums; @@ -93,6 +92,7 @@ public: Type part_type_, const IMergeTreeDataPart * parent_part_); + virtual MergeTreeReaderPtr getReader( const NamesAndTypesList & columns_, const StorageSnapshotPtr & storage_snapshot, @@ -374,6 +374,7 @@ public: void setIndex(Columns && cols_); void loadIndex() const TSA_REQUIRES(index_mutex); void unloadIndex(); + void loadIndexWithLock() const { std::scoped_lock lock(index_mutex); loadIndex(); } bool isIndexLoaded() const; /// For data in RAM ('index') diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 295791fc97a..faf1006109f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8662,15 +8662,7 @@ void MergeTreeData::loadPrimaryKeys() /// Define the states of parts that need to be processed DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - /// Thread pool to process loading in parallel - auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); - - /// Limit the number of tasks to avoid overloading the thread pool - size_t max_parallel_tasks = std::min(thread_pool.maxConcurrency(), getDataParts(affordable_states).size()); - - /// Keep track of scheduled tasks - size_t scheduled_tasks = 0; - + /// Sequentially load primary keys without using a thread pool for (const auto & data_part : getDataParts(affordable_states)) { /// Skip projection parts, as they do not need primary key loading @@ -8680,27 +8672,10 @@ void MergeTreeData::loadPrimaryKeys() /// Check if the index is already loaded to avoid redundant loading if (!data_part->isIndexLoaded()) { - /// Use thread pool to parallelize the work, limiting to max_parallel_tasks - if (scheduled_tasks < max_parallel_tasks) - { - thread_pool.scheduleOrThrowOnError([data_part] { - const_cast(*data_part).loadIndexWithLock(); - }); - scheduled_tasks++; - } - else - { - /// If task limit is reached, wait for the current batch to finish - thread_pool.wait(); - - /// Reset the task counter for the next batch - scheduled_tasks = 0; - } + /// Load the index for this part + const_cast(*data_part).loadIndexWithLock(); } } - - /// Ensure all remaining tasks finish - thread_pool.wait(); } From 78282bf67d95ac60aad50fd6601b791cd45f5e92 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 14 Sep 2024 21:50:57 -0700 Subject: [PATCH 028/433] Refactor loadPrimaryKeys Functions again --- src/Interpreters/InterpreterSystemQuery.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 16 ++++++++++------ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 7d8fa2274e5..0a10fc1b6fb 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1196,7 +1196,7 @@ void InterpreterSystemQuery::loadPrimaryKeys() { getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); - /// Process each database and table sequentially + /// Process databases and tables sequentially, without thread pool concurrency at the table level for (auto & database : DatabaseCatalog::instance().getDatabases()) { for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) @@ -1205,7 +1205,7 @@ void InterpreterSystemQuery::loadPrimaryKeys() { try { - /// Directly call loadPrimaryKeys without concurrency + /// Calls the improved loadPrimaryKeys in MergeTreeData merge_tree->loadPrimaryKeys(); } catch (const Exception & ex) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index faf1006109f..44973aaf89a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,23 +8659,27 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { - /// Define the states of parts that need to be processed DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - /// Sequentially load primary keys without using a thread pool + /// Thread pool to process parts within each table in parallel + auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); + for (const auto & data_part : getDataParts(affordable_states)) { - /// Skip projection parts, as they do not need primary key loading if (data_part->isProjectionPart()) continue; - /// Check if the index is already loaded to avoid redundant loading if (!data_part->isIndexLoaded()) { - /// Load the index for this part - const_cast(*data_part).loadIndexWithLock(); + /// Use thread pool to parallelize part loading + thread_pool.scheduleOrThrowOnError([data_part] { + const_cast(*data_part).loadIndexWithLock(); + }); } } + + /// Wait for all parts to finish loading + thread_pool.wait(); } From 8791357429de0a76d3002fea7648d980d6dcc2e2 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sun, 15 Sep 2024 09:54:30 -0700 Subject: [PATCH 029/433] Fixed the .reference file --- .../0_stateless/03202_system_load_primary_key.reference | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 43e1806c146..5fe123da976 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,6 +1,7 @@ 0 0 0 0 +0 0 1 a 2 b 3 c -0 0 + From 4ea66a18c725a4cf4507f8f3ea46cabcf1e8890b Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sun, 15 Sep 2024 10:55:24 -0700 Subject: [PATCH 030/433] Error Format in .reference --- .../0_stateless/03202_system_load_primary_key.reference | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 5fe123da976..f02fef9dcdc 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,7 +1,6 @@ 0 0 0 0 -0 0 +0 0 1 a 2 b 3 c - From 7d011d7343e254dd55f2e05fbf1ec1cd753cae76 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Mon, 23 Sep 2024 21:51:25 -0700 Subject: [PATCH 031/433] Make changes in testing and modify the catch() --- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- .../03202_system_load_primary_key.reference | 27 +++++++++--- .../03202_system_load_primary_key.sql | 43 +++++++++++++------ 3 files changed, 51 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 0a10fc1b6fb..c3da850c6a1 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1208,7 +1208,7 @@ void InterpreterSystemQuery::loadPrimaryKeys() /// Calls the improved loadPrimaryKeys in MergeTreeData merge_tree->loadPrimaryKeys(); } - catch (const Exception & ex) + catch (...) { LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); } diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index f02fef9dcdc..4b700a0c358 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,6 +1,21 @@ -0 0 -0 0 -0 0 -1 a -2 b -3 c +0 0 +0 0 +0 0 + +# After loading primary keys: +10000000 10000000 -- These numbers will vary depending on the system +10000000 10000000 + +# After unloading primary keys: +0 0 +0 0 + +# Data in test_load_primary_key: +1 a +2 b +3 c + +# Data in test_load_primary_key_2: +1 x +2 y +3 z diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 63b4e4111b9..95f98bc7192 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -1,35 +1,50 @@ -- Tags: no-parallel DROP TABLE IF EXISTS test_load_primary_key; +DROP TABLE IF EXISTS test_load_primary_key_2; --- Create the test table +-- Create the test tables CREATE TABLE test_load_primary_key (id Int32, value String) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE test_load_primary_key_2 (id Int32, value String) ENGINE = MergeTree() ORDER BY id; --- Inserting some data +-- Inserting some data into both tables INSERT INTO test_load_primary_key VALUES (1, 'a'), (2, 'b'), (3, 'c'); +INSERT INTO test_load_primary_key_2 VALUES (1, 'x'), (2, 'y'), (3, 'z'); --- Check primary key memory before loading (this checks if it's not loaded yet) -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +-- Check primary key memory before loading (this checks if it's not loaded yet) for both tables +SELECT + table, + round(primary_key_bytes_in_memory, -7), + round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() -AND table = 'test_load_primary_key'; +AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); --- Load primary key +-- Load primary key for both tables SYSTEM LOAD PRIMARY KEY test_load_primary_key; +SYSTEM LOAD PRIMARY KEY test_load_primary_key_2; --- Check primary key memory after loading -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +-- Check primary key memory after loading for both tables +SELECT + table, + round(primary_key_bytes_in_memory, -7), + round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() -AND table = 'test_load_primary_key'; +AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); --- Unload primary key +-- Unload primary key for both tables SYSTEM UNLOAD PRIMARY KEY test_load_primary_key; +SYSTEM UNLOAD PRIMARY KEY test_load_primary_key_2; --- Check primary key memory after unloading -SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) +-- Check primary key memory after unloading for both tables +SELECT + table, + round(primary_key_bytes_in_memory, -7), + round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts WHERE database = currentDatabase() -AND table = 'test_load_primary_key'; +AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); --- Select to verify the data is correctly loaded +-- Select to verify the data is correctly loaded for both tables SELECT * FROM test_load_primary_key ORDER BY id; +SELECT * FROM test_load_primary_key_2 ORDER BY id; From 7ab71ecf9048b00897df8c9ab107d8ce73afc6dd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 12 Oct 2024 15:21:08 +0800 Subject: [PATCH 032/433] enable jit for abs --- src/Functions/FunctionUnaryArithmetic.h | 12 ++++- src/Functions/abs.cpp | 52 ++++++++++++++++--- src/Storages/Hive/StorageHive.cpp | 1 + .../ObjectStorage/HDFS/Configuration.cpp | 1 + 4 files changed, 58 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 259dc1c42ba..8b6b7e327a4 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -523,8 +523,16 @@ public: if constexpr (!std::is_same_v && !IsDataTypeDecimal && Op::compilable) { auto & b = static_cast &>(builder); - auto * v = nativeCast(b, arguments[0], result_type); - result = Op::compile(b, v, is_signed_v); + if constexpr (std::is_same_v, AbsImpl>) + { + /// We don't need to cast the argument to the result type if it's abs function. + result = Op::compile(b, arguments[0].value, is_signed_v); + } + else + { + auto * v = nativeCast(b, arguments[0], result_type); + result = Op::compile(b, v, is_signed_v); + } return true; } diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index 742d3b85619..b251d98da29 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -27,25 +27,65 @@ struct AbsImpl } #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; /// special type handling, some other time + static constexpr bool compilable = true; /// special type handling, some other time + + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool sign) + { + const auto & type = arg->getType(); + if (type->isIntegerTy()) + { + if (sign) + { + auto & context = b.getContext(); + auto * signed_type = arg->getType(); + auto * unsigned_type = llvm::IntegerType::get(context, signed_type->getIntegerBitWidth()); + + auto * is_negative = b.CreateICmpSLT(arg, llvm::ConstantInt::get(signed_type, 0)); + auto * neg_value = b.CreateNeg(arg); + auto * abs_value = b.CreateSelect(is_negative, neg_value, arg); + return b.CreateZExt(abs_value, unsigned_type); + } + else + { + return arg; + } + } + else if (type->isDoubleTy() || type->isFloatTy()) + { + auto * func_fabs = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::fabs, {type}); + return b.CreateCall(func_fabs, {arg}); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "AbsImpl compilation expected native interger or floating point type"); + } #endif }; -struct NameAbs { static constexpr auto name = "abs"; }; +struct NameAbs +{ + static constexpr auto name = "abs"; +}; using FunctionAbs = FunctionUnaryArithmetic; -template <> struct FunctionUnaryArithmeticMonotonicity +template <> +struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } static IFunction::Monotonicity get(const Field & left, const Field & right) { - Float64 left_float = left.isNull() ? -std::numeric_limits::infinity() : applyVisitor(FieldVisitorConvertToNumber(), left); - Float64 right_float = right.isNull() ? std::numeric_limits::infinity() : applyVisitor(FieldVisitorConvertToNumber(), right); + Float64 left_float + = left.isNull() ? -std::numeric_limits::infinity() : applyVisitor(FieldVisitorConvertToNumber(), left); + Float64 right_float + = right.isNull() ? std::numeric_limits::infinity() : applyVisitor(FieldVisitorConvertToNumber(), right); if ((left_float < 0 && right_float > 0) || (left_float > 0 && right_float < 0)) return {}; - return { .is_monotonic = true, .is_positive = std::min(left_float, right_float) >= 0, .is_strict = true, }; + return { + .is_monotonic = true, + .is_positive = std::min(left_float, right_float) >= 0, + .is_strict = true, + }; } }; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 5d312de8e5d..f57b4835308 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 6bee4154b2f..18f71642dbd 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -2,6 +2,7 @@ #if USE_HDFS #include +#include #include #include #include From e5886d4e665b827be29324e61b6b983de36f6f85 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 12 Oct 2024 18:00:34 +0800 Subject: [PATCH 033/433] enable jit for bitCount --- src/Functions/FunctionUnaryArithmetic.h | 8 ++++-- src/Functions/abs.cpp | 2 +- src/Functions/bitCount.cpp | 21 ++++++++++++++- src/Functions/sign.cpp | 35 ++++++++++++++++++++++++- src/Interpreters/ExpressionActions.cpp | 4 +++ 5 files changed, 65 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 8b6b7e327a4..b4f46732cc9 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -497,7 +497,10 @@ public: using T0 = typename DataType::FieldType; using T1 = typename Op::ResultType; if constexpr (!std::is_same_v && !IsDataTypeDecimal && Op::compilable) + { + std::cout << "abs is compilable" << std::endl; return true; + } } return false; @@ -523,9 +526,10 @@ public: if constexpr (!std::is_same_v && !IsDataTypeDecimal && Op::compilable) { auto & b = static_cast &>(builder); - if constexpr (std::is_same_v, AbsImpl>) + if constexpr (std::is_same_v, AbsImpl> || std::is_same_v, BitCountImpl>) { - /// We don't need to cast the argument to the result type if it's abs function. + std::cout << "start to compile abs" << std::endl; + /// We don't need to cast the argument to the result type if it's abs/bitcount function. result = Op::compile(b, arguments[0].value, is_signed_v); } else diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index b251d98da29..32bd09ec220 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -27,7 +27,7 @@ struct AbsImpl } #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = true; /// special type handling, some other time + static constexpr bool compilable = true; static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool sign) { diff --git a/src/Functions/bitCount.cpp b/src/Functions/bitCount.cpp index 68555b1386c..b8b8747693b 100644 --- a/src/Functions/bitCount.cpp +++ b/src/Functions/bitCount.cpp @@ -38,7 +38,26 @@ struct BitCountImpl } #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; + static constexpr bool compilable = true; + + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool) + { + const auto & type = arg->getType(); + llvm::Value * int_value = nullptr; + + if (type->isIntegerTy()) + int_value = arg; + else if (type->isFloatTy()) + int_value = b.CreateBitCast(arg, llvm::Type::getInt32Ty(b.getContext())); + else if (type->isDoubleTy()) + int_value = b.CreateBitCast(arg, llvm::Type::getInt64Ty(b.getContext())); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "BitCountImpl compilation expected native integer or floating-point type"); + + auto * func_ctpop = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::ctpop, {int_value->getType()}); + llvm::Value * ctpop_value = b.CreateCall(func_ctpop, {int_value}); + return b.CreateZExtOrTrunc(ctpop_value, llvm::Type::getInt8Ty(b.getContext())); + } #endif }; diff --git a/src/Functions/sign.cpp b/src/Functions/sign.cpp index 914e1ad9e1f..54db9f5c0b9 100644 --- a/src/Functions/sign.cpp +++ b/src/Functions/sign.cpp @@ -22,7 +22,40 @@ struct SignImpl } #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; + static constexpr bool compilable = true; + + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool) + { + const auto & type = arg->getType(); + if (type->isIntegerTy()) + { + auto * zero = llvm::ConstantInt::get(type, 0); + auto * one = llvm::ConstantInt::get(type, 1); + auto * minus_one = llvm::ConstantInt::getSigned(type, -1); + + auto * is_zero = b.CreateICmpEQ(arg, zero); + auto * is_negative = b.CreateICmpSLT(arg, zero); + + auto * select_zero = b.CreateSelect(is_zero, zero, one); + return b.CreateSelect(is_negative, minus_one, select_zero); + } + else if (type->isDoubleTy() || type->isFloatTy()) + { + auto * zero = llvm::ConstantFP::get(type, 0.0); + auto * one = llvm::ConstantFP::get(type, 1.0); + auto * minus_one = llvm::ConstantFP::get(type, -1.0); + + auto * is_zero = b.CreateFCmpOEQ(arg, zero); + auto * is_negative = b.CreateFCmpOLT(arg, zero); + + auto * select_zero = b.CreateSelect(is_zero, zero, one); + return b.CreateSelect(is_negative, minus_one, select_zero); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "SignImpl compilation expected native integer or floating point type"); + } + + #endif }; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index edf419d404e..912a1863f8e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -59,7 +59,11 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) + { + std::cout << "old actions_dag: " << actions_dag.dumpDAG() << std::endl; actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); + std::cout << "new actions_dag: " << actions_dag.dumpDAG() << std::endl; + } #endif linearizeActions(lazy_executed_nodes); From ae3b3feef3c135e46deadb8fe01c5fd1306c300f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 13 Oct 2024 08:58:49 +0800 Subject: [PATCH 034/433] support sign compilation --- src/Functions/sign.cpp | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Functions/sign.cpp b/src/Functions/sign.cpp index 54db9f5c0b9..36ab5bb9677 100644 --- a/src/Functions/sign.cpp +++ b/src/Functions/sign.cpp @@ -24,32 +24,36 @@ struct SignImpl #if USE_EMBEDDED_COMPILER static constexpr bool compilable = true; - static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool) + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * arg, bool sign) { + auto * result_type = b.getInt8Ty(); + auto * res_zero = llvm::ConstantInt::getSigned(result_type, 0); + auto * res_one = llvm::ConstantInt::getSigned(result_type, 1); + auto * res_minus_one = llvm::ConstantInt::getSigned(result_type, -1); + const auto & type = arg->getType(); if (type->isIntegerTy()) { - auto * zero = llvm::ConstantInt::get(type, 0); - auto * one = llvm::ConstantInt::get(type, 1); - auto * minus_one = llvm::ConstantInt::getSigned(type, -1); - + auto * zero = llvm::ConstantInt::get(type, 0, sign); auto * is_zero = b.CreateICmpEQ(arg, zero); - auto * is_negative = b.CreateICmpSLT(arg, zero); - auto * select_zero = b.CreateSelect(is_zero, zero, one); - return b.CreateSelect(is_negative, minus_one, select_zero); + if (sign) + { + auto * is_negative = b.CreateICmpSLT(arg, res_zero); + auto * select_zero = b.CreateSelect(is_zero, res_zero, res_one); + return b.CreateSelect(is_negative, res_minus_one, select_zero); + } + else + return b.CreateSelect(is_zero, res_zero, res_one); } else if (type->isDoubleTy() || type->isFloatTy()) { auto * zero = llvm::ConstantFP::get(type, 0.0); - auto * one = llvm::ConstantFP::get(type, 1.0); - auto * minus_one = llvm::ConstantFP::get(type, -1.0); - auto * is_zero = b.CreateFCmpOEQ(arg, zero); auto * is_negative = b.CreateFCmpOLT(arg, zero); - auto * select_zero = b.CreateSelect(is_zero, zero, one); - return b.CreateSelect(is_negative, minus_one, select_zero); + auto * select_zero = b.CreateSelect(is_zero, res_zero, res_one); + return b.CreateSelect(is_negative, res_minus_one, select_zero); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "SignImpl compilation expected native integer or floating point type"); From 98d9077135b1172a2d605acf3575c62c3475445e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 24 Aug 2023 13:23:34 +0800 Subject: [PATCH 035/433] jit for compare functions --- src/Core/AccurateComparison.h | 6 ++ src/Functions/FunctionsComparison.h | 133 ++++++++++++++++++++++++---- 2 files changed, 120 insertions(+), 19 deletions(-) diff --git a/src/Core/AccurateComparison.h b/src/Core/AccurateComparison.h index 139ee4d88dc..83c50cc0ae7 100644 --- a/src/Core/AccurateComparison.h +++ b/src/Core/AccurateComparison.h @@ -209,12 +209,14 @@ template struct EqualsOp using SymmetricOp = EqualsOp; static UInt8 apply(A a, B b) { return accurate::equalsOp(a, b); } + static constexpr bool compilable = true; }; template struct NotEqualsOp { using SymmetricOp = NotEqualsOp; static UInt8 apply(A a, B b) { return accurate::notEqualsOp(a, b); } + static constexpr bool compilable = true; }; template struct GreaterOp; @@ -223,12 +225,14 @@ template struct LessOp { using SymmetricOp = GreaterOp; static UInt8 apply(A a, B b) { return accurate::lessOp(a, b); } + static constexpr bool compilable = true; }; template struct GreaterOp { using SymmetricOp = LessOp; static UInt8 apply(A a, B b) { return accurate::greaterOp(a, b); } + static constexpr bool compilable = true; }; template struct GreaterOrEqualsOp; @@ -237,12 +241,14 @@ template struct LessOrEqualsOp { using SymmetricOp = GreaterOrEqualsOp; static UInt8 apply(A a, B b) { return accurate::lessOrEqualsOp(a, b); } + static constexpr bool compilable = true; }; template struct GreaterOrEqualsOp { using SymmetricOp = LessOrEqualsOp; static UInt8 apply(A a, B b) { return accurate::greaterOrEqualsOp(a, b); } + static constexpr bool compilable = true; }; } diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index d84f6fe84ac..80b95e940ea 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1,17 +1,21 @@ #pragma once -#include -#include -#include +// Include this first, because `#define _asan_poison_address` from +// llvm/Support/Compiler.h conflicts with its forward declaration in +// sanitizer/asan_interface.h +#include +#include +#include -#include +#include #include #include -#include #include +#include #include -#include - +#include +#include +#include #include #include #include @@ -24,25 +28,20 @@ #include #include #include - -#include -#include - -#include #include +#include #include - -#include -#include - #include #include - -#include -#include +#include +#include +#include +#include +#include #if USE_EMBEDDED_COMPILER # include +# include # include #endif @@ -1367,6 +1366,102 @@ public: return executeGeneric(col_with_type_and_name_left, col_with_type_and_name_right); } + +#if USE_EMBEDDED_COMPILER + template + static bool castType(const IDataType * type, F && f) + { + return castTypeToEither< + DataTypeUInt8, + DataTypeUInt16, + DataTypeUInt32, + DataTypeUInt64, + DataTypeInt8, + DataTypeInt16, + DataTypeInt32, + DataTypeInt64, + DataTypeFloat32, + DataTypeFloat64>(type, std::forward(f)); + } + + template + static bool castBothTypes(const IDataType * left, const IDataType * right, F && f) + { + return castType(left, [&](const auto & left_) + { + return castType(right, [&](const auto & right_) + { + return f(left_, right_); + }); + }); + } + + bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr & result_type) const override + { + if (2 != arguments.size()) + return false; + + if (!canBeNativeType(*arguments[0]) || !canBeNativeType(*arguments[1]) || !canBeNativeType(*result_type)) + return false; + + WhichDataType data_type_lhs(arguments[0]); + WhichDataType data_type_rhs(arguments[1]); + if ((data_type_lhs.isDateOrDate32() || data_type_lhs.isDateTime()) || + (data_type_rhs.isDateOrDate32() || data_type_rhs.isDateTime())) + return false; + + return castBothTypes(arguments[0].get(), arguments[1].get(), [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + if constexpr (!std::is_same_v && + !std::is_same_v && + !std::is_same_v && + !std::is_same_v) + { + using OpSpec = Op; + return OpSpec::compilable && std::is_same_v; + } + return false; + }); + } + + llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr &) const override + { + assert(2 == arguments.size()); + + llvm::Value * result = nullptr; + castBothTypes(arguments[0].type.get(), arguments[1].type.get(), [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + if constexpr (!std::is_same_v && + !std::is_same_v && + !std::is_same_v && + !std::is_same_v) + { + using OpSpec = Op; + if constexpr (OpSpec::compilable && std::is_same_v) + { + auto & b = static_cast &>(builder); + // auto * lval = nativeCast(b, arguments[0], result_type); + // auto * rval = nativeCast(b, arguments[1], result_type); + // result = OpSpec::compile(b, lval, rval, std::is_signed_v); + result = b.CreateSelect( + CompileOp::compile( + b, arguments[0].value, arguments[1].value, std::is_signed_v), + b.getInt8(1), + b.getInt8(0)); + return true; + } + } + return false; + }); + + return result; + } +#endif + }; } From 61de692f2e3fcd98ae900672426fc4f46d450548 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 13 Oct 2024 11:41:10 +0800 Subject: [PATCH 036/433] enable jit for comparision functions --- src/Functions/FunctionsComparison.h | 38 +++++++++++++++++------------ 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 80b95e940ea..a5a81112ba3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -38,6 +38,7 @@ #include #include #include +#include "DataTypes/NumberTraits.h" #if USE_EMBEDDED_COMPILER # include @@ -1406,6 +1407,7 @@ public: WhichDataType data_type_lhs(arguments[0]); WhichDataType data_type_rhs(arguments[1]); + /// TODO support date/date32 if ((data_type_lhs.isDateOrDate32() || data_type_lhs.isDateTime()) || (data_type_rhs.isDateOrDate32() || data_type_rhs.isDateTime())) return false; @@ -1414,13 +1416,16 @@ public: { using LeftDataType = std::decay_t; using RightDataType = std::decay_t; - if constexpr (!std::is_same_v && - !std::is_same_v && - !std::is_same_v && - !std::is_same_v) + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = typename NumberTraits::ResultOfIf::Type; + if constexpr ( + !std::is_same_v && !std::is_same_v + && !std::is_same_v && !std::is_same_v + && (std::is_integral_v || std::is_floating_point_v)) { using OpSpec = Op; - return OpSpec::compilable && std::is_same_v; + return OpSpec::compilable; } return false; }); @@ -1435,23 +1440,24 @@ public: { using LeftDataType = std::decay_t; using RightDataType = std::decay_t; - if constexpr (!std::is_same_v && - !std::is_same_v && - !std::is_same_v && - !std::is_same_v) + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = typename NumberTraits::ResultOfIf::Type; + + if constexpr ( + !std::is_same_v && !std::is_same_v + && !std::is_same_v && !std::is_same_v + && (std::is_integral_v || std::is_floating_point_v)) { using OpSpec = Op; if constexpr (OpSpec::compilable && std::is_same_v) { + auto promoted_type = std::make_shared>(); auto & b = static_cast &>(builder); - // auto * lval = nativeCast(b, arguments[0], result_type); - // auto * rval = nativeCast(b, arguments[1], result_type); - // result = OpSpec::compile(b, lval, rval, std::is_signed_v); + auto * left_value = nativeCast(b, arguments[0], promoted_type); + auto * right_value = nativeCast(b, arguments[1], promoted_type); result = b.CreateSelect( - CompileOp::compile( - b, arguments[0].value, arguments[1].value, std::is_signed_v), - b.getInt8(1), - b.getInt8(0)); + CompileOp::compile(b, left_value, right_value, std::is_signed_v), b.getInt8(1), b.getInt8(0)); return true; } } From f4071806319c0cf9ad239aafcb655c3d01a65203 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 13 Oct 2024 11:42:00 +0800 Subject: [PATCH 037/433] enable jit for isnotnull --- src/Functions/isNotNull.cpp | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index 4af2ba8b987..f648809e4ff 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -11,6 +11,12 @@ #include #include + +#if USE_EMBEDDED_COMPILER +# include +# include +#endif + namespace DB { namespace Setting @@ -110,6 +116,24 @@ public: return DataTypeUInt8().createColumnConst(elem.column->size(), 1u); } +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override + { + auto & b = static_cast &>(builder); + if (arguments[0].type->isNullable()) + { + auto * is_null = b.CreateExtractValue(arguments[0].value, {1}); + return b.CreateNot(is_null); + } + else + return b.getInt8(true); + } +#endif + + private: MULTITARGET_FUNCTION_AVX2_SSE42( MULTITARGET_FUNCTION_HEADER(static void NO_INLINE), vectorImpl, MULTITARGET_FUNCTION_BODY((const PaddedPODArray & null_map, PaddedPODArray & res) /// NOLINT From 6065de3073b10bcd41886ee5fddbabd308453576 Mon Sep 17 00:00:00 2001 From: zhangwanyun1 <643044969@qq.com> Date: Sun, 13 Oct 2024 17:52:35 +0800 Subject: [PATCH 038/433] Add setting check_table_structure_completely --- .../statements/alter/partition.md | 6 +- src/Core/SettingsChangesHistory.cpp | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 8 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../__init__.py | 0 ..._with_check_table_structure_completely.xml | 5 + ...thout_check_table_structure_completely.xml | 5 + .../test.py | 420 ++++++++++++++++++ 8 files changed, 440 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_attach_with_different_projections_or_indices/__init__.py create mode 100644 tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml create mode 100644 tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml create mode 100644 tests/integration/test_attach_with_different_projections_or_indices/test.py diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 11926b2aa08..cbf83499a0f 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -131,8 +131,8 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key, the same order by key and the same primary key. -- Both tables must have the same indices and projections. - Both tables must have the same storage policy. +- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). ## REPLACE PARTITION @@ -151,8 +151,8 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key, the same order by key and the same primary key. -- Both tables must have the same indices and projections. - Both tables must have the same storage policy. +- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). ## MOVE PARTITION TO TABLE @@ -166,9 +166,9 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key, the same order by key and the same primary key. -- Both tables must have the same indices and projections. - Both tables must have the same storage policy. - Both tables must be the same engine family (replicated or non-replicated). +- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). ## CLEAR COLUMN IN PARTITION diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 42c92481fce..34e3e9ec3f4 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -68,6 +68,7 @@ static std::initializer_listgetPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); - - const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions) + const auto check_definitions = [this](const auto & my_descriptions, const auto & src_descriptions) { - if (my_descriptions.size() != src_descriptions.size()) + if (((*getSettings())[MergeTreeSetting::check_table_structure_completely] && my_descriptions.size() != src_descriptions.size()) + || (!(*getSettings())[MergeTreeSetting::check_table_structure_completely] && my_descriptions.size() < src_descriptions.size())) return false; std::unordered_set my_query_strings; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 77cff4ca527..cc4cd5a32ea 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -92,7 +92,7 @@ namespace ErrorCodes M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - \ + M(Bool, check_table_structure_completely, false, "If true, tables must have identical definitions, including projections and secondary indices. Otherwise, the source table's projections and secondary indices must be a subset of those in the target table.", 0) \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/tests/integration/test_attach_with_different_projections_or_indices/__init__.py b/tests/integration/test_attach_with_different_projections_or_indices/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml new file mode 100644 index 00000000000..dd9ae5e83d5 --- /dev/null +++ b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml @@ -0,0 +1,5 @@ + + + true + + diff --git a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml new file mode 100644 index 00000000000..734294478ba --- /dev/null +++ b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml @@ -0,0 +1,5 @@ + + + false + + \ No newline at end of file diff --git a/tests/integration/test_attach_with_different_projections_or_indices/test.py b/tests/integration/test_attach_with_different_projections_or_indices/test.py new file mode 100644 index 00000000000..bff993e608d --- /dev/null +++ b/tests/integration/test_attach_with_different_projections_or_indices/test.py @@ -0,0 +1,420 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/config_with_check_table_structure_completely.xml"] +) +# node1 = cluster.add_instance("node1") +node2 = cluster.add_instance( + "node2", main_configs=["configs/config_without_check_table_structure_completely.xml"] +) +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() +# def test_setting_check_table_structure_completely(start_cluster): +# assert node1.query("""select value from system.merge_tree_settings where name='check_table_structure_completely';""") == "0\n" +def test_check_completely_attach_with_different_indices(start_cluster): + node1.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node1.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node1.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1, + INDEX cf c TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node1.query("DROP TABLE attach_partition_t1") + node1.query("DROP TABLE attach_partition_t2") + node1.query("DROP TABLE attach_partition_t3") +def test_check_attach_with_different_indices(start_cluster): + node2.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node2.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node2.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + `c` String, + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1, + INDEX cf c TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" + assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `b` = '1'") == "1\n" + assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `c` = '1'") == "1\n" + node2.query("DROP TABLE attach_partition_t1") + node2.query("DROP TABLE attach_partition_t2") + node2.query("DROP TABLE attach_partition_t3") +def test_check_completely_attach_with_different_projections(start_cluster): + node1.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);") + node1.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + PROJECTION differently_named_proj ( + SELECT + b, + sum(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a; + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different projections" in str(exc.value) + node1.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + PROJECTION proj2 ( + SELECT + b, + avg(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different projections" in str(exc.value) + node1.query("DROP TABLE attach_partition_t1") + node1.query("DROP TABLE attach_partition_t2") + node1.query("DROP TABLE attach_partition_t3") +def test_check_attach_with_different_projections(start_cluster): + node2.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);") + node2.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + PROJECTION differently_named_proj ( + SELECT + b, + sum(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a; + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different projections" in str(exc.value) + node2.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + PROJECTION proj2 ( + SELECT + b, + avg(a) + GROUP BY b + ) + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" + node2.query("DROP TABLE attach_partition_t1") + node2.query("DROP TABLE attach_partition_t2") + node2.query("DROP TABLE attach_partition_t3") +def test_check_completely_attach_with_different_indices_and_projections(start_cluster): + node1.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node1.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj ( + SELECT + b, + sum(a) + GROUP BY b + ), + INDEX bf b TYPE bloom_filter GRANULARITY 1, + INDEX cf c TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node1.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + PROJECTION proj2 ( + SELECT + b, + avg(a) + GROUP BY b + ), + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1, + INDEX cf c TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node1.query("DROP TABLE attach_partition_t1") + node1.query("DROP TABLE attach_partition_t2") + node1.query("DROP TABLE attach_partition_t3") +def test_check_attach_with_different_indices_and_projections(start_cluster): + node2.query( + """ + CREATE TABLE attach_partition_t1 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node2.query( + """ + CREATE TABLE attach_partition_t2 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj ( + SELECT + b, + sum(a) + GROUP BY b + ), + INDEX bf b TYPE bloom_filter GRANULARITY 1, + INDEX cf c TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + # serverError 36 + with pytest.raises(QueryRuntimeException) as exc: + node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert "Tables have different secondary indices" in str(exc.value) + node2.query( + """ + CREATE TABLE attach_partition_t3 + ( + `a` UInt32, + `b` String, + `c` String, + PROJECTION proj1 ( + SELECT + b, + sum(a) + GROUP BY b + ), + PROJECTION proj2 ( + SELECT + b, + avg(a) + GROUP BY b + ), + INDEX bf b TYPE tokenbf_v1(8192, 3, 0) GRANULARITY 1, + INDEX cf c TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY a + """ + ) + node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" + assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `b` = '1'") == "1\n" + assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `c` = '1'") == "1\n" + node2.query("DROP TABLE attach_partition_t1") + node2.query("DROP TABLE attach_partition_t2") + node2.query("DROP TABLE attach_partition_t3") From f28836ad7c530c27633520f033aaaffba03d34e1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 13 Oct 2024 18:09:23 +0800 Subject: [PATCH 039/433] enable jit for assumenotnull --- src/Functions/assumeNotNull.cpp | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/src/Functions/assumeNotNull.cpp b/src/Functions/assumeNotNull.cpp index 991fe2296d5..04f158c1d2a 100644 --- a/src/Functions/assumeNotNull.cpp +++ b/src/Functions/assumeNotNull.cpp @@ -4,6 +4,11 @@ #include #include +#if USE_EMBEDDED_COMPILER +# include +# include +#endif + namespace DB { @@ -58,6 +63,22 @@ public: return nullable_col->getNestedColumnPtr(); return col; } + +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override + { + auto & b = static_cast &>(builder); + if (arguments[0].type->isNullable()) + return b.CreateExtractValue(arguments[0].value, {0}); + else + return arguments[0].value; + } +#endif + + }; } From fd43dcbcf63e994e97f93c2d21de11e2412b9569 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 10:22:43 +0800 Subject: [PATCH 040/433] enable jit for and/or/xor --- src/Functions/FunctionsLogical.cpp | 38 ++++++++++++ src/Functions/FunctionsLogical.h | 96 ++++++++++++++++++++---------- 2 files changed, 103 insertions(+), 31 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index f83a958bf31..bd6f078c825 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -510,6 +510,44 @@ ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count) } +namespace FunctionsLogicalDetail +{ + +#if USE_EMBEDDED_COMPILER + +/// Cast LLVM value with type to Tenary +llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value) +{ + auto * result_type = llvm::Type::getInt8Ty(b.getContext()); + + if (from_type->isNullable()) + { + auto * tenary_null = llvm::ConstantInt::get(result_type, 1); + auto * inner = nativeTenaryCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0})); + auto * is_null = b.CreateExtractValue(value, {1}); + return b.CreateSelect(b.CreateICmpNE(is_null, llvm::Constant::getNullValue(is_null->getType())), tenary_null, inner); + } + + auto * zero = llvm::Constant::getNullValue(value->getType()); + auto * tenary_true = llvm::ConstantInt::get(result_type, 2); + auto * tenary_false = llvm::ConstantInt::get(result_type, 0); + if (value->getType()->isIntegerTy()) + return b.CreateSelect(b.CreateICmpEQ(value, zero), tenary_true, tenary_false); + if (value->getType()->isFloatingPointTy()) + return b.CreateSelect(b.CreateFCmpOEQ(value, zero), tenary_true, tenary_false); + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to tenary", from_type->getName()); +} + +/// Cast LLVM value with type to Tenary +llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type) +{ + return nativeTenaryCast(b, value_with_type.type, value_with_type.value); +} + +#endif +} + template DataTypePtr FunctionAnyArityLogical::getReturnTypeImpl(const DataTypes & arguments) const { diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 3c2eb3ee0b8..49209ee8c44 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -79,6 +79,15 @@ namespace Ternary } } +#if USE_EMBEDDED_COMPILER + +/// Cast LLVM value with type to Tenary +llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value); + +/// Cast LLVM value with type to Tenary +llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type); + +#endif struct AndImpl { @@ -98,6 +107,18 @@ struct AndImpl /// Will use three-valued logic for NULLs (see above) or default implementation (any operation with NULL returns NULL). static constexpr bool specialImplementationForNulls() { return true; } + +#if USE_EMBEDDED_COMPILER + static llvm::Value * apply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + { + return builder.CreateAnd(a, b); + } + + static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + { + return builder.CreateSelect(builder.CreateICmpUGT(a, b), b, a); + } +#endif }; struct OrImpl @@ -110,6 +131,19 @@ struct OrImpl static constexpr ResultType apply(UInt8 a, UInt8 b) { return a | b; } static constexpr ResultType ternaryApply(UInt8 a, UInt8 b) { return std::max(a, b); } static constexpr bool specialImplementationForNulls() { return true; } + +#if USE_EMBEDDED_COMPILER + static llvm::Value * apply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + { + return builder.CreateOr(a, b); + } + + static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + { + return builder.CreateSelect(builder.CreateICmpUGT(a, b), a, b); + } +#endif + }; struct XorImpl @@ -128,6 +162,11 @@ struct XorImpl { return builder.CreateXor(a, b); } + + static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + { + return builder.CreateICmpNE(a, b); + } #endif }; @@ -184,47 +223,42 @@ public: ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override; #if USE_EMBEDDED_COMPILER - bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return useDefaultImplementationForNulls(); } + bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override + { + for (const auto & arg : arguments) + { + if (!canBeNativeType(arg)) + return false; + } + return true; + } - llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr &) const override + llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & values, const DataTypePtr & result_type) const override { assert(!values.empty()); auto & b = static_cast &>(builder); - if constexpr (!Impl::isSaturable()) + if (useDefaultImplementationForNulls() || !result_type->isNullable()) { - auto * result = nativeBoolCast(b, values[0]); + llvm::Value * result = nativeBoolCast(b, values[0]); for (size_t i = 1; i < values.size(); ++i) - result = Impl::apply(b, result, nativeBoolCast(b, values[i])); - return b.CreateSelect(result, b.getInt8(1), b.getInt8(0)); - } - - constexpr bool break_on_true = Impl::isSaturatedValue(true); - auto * next = b.GetInsertBlock(); - auto * stop = llvm::BasicBlock::Create(next->getContext(), "", next->getParent()); - b.SetInsertPoint(stop); - - auto * phi = b.CreatePHI(b.getInt8Ty(), static_cast(values.size())); - - for (size_t i = 0; i < values.size(); ++i) - { - b.SetInsertPoint(next); - auto * value = values[i].value; - auto * truth = nativeBoolCast(b, values[i]); - if (!values[i].type->equals(DataTypeUInt8{})) - value = b.CreateSelect(truth, b.getInt8(1), b.getInt8(0)); - phi->addIncoming(value, b.GetInsertBlock()); - if (i + 1 < values.size()) { - next = llvm::BasicBlock::Create(next->getContext(), "", next->getParent()); - b.CreateCondBr(truth, break_on_true ? stop : next, break_on_true ? next : stop); + llvm::Value * casted_value = nativeBoolCast(b, values[i]); + result = Impl::apply(b, result, casted_value); } + return result; + } + else + { + /// First we need to cast all values to ternary logic + llvm::Value * result = nativeTenaryCast(b, values[0]); + for (size_t i = 1; i < values.size(); ++i) + { + llvm::Value * casted_value = nativeTenaryCast(b, values[i]); + result = Impl::tenaryApply(b, result, casted_value); + } + return result; } - - b.CreateBr(stop); - b.SetInsertPoint(stop); - - return phi; } #endif }; From a8e905b6284f910dea44147fec37dd90c650a8c3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 11:16:44 +0800 Subject: [PATCH 041/433] fix style --- src/Functions/FunctionUnaryArithmetic.h | 1 - src/Functions/FunctionsLogical.cpp | 1 + src/Functions/abs.cpp | 5 +++++ src/Functions/bitCount.cpp | 4 ++++ src/Functions/sign.cpp | 6 ++++++ src/Interpreters/ExpressionActions.cpp | 6 ++++-- src/Interpreters/ExpressionActions.h | 1 + 7 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index b4f46732cc9..c483463eb26 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -498,7 +498,6 @@ public: using T1 = typename Op::ResultType; if constexpr (!std::is_same_v && !IsDataTypeDecimal && Op::compilable) { - std::cout << "abs is compilable" << std::endl; return true; } } diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index bd6f078c825..3a6a811d8e2 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; } namespace diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index 32bd09ec220..e11152945bd 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + template struct AbsImpl { diff --git a/src/Functions/bitCount.cpp b/src/Functions/bitCount.cpp index b8b8747693b..ac240d7d4e1 100644 --- a/src/Functions/bitCount.cpp +++ b/src/Functions/bitCount.cpp @@ -7,6 +7,10 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} template struct BitCountImpl { diff --git a/src/Functions/sign.cpp b/src/Functions/sign.cpp index 36ab5bb9677..4fc1b606e64 100644 --- a/src/Functions/sign.cpp +++ b/src/Functions/sign.cpp @@ -5,6 +5,12 @@ namespace DB { + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + template struct SignImpl { diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 912a1863f8e..91ed36a2eca 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -53,6 +54,7 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc : actions_dag(std::move(actions_dag_)) , project_inputs(project_inputs_) , settings(settings_) + , log(getLogger("ExpressionActions")) { /// It's important to determine lazy executed nodes before compiling expressions. std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation); @@ -60,9 +62,9 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) { - std::cout << "old actions_dag: " << actions_dag.dumpDAG() << std::endl; + LOG_TRACE(log, "Actions before compilation: {} with {} lazy_executed_nodes", actions_dag.dumpDAG(), lazy_executed_nodes.size()); actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); - std::cout << "new actions_dag: " << actions_dag.dumpDAG() << std::endl; + LOG_TRACE(log, "Actions after compilation: {}", actions_dag.dumpDAG()); } #endif diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 539c7c8d141..b7d58adf0e8 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -80,6 +80,7 @@ private: bool project_inputs = false; ExpressionActionsSettings settings; + const LoggerPtr log; public: explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); From 3405fb911cd53e814b3a48cf767165f831c6c440 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 11:19:07 +0800 Subject: [PATCH 042/433] fix style --- src/Functions/FunctionUnaryArithmetic.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index c483463eb26..729f3d0ce87 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -527,7 +527,6 @@ public: auto & b = static_cast &>(builder); if constexpr (std::is_same_v, AbsImpl> || std::is_same_v, BitCountImpl>) { - std::cout << "start to compile abs" << std::endl; /// We don't need to cast the argument to the result type if it's abs/bitcount function. result = Op::compile(b, arguments[0].value, is_signed_v); } From 5349511d3b0354aa888f2def10d1d4d026dcdb08 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 11:28:14 +0800 Subject: [PATCH 043/433] fix typo --- src/Functions/abs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index e11152945bd..8293efa8146 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -61,7 +61,7 @@ struct AbsImpl return b.CreateCall(func_fabs, {arg}); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "AbsImpl compilation expected native interger or floating point type"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "AbsImpl compilation expected native integer or floating point type"); } #endif }; From 782a13b375ca1382a686ffa1d22bfda9f8db139e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 12:08:49 +0800 Subject: [PATCH 044/433] fix building --- src/Interpreters/ExpressionActions.cpp | 9 ++++++--- src/Interpreters/ExpressionActions.h | 1 - 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 91ed36a2eca..a7810063dec 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -54,7 +54,6 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc : actions_dag(std::move(actions_dag_)) , project_inputs(project_inputs_) , settings(settings_) - , log(getLogger("ExpressionActions")) { /// It's important to determine lazy executed nodes before compiling expressions. std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation); @@ -62,9 +61,13 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) { - LOG_TRACE(log, "Actions before compilation: {} with {} lazy_executed_nodes", actions_dag.dumpDAG(), lazy_executed_nodes.size()); + LOG_TRACE( + getLogger("ExpressionActions"), + "Actions before compilation: {} with {} lazy_executed_nodes", + actions_dag.dumpDAG(), + lazy_executed_nodes.size()); actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); - LOG_TRACE(log, "Actions after compilation: {}", actions_dag.dumpDAG()); + LOG_TRACE(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG()); } #endif diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index b7d58adf0e8..539c7c8d141 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -80,7 +80,6 @@ private: bool project_inputs = false; ExpressionActionsSettings settings; - const LoggerPtr log; public: explicit ExpressionActions(ActionsDAG actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); From 458872ba18e2f6c853b81dd1599ddbdf66f4fc49 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 15:49:45 +0800 Subject: [PATCH 045/433] fix bugs --- src/Functions/FunctionsComparison.h | 4 +++- src/Functions/FunctionsLogical.h | 21 +++++++++++++++------ src/Interpreters/ExpressionActions.cpp | 4 ++-- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index a5a81112ba3..361ad9874b3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1397,6 +1397,7 @@ public: }); } + // bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return false; } bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr & result_type) const override { if (2 != arguments.size()) @@ -1429,6 +1430,7 @@ public: } return false; }); + return false; } llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr &) const override @@ -1450,7 +1452,7 @@ public: && (std::is_integral_v || std::is_floating_point_v)) { using OpSpec = Op; - if constexpr (OpSpec::compilable && std::is_same_v) + if constexpr (OpSpec::compilable) { auto promoted_type = std::make_shared>(); auto & b = static_cast &>(builder); diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 49209ee8c44..297b436bd83 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -6,7 +6,6 @@ #include #include #include -#include #include @@ -165,7 +164,8 @@ struct XorImpl static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) { - return builder.CreateICmpNE(a, b); + llvm::Value * xor_result = builder.CreateXor(a, b); + return builder.CreateSelect(xor_result, builder.getInt8(Ternary::True), builder.getInt8(Ternary::False)); } #endif }; @@ -246,18 +246,27 @@ public: llvm::Value * casted_value = nativeBoolCast(b, values[i]); result = Impl::apply(b, result, casted_value); } - return result; + return b.CreateSelect(result, b.getInt8(1), b.getInt8(0)); } else { /// First we need to cast all values to ternary logic - llvm::Value * result = nativeTenaryCast(b, values[0]); + llvm::Value * tenary_result = nativeTenaryCast(b, values[0]); for (size_t i = 1; i < values.size(); ++i) { llvm::Value * casted_value = nativeTenaryCast(b, values[i]); - result = Impl::tenaryApply(b, result, casted_value); + tenary_result = Impl::tenaryApply(b, tenary_result, casted_value); } - return result; + + /// Then transform ternary logic to struct which represents nullable result + llvm::Value * is_null = b.CreateICmpEQ(tenary_result, b.getInt8(Ternary::Null)); + llvm::Value * is_true = b.CreateICmpEQ(tenary_result, b.getInt8(Ternary::True)); + + auto * nullable_result_type = toNativeType(b, result_type); + auto * nullable_result = llvm::Constant::getNullValue(nullable_result_type); + auto * nullable_result_with_value + = b.CreateInsertValue(nullable_result, b.CreateSelect(is_true, b.getInt8(1), b.getInt8(0)), {0}); + return b.CreateInsertValue(nullable_result_with_value, b.CreateSelect(is_null, b.getInt8(1), b.getInt8(0)), {1}); } } #endif diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index a7810063dec..f682ebd75c0 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -61,13 +61,13 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) { - LOG_TRACE( + LOG_DEBUG( getLogger("ExpressionActions"), "Actions before compilation: {} with {} lazy_executed_nodes", actions_dag.dumpDAG(), lazy_executed_nodes.size()); actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); - LOG_TRACE(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG()); + LOG_DEBUG(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG()); } #endif From 47971c9d7a8be1ca43066fcb3897d861baa40ad7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Oct 2024 18:11:10 +0800 Subject: [PATCH 046/433] turn on expression compilation and set min_count_to_compile_expression to 1 --- src/Core/Settings.cpp | 4 ++-- src/Functions/isNotNull.cpp | 4 +--- src/Functions/isNull.cpp | 20 ++++++++++++++++++++ 3 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 091a1845107..95a5470c1b9 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -861,10 +861,10 @@ In CREATE TABLE statement allows specifying Variant type with similar variant ty M(Bool, allow_suspicious_primary_key, false, R"( Allow suspicious `PRIMARY KEY`/`ORDER BY` for MergeTree (i.e. SimpleAggregateFunction). )", 0) \ - M(Bool, compile_expressions, false, R"( + M(Bool, compile_expressions, true, R"( Compile some scalar functions and operators to native code. Due to a bug in the LLVM compiler infrastructure, on AArch64 machines, it is known to lead to a nullptr dereference and, consequently, server crash. Do not enable this setting. )", 0) \ - M(UInt64, min_count_to_compile_expression, 3, R"( + M(UInt64, min_count_to_compile_expression, 1, R"( Minimum count of executing same expression before it is get compiled. )", 0) \ M(Bool, compile_aggregate_expressions, true, R"( diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index f648809e4ff..6ac47ebf91b 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -11,7 +11,6 @@ #include #include - #if USE_EMBEDDED_COMPILER # include # include @@ -129,11 +128,10 @@ public: return b.CreateNot(is_null); } else - return b.getInt8(true); + return b.getInt8(1); } #endif - private: MULTITARGET_FUNCTION_AVX2_SSE42( MULTITARGET_FUNCTION_HEADER(static void NO_INLINE), vectorImpl, MULTITARGET_FUNCTION_BODY((const PaddedPODArray & null_map, PaddedPODArray & res) /// NOLINT diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index f66c5c0f7ab..817f8b0dd1b 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -10,6 +10,11 @@ #include #include +#if USE_EMBEDDED_COMPILER +# include +# include +#endif + namespace DB { @@ -107,6 +112,21 @@ public: return DataTypeUInt8().createColumnConst(elem.column->size(), 0u); } +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr &) const override { return canBeNativeType(arguments[0]); } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override + { + auto & b = static_cast &>(builder); + if (arguments[0].type->isNullable()) + return b.CreateExtractValue(arguments[0].value, {1}); + else + return b.getInt8(0); + } +#endif + + private: bool use_analyzer; }; From 664f565cee7eb0111d4978577ecb9646a21fafdc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Oct 2024 09:03:59 +0800 Subject: [PATCH 047/433] support jit for identity --- src/Functions/identity.h | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/src/Functions/identity.h b/src/Functions/identity.h index 01af9c98368..9d799bbb2bb 100644 --- a/src/Functions/identity.h +++ b/src/Functions/identity.h @@ -2,6 +2,10 @@ #include #include +#if USE_EMBEDDED_COMPILER +# include +# include +#endif namespace DB { @@ -11,6 +15,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +struct IdentityName +{ + static constexpr auto name = "identity"; +}; + template class FunctionIdentityBase : public IFunction { @@ -32,12 +41,21 @@ public: { return arguments.front().column; } + +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & /*types*/, const DataTypePtr & result_type) const override + { + return Name::name == IdentityName::name && canBeNativeType(result_type); + } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & /*builder*/, const ValuesWithType & arguments, const DataTypePtr & /*result_type*/) const override + { + return arguments[0].value; + } +#endif }; -struct IdentityName -{ - static constexpr auto name = "identity"; -}; struct ScalarSubqueryResultName { From 7fd58a78dffef53800a072b8f6d084e7e7dd075b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Oct 2024 09:09:23 +0800 Subject: [PATCH 048/433] fix incompatiable history --- src/Core/SettingsChangesHistory.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d3814d6d671..5f6e3c8b3ed 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -94,6 +94,8 @@ static std::initializer_list Date: Tue, 15 Oct 2024 18:49:25 +0800 Subject: [PATCH 049/433] enable jit for modulo --- src/Functions/DivisionUtils.h | 20 +++++++++++++- src/Functions/FunctionBinaryArithmetic.h | 34 ++++++++++++++++++++---- 2 files changed, 48 insertions(+), 6 deletions(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 7fd5b7476e1..f5767b01bc4 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -8,6 +8,10 @@ #include "config.h" +#if USE_EMBEDDED_COMPILER +# include +#endif + namespace DB { @@ -15,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_DIVISION; + extern const int LOGICAL_ERROR; } template @@ -158,7 +163,20 @@ struct ModuloImpl } #if USE_EMBEDDED_COMPILER - static constexpr bool compilable = false; /// don't know how to throw from LLVM IR + static constexpr bool compilable = true; /// Ignore exceptions in LLVM IR + + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) + { + if (left->getType()->isFloatingPointTy()) + { + auto * func_frem = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::vp_frem, left->getType()); + return b.CreateCall(func_frem, {left, right}); + } + else if (left->getType()->isIntegerTy()) + return is_signed ? b.CreateSRem(left, right) : b.CreateURem(left, right); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "ModuloImpl compilation expected native integer or floating point type"); + } #endif }; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index df239b820af..500eb471a13 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -2368,7 +2368,18 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A using ResultDataType = typename BinaryOperationTraits::ResultDataType; using OpSpec = Op; if constexpr (!std::is_same_v && !IsDataTypeDecimal && OpSpec::compilable) - return true; + { + if constexpr (is_modulo) + { + using PromotedType = std::conditional_t< + std::is_floating_point_v, + Float64, + NumberTraits::ResultOfIf>; + return std::is_integral_v || std::is_floating_point_v; + } + else + return true; + } } return false; }); @@ -2393,10 +2404,23 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A if constexpr (!std::is_same_v && !IsDataTypeDecimal && OpSpec::compilable) { auto & b = static_cast &>(builder); - auto * lval = nativeCast(b, arguments[0], result_type); - auto * rval = nativeCast(b, arguments[1], result_type); - result = OpSpec::compile(b, lval, rval, std::is_signed_v); - + if constexpr (is_modulo) + { + using PromotedType = std::conditional_t< + std::is_floating_point_v, + Float64, + NumberTraits::ResultOfIf>; + auto promoted_type = std::make_shared>(); + auto * lval = nativeCast(b, arguments[0], promoted_type); + auto * rval = nativeCast(b, arguments[1], promoted_type); + result = OpSpec::compile(b, lval, rval, std::is_signed_v); + } + else + { + auto * lval = nativeCast(b, arguments[0], result_type); + auto * rval = nativeCast(b, arguments[1], result_type); + result = OpSpec::compile(b, lval, rval, std::is_signed_v); + } return true; } } From d8230416d1a43fe262ace9568c16aa95f18ad1c0 Mon Sep 17 00:00:00 2001 From: zhangwanyun1 <643044969@qq.com> Date: Wed, 16 Oct 2024 11:29:06 +0800 Subject: [PATCH 050/433] clarify the documentation --- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index cc4cd5a32ea..5bc84a73334 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -92,7 +92,7 @@ namespace ErrorCodes M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - M(Bool, check_table_structure_completely, false, "If true, tables must have identical definitions, including projections and secondary indices. Otherwise, the source table's projections and secondary indices must be a subset of those in the target table.", 0) \ + M(Bool, check_table_structure_completely, false, "Whether to check table structure completely when manipulate partitions. If true, the source and target tables must have identical definitions including projections and secondary indices. Otherwise, the source table's projections and secondary indices must be a subset of those in the target table.", 0) \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ From 9df69dc6c2d5905288eb68a74b59a359cf0f1b48 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Oct 2024 16:01:19 +0800 Subject: [PATCH 051/433] enable modulo jit --- src/DataTypes/NumberTraits.h | 32 +++++++++++++++++++++ src/Functions/FunctionBinaryArithmetic.h | 36 ++++++++++++++---------- 2 files changed, 53 insertions(+), 15 deletions(-) diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index 59a64017af3..ca482f19819 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -204,6 +204,38 @@ struct ResultOfIf std::conditional_t && !is_decimal, ConstructedType, Error>>>; }; +/** Type casting for `modulo` function: + * UInt, UInt -> UInt + * Int, Int -> Int + * UInt, Int -> Int + * UInt64, Int -> Error + * Float, Float -> Float64 + * Float, [U]Int -> Float64 + */ +template +struct ResultOfModuloNativePromotion +{ + static_assert(std::is_arithmetic_v && std::is_arithmetic_v); + + static constexpr bool has_float = std::is_floating_point_v || std::is_floating_point_v; + static constexpr bool has_integer = is_integer || is_integer; + static constexpr bool has_signed = is_signed_v || is_signed_v; + static constexpr bool has_unsigned = !is_signed_v || !is_signed_v; + + static constexpr size_t max_size_of_unsigned_integer = max(is_signed_v ? 0 : sizeof(A), is_signed_v ? 0 : sizeof(B)); + static constexpr size_t max_size_of_signed_integer = max(is_signed_v ? sizeof(A) : 0, is_signed_v ? sizeof(B) : 0); + static constexpr size_t max_size_of_integer = max(is_integer ? sizeof(A) : 0, is_integer ? sizeof(B) : 0); + + using ConstructedType = typename Construct< + has_signed, + false, + (has_signed ^ has_unsigned) ? max(max_size_of_unsigned_integer * 2, max_size_of_signed_integer) : max(sizeof(A), sizeof(B))>::Type; + + using Type = std::conditional_t< + std::is_same_v, + A, + std::conditional_t>>; +}; /** Before applying operator `%` and bitwise operations, operands are casted to whole numbers. */ template struct ToInteger diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 500eb471a13..026a695b077 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -2371,11 +2371,11 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A { if constexpr (is_modulo) { - using PromotedType = std::conditional_t< - std::is_floating_point_v, - Float64, - NumberTraits::ResultOfIf>; - return std::is_integral_v || std::is_floating_point_v; + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = NumberTraits::ResultOfModuloNativePromotion; + if constexpr (std::is_arithmetic_v) + return true; } else return true; @@ -2406,25 +2406,31 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A auto & b = static_cast &>(builder); if constexpr (is_modulo) { - using PromotedType = std::conditional_t< - std::is_floating_point_v, - Float64, - NumberTraits::ResultOfIf>; - auto promoted_type = std::make_shared>(); - auto * lval = nativeCast(b, arguments[0], promoted_type); - auto * rval = nativeCast(b, arguments[1], promoted_type); - result = OpSpec::compile(b, lval, rval, std::is_signed_v); + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = NumberTraits::ResultOfModuloNativePromotion; + if constexpr (std::is_arithmetic_v) + { + DataTypePtr promoted_type = std::make_shared>(); + if (result_type->isNullable()) + promoted_type = std::make_shared(promoted_type); + + auto * lval = nativeCast(b, arguments[0], promoted_type); + auto * rval = nativeCast(b, arguments[1], promoted_type); + result + = nativeCast(b, promoted_type, OpSpec::compile(b, lval, rval, std::is_signed_v), result_type); + return true; + } } else { auto * lval = nativeCast(b, arguments[0], result_type); auto * rval = nativeCast(b, arguments[1], result_type); result = OpSpec::compile(b, lval, rval, std::is_signed_v); + return true; } - return true; } } - return false; }); From f7062a4af471517a770f1e0519aac0fa92dae486 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Oct 2024 16:16:55 +0800 Subject: [PATCH 052/433] fix failed uts in https://s3.amazonaws.com/clickhouse-test-reports/70598/91ac8ffcc04d793cd17c30b3221f228d525bcba1/stateless_tests__asan__\[1_2\]/fatal_messages.txt --- src/Interpreters/ExpressionJIT.cpp | 2 +- src/Interpreters/JIT/CompileDAG.cpp | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 16275b23053..cd23599fa20 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -298,7 +298,7 @@ static FunctionBasePtr compile( { auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] () { - LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); + LOG_DEBUG(getLogger(), "Compile expression {}", llvm_function->getName()); auto compiled_function = compileFunction(getJITInstance(), *llvm_function); return std::make_shared(compiled_function); }); diff --git a/src/Interpreters/JIT/CompileDAG.cpp b/src/Interpreters/JIT/CompileDAG.cpp index 9c56341eae0..d9a0e196e45 100644 --- a/src/Interpreters/JIT/CompileDAG.cpp +++ b/src/Interpreters/JIT/CompileDAG.cpp @@ -46,14 +46,18 @@ ValueWithType CompileDAG::compile(llvm::IRBuilderBase & builder, const ValuesWit { ValuesWithType temporary_values; temporary_values.reserve(node.arguments.size()); - for (auto argument_index : node.arguments) { assert(compiled_values[argument_index].value != nullptr); temporary_values.emplace_back(compiled_values[argument_index]); } - compiled_values[compiled_values_index] = {node.function->compile(builder, temporary_values), node.result_type}; + ValueWithType compiled_value{node.function->compile(builder, temporary_values), node.function->getResultType()}; + if (!node.result_type->equals(*node.function->getResultType())) + compiled_values[compiled_values_index] = {nativeCast(b, compiled_value, node.result_type), node.result_type}; + else + compiled_values[compiled_values_index] = std::move(compiled_value); + break; } case CompileType::INPUT: From 51ceb70f35544e1df3d7f81e825b9e2214f4f73a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Oct 2024 18:48:19 +0800 Subject: [PATCH 053/433] modulo jit wip --- src/Functions/DivisionUtils.h | 78 ++++++++++++++++++++++++++++++++++- 1 file changed, 77 insertions(+), 1 deletion(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index f5767b01bc4..d1dc1134355 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -6,9 +6,11 @@ #include #include +#include "DataTypes/Native.h" #include "config.h" #if USE_EMBEDDED_COMPILER +# include # include #endif @@ -22,6 +24,39 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +#if USE_EMBEDDED_COMPILER + +template +static llvm::Value * wrapInNullable(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed, F && f) +{ + auto * left_type = left->getType(); + auto * right_type = right->getType(); + + if (!left_type->isStructTy() && !right_type->isStructTy()) + { + // Both arguments are not nullable. + return f(b, left, right, is_signed); + } + + auto * denull_left = left_type->isStructTy() ? b.CreateExtractValue(left, {1}) : left; + auto * denull_right = right_type->isStructTy() ? b.CreateExtractValue(right, {1}) : right; + auto * denull_result = f(b, denull_left, denull_right, is_signed); + + auto * nullable_result_type = toNullableType(b, denull_result->getType()); + llvm::Value * nullable_result = llvm::Constant::getNullValue(nullable_result_type); + nullable_result = b.CreateInsertValue(nullable_result, denull_result, {0}); + + auto * result_is_null = b.CreateExtractValue(nullable_result, {1}); + if (left_type->isStructTy()) + result_is_null = b.CreateOr(result_is_null, b.CreateExtractValue(left, {1})); + if (right_type->isStructTy()) + result_is_null = b.CreateOr(result_is_null, b.CreateExtractValue(right, {1})); + + return b.CreateInsertValue(nullable_result, result_is_null, {1}); +} + +#endif + template inline void throwIfDivisionLeadsToFPE(A a, B b) { @@ -166,6 +201,16 @@ struct ModuloImpl static constexpr bool compilable = true; /// Ignore exceptions in LLVM IR static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) + { + return wrapInNullable( + b, + left, + right, + is_signed, + [](auto & b_, auto * left_, auto * right_, auto is_signed_) { return compileImpl(b_, left_, right_, is_signed_); }); + } + + static llvm::Value * compileImpl(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) { if (left->getType()->isFloatingPointTy()) { @@ -177,7 +222,8 @@ struct ModuloImpl else throw Exception(ErrorCodes::LOGICAL_ERROR, "ModuloImpl compilation expected native integer or floating point type"); } -#endif + + #endif }; template @@ -212,6 +258,36 @@ struct PositiveModuloImpl : ModuloImpl } return static_cast(res); } + +#if USE_EMBEDDED_COMPILER + static constexpr bool compilable = true; /// Ignore exceptions in LLVM IR + + static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) + { + return wrapInNullable( + b, + left, + right, + is_signed, + [](auto & b_, auto * left_, auto * right_, auto is_signed_) { return compileImpl(b_, left_, right_, is_signed_); }); + } + + static llvm::Value * compileImpl(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) + { + auto * result = ModuloImpl::compileImpl(b, left, right, is_signed); + if (is_signed) + { + /// If result is negative, result += abs(right). + auto * zero = llvm::Constant::getNullValue(result->getType()); + auto * is_negative = b.CreateICmpSLT(result, zero); + auto * abs_right = b.CreateSelect(b.CreateICmpSLT(right, zero), b.CreateNeg(right), right); + return b.CreateSelect(is_negative, b.CreateAdd(result, abs_right), result); + } + else + return result; + } +#endif + }; } From a4a9a676db83bf155342c04e2ae222e1dad742ea Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 17 Oct 2024 10:02:23 +0800 Subject: [PATCH 054/433] fix modulo/pmod compilation --- src/DataTypes/NumberTraits.h | 2 +- src/Functions/FunctionBinaryArithmetic.h | 139 +++++++++++++---------- 2 files changed, 79 insertions(+), 62 deletions(-) diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index ca482f19819..7e535c1e35d 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -215,7 +215,7 @@ struct ResultOfIf template struct ResultOfModuloNativePromotion { - static_assert(std::is_arithmetic_v && std::is_arithmetic_v); + static_assert(is_arithmetic_v && is_arithmetic_v); static constexpr bool has_float = std::is_floating_point_v || std::is_floating_point_v; static constexpr bool has_integer = is_integer || is_integer; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 026a695b077..0744348db2d 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -787,6 +787,7 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; static constexpr bool is_modulo = IsOperation::modulo; + static constexpr bool is_positive_modulo = IsOperation::positive_modulo; static constexpr bool is_int_div = IsOperation::int_div; static constexpr bool is_int_div_or_zero = IsOperation::int_div_or_zero; @@ -2350,89 +2351,105 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A if (!canBeNativeType(*arguments[0]) || !canBeNativeType(*arguments[1]) || !canBeNativeType(*result_type)) return false; - WhichDataType data_type_lhs(arguments[0]); - WhichDataType data_type_rhs(arguments[1]); + auto denull_left_type = removeNullable(arguments[0]); + auto denull_right_type = removeNullable(arguments[1]); + WhichDataType data_type_lhs(denull_left_type); + WhichDataType data_type_rhs(denull_right_type); if ((data_type_lhs.isDateOrDate32() || data_type_lhs.isDateTime()) || (data_type_rhs.isDateOrDate32() || data_type_rhs.isDateTime())) return false; - return castBothTypes(arguments[0].get(), arguments[1].get(), [&](const auto & left, const auto & right) - { - using LeftDataType = std::decay_t; - using RightDataType = std::decay_t; - if constexpr (!std::is_same_v && - !std::is_same_v && - !std::is_same_v && - !std::is_same_v) + return castBothTypes( + denull_left_type.get(), + denull_right_type.get(), + [&](const auto & left, const auto & right) { - using ResultDataType = typename BinaryOperationTraits::ResultDataType; - using OpSpec = Op; - if constexpr (!std::is_same_v && !IsDataTypeDecimal && OpSpec::compilable) + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + if constexpr ( + !std::is_same_v && !std::is_same_v + && !std::is_same_v && !std::is_same_v) { - if constexpr (is_modulo) + using ResultDataType = typename BinaryOperationTraits::ResultDataType; + using OpSpec = Op; + + if constexpr ( + !std::is_same_v && !IsDataTypeDecimal + && !IsDataTypeDecimal && !IsDataTypeDecimal && OpSpec::compilable) { - using LeftType = typename LeftDataType::FieldType; - using RightType = typename RightDataType::FieldType; - using PromotedType = NumberTraits::ResultOfModuloNativePromotion; - if constexpr (std::is_arithmetic_v) + if constexpr (is_modulo || is_positive_modulo) + { + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = typename NumberTraits::ResultOfModuloNativePromotion::Type; + if constexpr (std::is_arithmetic_v) + { + return true; + } + } + else return true; } - else - return true; } - } - return false; - }); + return false; + }); } llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override { assert(2 == arguments.size()); + auto denull_left_type = removeNullable(arguments[0].type); + auto denull_right_type = removeNullable(arguments[1].type); llvm::Value * result = nullptr; - castBothTypes(arguments[0].type.get(), arguments[1].type.get(), [&](const auto & left, const auto & right) - { - using LeftDataType = std::decay_t; - using RightDataType = std::decay_t; - if constexpr (!std::is_same_v && - !std::is_same_v && - !std::is_same_v && - !std::is_same_v) - { - using ResultDataType = typename BinaryOperationTraits::ResultDataType; - using OpSpec = Op; - if constexpr (!std::is_same_v && !IsDataTypeDecimal && OpSpec::compilable) - { - auto & b = static_cast &>(builder); - if constexpr (is_modulo) - { - using LeftType = typename LeftDataType::FieldType; - using RightType = typename RightDataType::FieldType; - using PromotedType = NumberTraits::ResultOfModuloNativePromotion; - if constexpr (std::is_arithmetic_v) - { - DataTypePtr promoted_type = std::make_shared>(); - if (result_type->isNullable()) - promoted_type = std::make_shared(promoted_type); - auto * lval = nativeCast(b, arguments[0], promoted_type); - auto * rval = nativeCast(b, arguments[1], promoted_type); - result - = nativeCast(b, promoted_type, OpSpec::compile(b, lval, rval, std::is_signed_v), result_type); + castBothTypes( + denull_left_type.get(), + denull_right_type.get(), + [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + if constexpr ( + !std::is_same_v && !std::is_same_v + && !std::is_same_v && !std::is_same_v) + { + using ResultDataType = typename BinaryOperationTraits::ResultDataType; + using OpSpec = Op; + if constexpr ( + !std::is_same_v && !IsDataTypeDecimal + && !IsDataTypeDecimal && !IsDataTypeDecimal && OpSpec::compilable) + { + auto & b = static_cast &>(builder); + if constexpr (is_modulo || is_positive_modulo) + { + using LeftType = typename LeftDataType::FieldType; + using RightType = typename RightDataType::FieldType; + using PromotedType = typename NumberTraits::ResultOfModuloNativePromotion::Type; + if constexpr (std::is_arithmetic_v) + { + DataTypePtr promoted_type = std::make_shared>(); + if (result_type->isNullable()) + promoted_type = std::make_shared(promoted_type); + + auto * lval = nativeCast(b, arguments[0], promoted_type); + auto * rval = nativeCast(b, arguments[1], promoted_type); + result = nativeCast( + b, promoted_type, OpSpec::compile(b, lval, rval, std::is_signed_v), result_type); + return true; + } + } + else + { + auto * lval = nativeCast(b, arguments[0], result_type); + auto * rval = nativeCast(b, arguments[1], result_type); + result = OpSpec::compile(b, lval, rval, std::is_signed_v); return true; } } - else - { - auto * lval = nativeCast(b, arguments[0], result_type); - auto * rval = nativeCast(b, arguments[1], result_type); - result = OpSpec::compile(b, lval, rval, std::is_signed_v); - return true; - } } - } - return false; - }); + return false; + }); return result; } From 4970c06c30d07266c90146ef70f1c0ca21bf1b7e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 17 Oct 2024 10:24:31 +0800 Subject: [PATCH 055/433] fix code style --- src/Functions/DivisionUtils.h | 10 +++++----- src/Functions/FunctionBinaryArithmetic.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index d1dc1134355..67c3a7f3b99 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -27,7 +27,7 @@ namespace ErrorCodes #if USE_EMBEDDED_COMPILER template -static llvm::Value * wrapInNullable(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed, F && f) +static llvm::Value * compileWithNullableValues(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed, F && compile_func) { auto * left_type = left->getType(); auto * right_type = right->getType(); @@ -35,12 +35,12 @@ static llvm::Value * wrapInNullable(llvm::IRBuilder<> & b, llvm::Value * left, l if (!left_type->isStructTy() && !right_type->isStructTy()) { // Both arguments are not nullable. - return f(b, left, right, is_signed); + return compile_func(b, left, right, is_signed); } auto * denull_left = left_type->isStructTy() ? b.CreateExtractValue(left, {1}) : left; auto * denull_right = right_type->isStructTy() ? b.CreateExtractValue(right, {1}) : right; - auto * denull_result = f(b, denull_left, denull_right, is_signed); + auto * denull_result = compile_func(b, denull_left, denull_right, is_signed); auto * nullable_result_type = toNullableType(b, denull_result->getType()); llvm::Value * nullable_result = llvm::Constant::getNullValue(nullable_result_type); @@ -202,7 +202,7 @@ struct ModuloImpl static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) { - return wrapInNullable( + return compileWithNullableValues( b, left, right, @@ -264,7 +264,7 @@ struct PositiveModuloImpl : ModuloImpl static llvm::Value * compile(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) { - return wrapInNullable( + return compileWithNullableValues( b, left, right, diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 0744348db2d..cb360a17e97 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -2240,7 +2240,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A const auto & null_bytemap = nullable_column->getNullMapData(); auto res = executeImpl2(createBlockWithNestedColumns(arguments), removeNullable(result_type), input_rows_count, &null_bytemap); - return wrapInNullable(res, arguments, result_type, input_rows_count); + return compileWithNullableValues(res, arguments, result_type, input_rows_count); } /// Special case - one or both arguments are IPv4 From 7248bbe094bb07fb27f1379ae0ac684deba10653 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 17 Oct 2024 18:25:11 +0800 Subject: [PATCH 056/433] enable jit for toIntxx or toFloatxx --- src/DataTypes/IDataType.h | 4 + src/Functions/FunctionBinaryArithmetic.h | 2 +- src/Functions/FunctionsConversion.cpp | 142 +++++++++++++++++++++++ 3 files changed, 147 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index b9b0029fd94..fe7ac38106b 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -540,6 +540,7 @@ inline bool isNullableOrLowCardinalityNullable(const DataTypePtr & data_type) template constexpr bool IsDataTypeDecimal = false; template constexpr bool IsDataTypeNumber = false; +template constexpr bool IsDataTypeNativeNumber = false; template constexpr bool IsDataTypeDateOrDateTime = false; template constexpr bool IsDataTypeDate = false; template constexpr bool IsDataTypeEnum = false; @@ -566,6 +567,9 @@ template constexpr bool IsDataTypeDecimal> = t template <> inline constexpr bool IsDataTypeDecimal = true; template constexpr bool IsDataTypeNumber> = true; +template +requires std::is_arithmetic_v +constexpr bool IsDataTypeNativeNumber> = true; template <> inline constexpr bool IsDataTypeDate = true; template <> inline constexpr bool IsDataTypeDate = true; diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index cb360a17e97..0744348db2d 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -2240,7 +2240,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A const auto & null_bytemap = nullable_column->getNullMapData(); auto res = executeImpl2(createBlockWithNestedColumns(arguments), removeNullable(result_type), input_rows_count, &null_bytemap); - return compileWithNullableValues(res, arguments, result_type, input_rows_count); + return wrapInNullable(res, arguments, result_type, input_rows_count); } /// Special case - one or both arguments are IPv4 diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 615589a7d43..f4180e6b814 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -73,6 +73,10 @@ #include #include +#if USE_EMBEDDED_COMPILER +#include +#endif + namespace DB { namespace Setting @@ -114,6 +118,43 @@ namespace ErrorCodes namespace { +#if USE_EMBEDDED_COMPILER +bool castType(const IDataType * type, auto && f) +{ + using Types = TypeList< + DataTypeUInt8, + DataTypeUInt16, + DataTypeUInt32, + DataTypeUInt64, + DataTypeUInt128, + DataTypeUInt256, + DataTypeInt8, + DataTypeInt16, + DataTypeInt32, + DataTypeInt64, + DataTypeInt128, + DataTypeInt256, + DataTypeFloat32, + DataTypeFloat64, + DataTypeDecimal32, + DataTypeDecimal64, + DataTypeDecimal128, + DataTypeDecimal256, + DataTypeDate, + DataTypeDateTime, + DataTypeFixedString, + DataTypeString, + DataTypeInterval>; + return castTypeToEither(Types{}, type, std::forward(f)); +} + +template +bool castBothTypes(const IDataType * left, const IDataType * right, F && f) +{ + return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); }); +} +#endif + /** Type conversion functions. * toType - conversion in "natural way"; */ @@ -2207,6 +2248,52 @@ public: return Monotonic::get(type, left, right); } +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & types, const DataTypePtr & result_type) const override + { + if (types.size() != 1) + return false; + + if (!canBeNativeType(types[0]) || !canBeNativeType(result_type)) + return false; + + return castBothTypes(types[0].get(), result_type.get(), [](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + return true; + + return false; + }); + } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override + { + llvm::Value * result = nullptr; + castBothTypes( + arguments[0].type.get(), + result_type.get(), + [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + { + result = nativeCast(builder, arguments[0], result_type); + return true; + } + + return false; + }); + + return result; + } +#endif + private: ContextPtr context; mutable bool checked_return_type = false; @@ -3190,6 +3277,61 @@ public: return monotonicity_for_range(type, left, right); } +#if USE_EMBEDDED_COMPILER + bool isCompilable() const override + { + if (getName() != "CAST" || argument_types.size() != 2) + return false; + + const auto & from_type = argument_types[0]; + const auto & to_type = return_type; + auto denull_from_type = removeNullable(from_type); + auto denull_to_type = removeNullable(to_type); + if (!canBeNativeType(denull_from_type) || !canBeNativeType(denull_to_type)) + return false; + + return castBothTypes(denull_from_type.get(), denull_to_type.get(), [](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + return true; + + return false; + }); + } + + llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments) const override + { + llvm::Value * result = nullptr; + + const auto & from_type = arguments[0].type; + const auto & to_type = return_type; + auto denull_from_type = removeNullable(from_type); + auto denull_to_type = removeNullable(to_type); + castBothTypes( + denull_from_type.get(), + denull_to_type.get(), + [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + { + result = nativeCast(builder, arguments[0], return_type); + return true; + } + + return false; + }); + + return result; + } + +#endif + private: const char * cast_name; MonotonicityForRange monotonicity_for_range; From dff73d5baf9b2293fac48bea0d7e6d575bb95c3d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 17 Oct 2024 17:07:46 +0000 Subject: [PATCH 057/433] move the code --- src/Common/computeMaxTableNameLength.cpp | 34 ++++++ src/Common/computeMaxTableNameLength.h | 10 ++ src/Databases/DatabaseOnDisk.cpp | 28 +++++ src/Functions/getMaxTableNameLength.cpp | 101 ++++++++++++++++++ ..._improvement_table_name_too_long.reference | 1 + .../03167_improvement_table_name_too_long.sh | 15 +++ .../03253_getMaxTableNameLength.reference | 2 + .../03253_getMaxTableNameLength.sql | 3 + .../aspell-ignore/en/aspell-dict.txt | 1 + 9 files changed, 195 insertions(+) create mode 100644 src/Common/computeMaxTableNameLength.cpp create mode 100644 src/Common/computeMaxTableNameLength.h create mode 100644 src/Functions/getMaxTableNameLength.cpp create mode 100644 tests/queries/0_stateless/03167_improvement_table_name_too_long.reference create mode 100755 tests/queries/0_stateless/03167_improvement_table_name_too_long.sh create mode 100644 tests/queries/0_stateless/03253_getMaxTableNameLength.reference create mode 100644 tests/queries/0_stateless/03253_getMaxTableNameLength.sql diff --git a/src/Common/computeMaxTableNameLength.cpp b/src/Common/computeMaxTableNameLength.cpp new file mode 100644 index 00000000000..eb4a72f1863 --- /dev/null +++ b/src/Common/computeMaxTableNameLength.cpp @@ -0,0 +1,34 @@ +#include +#include + +namespace DB +{ + +size_t computeMaxTableNameLength(const String & database_name, ContextPtr context) +{ + namespace fs = std::filesystem; + + const String suffix = ".sql.detached"; + const String metadata_path = fs::path(context->getPath()) / "metadata"; + const String metadata_dropped_path = fs::path(context->getPath()) / "metadata_dropped"; + + // Helper lambda to get the maximum name length + auto get_max_name_length = [](const String & path) -> size_t { + auto length = pathconf(path.c_str(), _PC_NAME_MAX); + return (length == -1) ? NAME_MAX : static_cast(length); + }; + + size_t max_create_length = get_max_name_length(metadata_path) - suffix.length(); + size_t max_dropped_length = get_max_name_length(metadata_dropped_path); + + size_t escaped_db_name_length = escapeForFileName(database_name).length(); + const size_t uuid_length = 36; // Standard UUID length + const size_t extension_length = 6; // Length of ".sql" including three dots + + // Adjust for database name and UUID in dropped table filenames + size_t max_to_drop = max_dropped_length - escaped_db_name_length - uuid_length - extension_length; + + // Return the minimum of the two calculated lengths + return std::min(max_create_length, max_to_drop); +} +} diff --git a/src/Common/computeMaxTableNameLength.h b/src/Common/computeMaxTableNameLength.h new file mode 100644 index 00000000000..6096ec51e33 --- /dev/null +++ b/src/Common/computeMaxTableNameLength.h @@ -0,0 +1,10 @@ +#pragma once + +#include +#include + +namespace DB +{ + +size_t computeMaxTableNameLength(const String & database_name, ContextPtr context); +} diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f914d9024e1..e3b9d90b1d9 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -67,6 +68,7 @@ namespace ErrorCodes extern const int EMPTY_LIST_OF_COLUMNS_PASSED; extern const int DATABASE_NOT_EMPTY; extern const int INCORRECT_QUERY; + extern const int ARGUMENT_OUT_OF_BOUND; } @@ -401,6 +403,32 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to } } +void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to_table_name) const +{ + // Compute allowed max length directly + size_t allowed_max_length = computeMaxTableNameLength(database_name, getContext()); + String table_metadata_path = getObjectMetadataPath(to_table_name); + + if (escapeForFileName(to_table_name).length() > allowed_max_length) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "The max length of table name for database {} is {}, current length is {}", + database_name, allowed_max_length, to_table_name.length()); + + if (fs::exists(table_metadata_path)) + { + fs::path detached_permanently_flag(table_metadata_path + detached_suffix); + + if (fs::exists(detached_permanently_flag)) + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, + "Table {}.{} already exists (detached permanently)", + backQuote(database_name), backQuote(to_table_name)); + else + throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, + "Table {}.{} already exists (detached)", + backQuote(database_name), backQuote(to_table_name)); + } +} + void DatabaseOnDisk::renameTable( ContextPtr local_context, const String & table_name, diff --git a/src/Functions/getMaxTableNameLength.cpp b/src/Functions/getMaxTableNameLength.cpp new file mode 100644 index 00000000000..0b45fed28cd --- /dev/null +++ b/src/Functions/getMaxTableNameLength.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int INCORRECT_DATA; +} + + + +class FunctionGetMaxTableNameLength : public IFunction, WithContext +{ +public: + static constexpr auto name = "getMaxTableNameLengthForDatabase"; + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + explicit FunctionGetMaxTableNameLength(ContextPtr context_) : WithContext(context_) + { + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 1) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Number of arguments for function {} can't be {}, should be 1", getName(), arguments.size()); + + WhichDataType which(arguments[0]); + + if (!which.isStringOrFixedString()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}, expected String or FixedString", + arguments[0]->getName(), getName()); + + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + size_t allowed_max_length; + + if (!isColumnConst(*arguments[0].column.get())) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "The argument of function {} must be constant.", getName()); + + const ColumnConst * col_const = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + if (!col_const) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected a constant string as argument for function {}", getName()); + + String database_name = col_const->getValue(); + + if (database_name.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect name for a database. It shouldn't be empty"); + + allowed_max_length = computeMaxTableNameLength(database_name, getContext()); + return DataTypeUInt64().createColumnConst(input_rows_count, allowed_max_length); + } + +private: + const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) const + { + if (const auto * col = checkAndGetColumnConst(column)) + return col; + if (const auto * col = checkAndGetColumnConst(column)) + return col; + return nullptr; + } +}; + +REGISTER_FUNCTION(getMaxTableName) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.reference b/tests/queries/0_stateless/03167_improvement_table_name_too_long.reference new file mode 100644 index 00000000000..ce34916a254 --- /dev/null +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.reference @@ -0,0 +1 @@ +ARGUMENT_OUT_OF_BOUND diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh new file mode 100755 index 00000000000..7b3c5bb9b28 --- /dev/null +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameForDatabase('$CLICKHOUSE_DATABASE')") +let excess_length=allowed_name_length+1 + +long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) +allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) + +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 ARGUMENT_OUT_OF_BOUND +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" +$CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" diff --git a/tests/queries/0_stateless/03253_getMaxTableNameLength.reference b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference new file mode 100644 index 00000000000..560a78db396 --- /dev/null +++ b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference @@ -0,0 +1,2 @@ +206 +204 diff --git a/tests/queries/0_stateless/03253_getMaxTableNameLength.sql b/tests/queries/0_stateless/03253_getMaxTableNameLength.sql new file mode 100644 index 00000000000..8500b41725d --- /dev/null +++ b/tests/queries/0_stateless/03253_getMaxTableNameLength.sql @@ -0,0 +1,3 @@ +SELECT getMaxTableNameLengthForDatabase('default'); +SELECT getMaxTableNameLengthForDatabase('default21'); +SELECT getMaxTableNameLengthForDatabase(''); -- { INCORRECT_DATA } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a5e06af213e..3ef2755c008 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1748,6 +1748,7 @@ geoip geospatial getClientHTTPHeader getMacro +getMaxTableNameLengthForDatabase getOSKernelVersion getServerPort getSetting From 7e1517ffb2248dc33a75a8f1720d9e9e4abca6b2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 17 Oct 2024 19:10:47 +0200 Subject: [PATCH 058/433] docs fix --- .../functions/other-functions.md | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 0a50960b453..6276754b6a2 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -4423,4 +4423,38 @@ Result: ┌─globalVariable('max_allowed_packet')─┐ │ 67108864 │ └──────────────────────────────────────┘ -``` \ No newline at end of file +``` + +## getMaxTableNameLengthForDatabase + +Returns the maximum table name length in a specified database. + +**Syntax** + +```sql +getMaxTableNameLengthForDatabase(database_name) +``` + +**Arguments** + +- `database_name` — The name of the specified database. [String](../data-types/string.md). + +**Returned value** + +- Returns the length of the maximum table name. + +**Example** + +Query: + +```sql +SELECT getMaxTableNameLengthForDatabase('default'); +``` + +Result: + +```response +┌─getMaxTableNameLengthForDatabase('default')─┐ +│ 206 │ +└─────────────────────────────────────────────┘ +``` From d50644ce96addb6a2643d698bd560ec4c68b4356 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 17 Oct 2024 19:15:56 +0200 Subject: [PATCH 059/433] empty commit From d8899f8758e2c14976ac4697316e160b041b0917 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 17 Oct 2024 19:28:56 +0200 Subject: [PATCH 060/433] fix style --- src/Functions/getMaxTableNameLength.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/getMaxTableNameLength.cpp b/src/Functions/getMaxTableNameLength.cpp index 0b45fed28cd..263fa2f2919 100644 --- a/src/Functions/getMaxTableNameLength.cpp +++ b/src/Functions/getMaxTableNameLength.cpp @@ -19,8 +19,6 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } - - class FunctionGetMaxTableNameLength : public IFunction, WithContext { public: From 66de8310c31db09dcf0b13028909b065f34f1dd3 Mon Sep 17 00:00:00 2001 From: zhangwanyun1 <643044969@qq.com> Date: Fri, 18 Oct 2024 01:34:14 +0800 Subject: [PATCH 061/433] fix ci --- .../test.py | 93 +++++++++++++++---- 1 file changed, 73 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_attach_with_different_projections_or_indices/test.py b/tests/integration/test_attach_with_different_projections_or_indices/test.py index bff993e608d..75ce524b006 100644 --- a/tests/integration/test_attach_with_different_projections_or_indices/test.py +++ b/tests/integration/test_attach_with_different_projections_or_indices/test.py @@ -1,14 +1,19 @@ import pytest -from helpers.cluster import ClickHouseCluster + from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( "node1", main_configs=["configs/config_with_check_table_structure_completely.xml"] ) # node1 = cluster.add_instance("node1") node2 = cluster.add_instance( - "node2", main_configs=["configs/config_without_check_table_structure_completely.xml"] + "node2", + main_configs=["configs/config_without_check_table_structure_completely.xml"], ) + + @pytest.fixture(scope="module") def start_cluster(): try: @@ -16,6 +21,8 @@ def start_cluster(): yield cluster finally: cluster.shutdown() + + # def test_setting_check_table_structure_completely(start_cluster): # assert node1.query("""select value from system.merge_tree_settings where name='check_table_structure_completely';""") == "0\n" def test_check_completely_attach_with_different_indices(start_cluster): @@ -32,7 +39,9 @@ def test_check_completely_attach_with_different_indices(start_cluster): ORDER BY a """ ) - node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node1.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);" + ) node1.query( """ CREATE TABLE attach_partition_t2 @@ -48,7 +57,9 @@ def test_check_completely_attach_with_different_indices(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node1.query( """ @@ -66,11 +77,15 @@ def test_check_completely_attach_with_different_indices(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node1.query("DROP TABLE attach_partition_t1") node1.query("DROP TABLE attach_partition_t2") node1.query("DROP TABLE attach_partition_t3") + + def test_check_attach_with_different_indices(start_cluster): node2.query( """ @@ -85,7 +100,9 @@ def test_check_attach_with_different_indices(start_cluster): ORDER BY a """ ) - node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node2.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);" + ) node2.query( """ CREATE TABLE attach_partition_t2 @@ -101,7 +118,9 @@ def test_check_attach_with_different_indices(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node2.query( """ @@ -117,13 +136,17 @@ def test_check_attach_with_different_indices(start_cluster): ORDER BY a """ ) - node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `b` = '1'") == "1\n" assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `c` = '1'") == "1\n" node2.query("DROP TABLE attach_partition_t1") node2.query("DROP TABLE attach_partition_t2") node2.query("DROP TABLE attach_partition_t3") + + def test_check_completely_attach_with_different_projections(start_cluster): node1.query( """ @@ -142,7 +165,9 @@ def test_check_completely_attach_with_different_projections(start_cluster): ORDER BY a """ ) - node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);") + node1.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);" + ) node1.query( """ CREATE TABLE attach_partition_t2 @@ -162,7 +187,9 @@ def test_check_completely_attach_with_different_projections(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different projections" in str(exc.value) node1.query( """ @@ -189,11 +216,15 @@ def test_check_completely_attach_with_different_projections(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different projections" in str(exc.value) node1.query("DROP TABLE attach_partition_t1") node1.query("DROP TABLE attach_partition_t2") node1.query("DROP TABLE attach_partition_t3") + + def test_check_attach_with_different_projections(start_cluster): node2.query( """ @@ -212,7 +243,9 @@ def test_check_attach_with_different_projections(start_cluster): ORDER BY a """ ) - node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);") + node2.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number) FROM numbers(10);" + ) node2.query( """ CREATE TABLE attach_partition_t2 @@ -232,7 +265,9 @@ def test_check_attach_with_different_projections(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different projections" in str(exc.value) node2.query( """ @@ -257,11 +292,15 @@ def test_check_attach_with_different_projections(start_cluster): ORDER BY a """ ) - node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" node2.query("DROP TABLE attach_partition_t1") node2.query("DROP TABLE attach_partition_t2") node2.query("DROP TABLE attach_partition_t3") + + def test_check_completely_attach_with_different_indices_and_projections(start_cluster): node1.query( """ @@ -282,7 +321,9 @@ def test_check_completely_attach_with_different_indices_and_projections(start_cl ORDER BY a """ ) - node1.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node1.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);" + ) node1.query( """ CREATE TABLE attach_partition_t2 @@ -305,7 +346,9 @@ def test_check_completely_attach_with_different_indices_and_projections(start_cl ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node1.query( """ @@ -335,11 +378,15 @@ def test_check_completely_attach_with_different_indices_and_projections(start_cl ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node1.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node1.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node1.query("DROP TABLE attach_partition_t1") node1.query("DROP TABLE attach_partition_t2") node1.query("DROP TABLE attach_partition_t3") + + def test_check_attach_with_different_indices_and_projections(start_cluster): node2.query( """ @@ -360,7 +407,9 @@ def test_check_attach_with_different_indices_and_projections(start_cluster): ORDER BY a """ ) - node2.query("INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);") + node2.query( + "INSERT INTO attach_partition_t1 SELECT number, toString(number), toString(number) FROM numbers(10);" + ) node2.query( """ CREATE TABLE attach_partition_t2 @@ -383,7 +432,9 @@ def test_check_attach_with_different_indices_and_projections(start_cluster): ) # serverError 36 with pytest.raises(QueryRuntimeException) as exc: - node2.query("ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t2 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert "Tables have different secondary indices" in str(exc.value) node2.query( """ @@ -411,7 +462,9 @@ def test_check_attach_with_different_indices_and_projections(start_cluster): ORDER BY a """ ) - node2.query("ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;") + node2.query( + "ALTER TABLE attach_partition_t3 ATTACH PARTITION tuple() FROM attach_partition_t1;" + ) assert node2.query("SELECT COUNT() FROM attach_partition_t3") == "10\n" assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `b` = '1'") == "1\n" assert node2.query("SELECT `a` FROM attach_partition_t3 WHERE `c` = '1'") == "1\n" From 5f3ccdc3f1f57bc769efe19cef299c4bc8cd8a38 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 17 Oct 2024 20:46:59 +0200 Subject: [PATCH 062/433] Update DatabaseOnDisk.cpp --- src/Databases/DatabaseOnDisk.cpp | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index e3b9d90b1d9..26486c5fdb1 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -387,22 +387,6 @@ void DatabaseOnDisk::checkMetadataFilenameAvailability(const String & to_table_n checkMetadataFilenameAvailabilityUnlocked(to_table_name); } -void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to_table_name) const -{ - String table_metadata_path = getObjectMetadataPath(to_table_name); - - if (fs::exists(table_metadata_path)) - { - fs::path detached_permanently_flag(table_metadata_path + detached_suffix); - - if (fs::exists(detached_permanently_flag)) - throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", - backQuote(database_name), backQuote(to_table_name)); - throw Exception( - ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(database_name), backQuote(to_table_name)); - } -} - void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to_table_name) const { // Compute allowed max length directly From 29b0f31f856de1c4c8f6a3ee98965c873ee1f85e Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 18 Oct 2024 11:05:56 +0800 Subject: [PATCH 063/433] fix failed ut in https://s3.amazonaws.com/clickhouse-test-reports/70598/44f23d60eed468333a9e660e5b4e64bbe63927f5/stateless_tests__release_/fatal_messages.txt --- src/Functions/DivisionUtils.h | 5 +---- src/Interpreters/JIT/CHJIT.cpp | 4 ++++ 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 67c3a7f3b99..08e4fb091c9 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -213,10 +213,7 @@ struct ModuloImpl static llvm::Value * compileImpl(llvm::IRBuilder<> & b, llvm::Value * left, llvm::Value * right, bool is_signed) { if (left->getType()->isFloatingPointTy()) - { - auto * func_frem = llvm::Intrinsic::getDeclaration(b.GetInsertBlock()->getModule(), llvm::Intrinsic::vp_frem, left->getType()); - return b.CreateCall(func_frem, {left, right}); - } + return b.CreateFRem(left, right); else if (left->getType()->isIntegerTy()) return is_signed ? b.CreateSRem(left, right) : b.CreateURem(left, right); else diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 8e4f73a1bea..c7d5681c6ab 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -3,6 +3,7 @@ #if USE_EMBEDDED_COMPILER #include +#include #include @@ -370,6 +371,9 @@ CHJIT::CHJIT() symbol_resolver->registerSymbol("memset", reinterpret_cast(&memset)); symbol_resolver->registerSymbol("memcpy", reinterpret_cast(&memcpy)); symbol_resolver->registerSymbol("memcmp", reinterpret_cast(&memcmp)); + + double (*fmod_ptr)(double, double) = &fmod; + symbol_resolver->registerSymbol("fmod", reinterpret_cast(fmod_ptr)); } CHJIT::~CHJIT() = default; From 0a6556d6969c41f560cfbf8e4e9fe0d6f57fe560 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 18 Oct 2024 13:56:02 +0000 Subject: [PATCH 064/433] fix fast tests --- .../02415_all_new_functions_must_be_documented.reference | 1 + .../0_stateless/03167_improvement_table_name_too_long.sh | 2 +- tests/queries/0_stateless/03253_getMaxTableNameLength.sql | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 7c541f272c8..d1c41ad8025 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -319,6 +319,7 @@ geohashDecode geohashEncode geohashesInBox getMacro +getMaxTableNameLengthForDatabase getOSKernelVersion getServerPort getSizeOfEnumType diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 7b3c5bb9b28..4f0056ca302 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameForDatabase('$CLICKHOUSE_DATABASE')") +allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") let excess_length=allowed_name_length+1 long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) diff --git a/tests/queries/0_stateless/03253_getMaxTableNameLength.sql b/tests/queries/0_stateless/03253_getMaxTableNameLength.sql index 8500b41725d..6d7abc2994b 100644 --- a/tests/queries/0_stateless/03253_getMaxTableNameLength.sql +++ b/tests/queries/0_stateless/03253_getMaxTableNameLength.sql @@ -1,3 +1,3 @@ SELECT getMaxTableNameLengthForDatabase('default'); SELECT getMaxTableNameLengthForDatabase('default21'); -SELECT getMaxTableNameLengthForDatabase(''); -- { INCORRECT_DATA } +SELECT getMaxTableNameLengthForDatabase(''); -- { serverError INCORRECT_DATA } From 9835894ed603c5aea5de3288be9e2a54b7b1fb0d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 18 Oct 2024 15:25:18 +0000 Subject: [PATCH 065/433] fix --- .../0_stateless/03167_improvement_table_name_too_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 4f0056ca302..fde4249f70c 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -10,6 +10,6 @@ let excess_length=allowed_name_length+1 long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 ARGUMENT_OUT_OF_BOUND +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m "ARGUMENT_OUT_OF_BOUND" $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" From d828adf2c53b5b055966b53ed603150f1173c398 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 18 Oct 2024 16:20:20 +0000 Subject: [PATCH 066/433] fix tests (stderr -> stdout) --- .../0_stateless/03167_improvement_table_name_too_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index fde4249f70c..9d52179c458 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -10,6 +10,6 @@ let excess_length=allowed_name_length+1 long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m "ARGUMENT_OUT_OF_BOUND" +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" From ac2140d3af01fd95ebfd89131be1f467dc6a42e3 Mon Sep 17 00:00:00 2001 From: zhangwanyun1 <643044969@qq.com> Date: Sat, 19 Oct 2024 20:12:16 +0800 Subject: [PATCH 067/433] fix the location of the new setting --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index bfb78b36a2d..3604137fd35 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -68,7 +68,6 @@ static std::initializer_list Date: Sun, 20 Oct 2024 00:54:29 +0800 Subject: [PATCH 068/433] retry ci From f127a66768145bdf90862f808669a6f99c7166a8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 21 Oct 2024 16:05:41 +0800 Subject: [PATCH 069/433] fix failed uts in tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.sql --- src/Functions/FunctionsLogical.cpp | 12 ++++++------ src/Functions/FunctionsLogical.h | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 3a6a811d8e2..02efb8f8d8f 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -526,18 +526,18 @@ llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from auto * tenary_null = llvm::ConstantInt::get(result_type, 1); auto * inner = nativeTenaryCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0})); auto * is_null = b.CreateExtractValue(value, {1}); - return b.CreateSelect(b.CreateICmpNE(is_null, llvm::Constant::getNullValue(is_null->getType())), tenary_null, inner); + return b.CreateSelect(is_null, tenary_null, inner); } auto * zero = llvm::Constant::getNullValue(value->getType()); auto * tenary_true = llvm::ConstantInt::get(result_type, 2); auto * tenary_false = llvm::ConstantInt::get(result_type, 0); if (value->getType()->isIntegerTy()) - return b.CreateSelect(b.CreateICmpEQ(value, zero), tenary_true, tenary_false); - if (value->getType()->isFloatingPointTy()) - return b.CreateSelect(b.CreateFCmpOEQ(value, zero), tenary_true, tenary_false); - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to tenary", from_type->getName()); + return b.CreateSelect(b.CreateICmpNE(value, zero), tenary_true, tenary_false); + else if (value->getType()->isFloatingPointTy()) + return b.CreateSelect(b.CreateFCmpONE(value, zero), tenary_true, tenary_false); + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to tenary", from_type->getName()); } /// Cast LLVM value with type to Tenary diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 297b436bd83..5a23588c2e7 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -266,7 +266,7 @@ public: auto * nullable_result = llvm::Constant::getNullValue(nullable_result_type); auto * nullable_result_with_value = b.CreateInsertValue(nullable_result, b.CreateSelect(is_true, b.getInt8(1), b.getInt8(0)), {0}); - return b.CreateInsertValue(nullable_result_with_value, b.CreateSelect(is_null, b.getInt8(1), b.getInt8(0)), {1}); + return b.CreateInsertValue(nullable_result_with_value, is_null, {1}); } } #endif From aa58fde1d70591ad13d02cfe63b376ef11925816 Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Mon, 21 Oct 2024 12:55:53 +0200 Subject: [PATCH 070/433] Cosmetic changes, upd setting name --- docs/en/sql-reference/statements/alter/partition.md | 6 +++--- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeData.cpp | 7 ++++--- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../config_with_check_table_structure_completely.xml | 2 +- .../config_without_check_table_structure_completely.xml | 2 +- .../test.py | 2 +- 8 files changed, 16 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index cbf83499a0f..087458a6646 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -132,7 +132,7 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key, the same order by key and the same primary key. - Both tables must have the same storage policy. -- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). +- The destination table must include all indices and projections from the source table. If the `enforce_index_structure_match_on_partition_manipulation` setting is enabled in destination table, the indices and projections must be identical. Otherwise, the destination table can have a superset of the source table’s indices and projections. ## REPLACE PARTITION @@ -152,7 +152,7 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. - Both tables must have the same partition key, the same order by key and the same primary key. - Both tables must have the same storage policy. -- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). +- The destination table must include all indices and projections from the source table. If the `enforce_index_structure_match_on_partition_manipulation` setting is enabled in destination table, the indices and projections must be identical. Otherwise, the destination table can have a superset of the source table’s indices and projections. ## MOVE PARTITION TO TABLE @@ -168,7 +168,7 @@ For the query to run successfully, the following conditions must be met: - Both tables must have the same partition key, the same order by key and the same primary key. - Both tables must have the same storage policy. - Both tables must be the same engine family (replicated or non-replicated). -- The dest table should have at least the same definitions as the source table, and can have more projections and secondary indices(If check_table_structure_completely is set to true, both tables must have the same indices and projections). +- The destination table must include all indices and projections from the source table. If the `enforce_index_structure_match_on_partition_manipulation` setting is enabled in destination table, the indices and projections must be identical. Otherwise, the destination table can have a superset of the source table’s indices and projections. ## CLEAR COLUMN IN PARTITION diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3604137fd35..3edbc682810 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -580,7 +580,7 @@ static std::initializer_listclone(), properties.columns, getContext()); if (properties.indices.has(index_desc.name)) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use different index names.", backQuoteIfNeed(index_desc.name)); + throw Exception(ErrorCodes::ILLEGAL_INDEX, "Duplicated index name {} is not allowed. Please use a different index name", backQuoteIfNeed(index_desc.name)); const auto & settings = getContext()->getSettingsRef(); if (index_desc.type == FULL_TEXT_INDEX_NAME && !settings[Setting::allow_experimental_full_text_index]) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is disabled. Turn on setting 'allow_experimental_full_text_index'"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The experimental full-text index feature is disabled. Enable the setting 'allow_experimental_full_text_index' to use it"); /// ---- /// Temporary check during a transition period. Please remove at the end of 2024. if (index_desc.type == INVERTED_INDEX_NAME && !settings[Setting::allow_experimental_inverted_index]) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Please use index type 'full_text' instead of 'inverted'"); + throw Exception(ErrorCodes::ILLEGAL_INDEX, "The 'inverted' index type is deprecated. Please use the 'full_text' index type instead"); /// ---- if (index_desc.type == "vector_similarity" && !settings[Setting::allow_experimental_vector_similarity_index]) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled. Turn on setting 'allow_experimental_vector_similarity_index'"); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The experimental vector similarity index feature is disabled. Enable the setting 'allow_experimental_vector_similarity_index' to use it"); properties.indices.push_back(index_desc); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 87e5ae21a03..44363b53540 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -229,7 +229,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString storage_policy; extern const MergeTreeSettingsFloat zero_copy_concurrent_part_removal_max_postpone_ratio; extern const MergeTreeSettingsUInt64 zero_copy_concurrent_part_removal_max_split_times; - extern const MergeTreeSettingsBool check_table_structure_completely; + extern const MergeTreeSettingsBool enforce_index_structure_match_on_partition_manipulation; } @@ -7414,8 +7414,9 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); const auto check_definitions = [this](const auto & my_descriptions, const auto & src_descriptions) { - if (((*getSettings())[MergeTreeSetting::check_table_structure_completely] && my_descriptions.size() != src_descriptions.size()) - || (!(*getSettings())[MergeTreeSetting::check_table_structure_completely] && my_descriptions.size() < src_descriptions.size())) + bool strict_match = (*getSettings())[MergeTreeSetting::enforce_index_structure_match_on_partition_manipulation]; + if ((my_descriptions.size() < src_descriptions.size()) || + (strict_match && my_descriptions.size() != src_descriptions.size())) return false; std::unordered_set my_query_strings; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index b1e1a431ff9..753853ab2f9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -97,7 +97,7 @@ namespace ErrorCodes M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - M(Bool, check_table_structure_completely, false, "Whether to check table structure completely when manipulate partitions. If true, the source and target tables must have identical definitions including projections and secondary indices. Otherwise, the source table's projections and secondary indices must be a subset of those in the target table.", 0) \ + M(Bool, enforce_index_structure_match_on_partition_manipulation, false, "If this setting is enabled for destination table of a partition manipulation query (`ATTACH/MOVE/REPLACE PARTITION`), the indices and projections must be identical between the source and destination tables. Otherwise, the destination table can have a superset of the source table's indices and projections.", 0) \ M(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", 0) \ \ /** Inserts settings. */ \ diff --git a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml index dd9ae5e83d5..06a360847e4 100644 --- a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml +++ b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_with_check_table_structure_completely.xml @@ -1,5 +1,5 @@ - true + true diff --git a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml index 734294478ba..fd78e9d1954 100644 --- a/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml +++ b/tests/integration/test_attach_with_different_projections_or_indices/configs/config_without_check_table_structure_completely.xml @@ -1,5 +1,5 @@ - false + false \ No newline at end of file diff --git a/tests/integration/test_attach_with_different_projections_or_indices/test.py b/tests/integration/test_attach_with_different_projections_or_indices/test.py index 75ce524b006..2db2dc7a7f4 100644 --- a/tests/integration/test_attach_with_different_projections_or_indices/test.py +++ b/tests/integration/test_attach_with_different_projections_or_indices/test.py @@ -24,7 +24,7 @@ def start_cluster(): # def test_setting_check_table_structure_completely(start_cluster): -# assert node1.query("""select value from system.merge_tree_settings where name='check_table_structure_completely';""") == "0\n" +# assert node1.query("""select value from system.merge_tree_settings where name='enforce_index_structure_match_on_partition_manipulation';""") == "0\n" def test_check_completely_attach_with_different_indices(start_cluster): node1.query( """ From eab454f3b9bac9d82b8f36f019cb923cf52063b9 Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Mon, 21 Oct 2024 15:13:58 +0200 Subject: [PATCH 071/433] upd aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 616ad4a800c..3226bf6d6fd 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2984 +personal_ws-1.1 en 2985 AArch ACLs ALTERs @@ -2722,6 +2722,7 @@ summapwithoverflow summingmergetree sumwithoverflow superaggregates +superset supertype supremum symlink From 6645ce546b3d6be5c9cdbd7aec80e2d77f29b738 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 22 Oct 2024 12:25:28 +0800 Subject: [PATCH 072/433] fix failed uts in https://s3.amazonaws.com/clickhouse-test-reports/70598/f127a66768145bdf90862f808669a6f99c7166a8/stateless_tests__release_.html --- src/Interpreters/ActionsDAG.cpp | 12 ++++++- src/Interpreters/ActionsDAG.h | 3 +- src/Processors/QueryPlan/ExpressionStep.cpp | 39 +++++++++++++++++++-- 3 files changed, 49 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e536ca9c0c6..9e4c871662c 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1460,7 +1460,8 @@ ActionsDAG ActionsDAG::makeConvertingActions( MatchColumnsMode mode, bool ignore_constant_values, bool add_casted_columns, - NameToNameMap * new_names) + NameToNameMap * new_names, + NameSet * columns_contain_compiled_function) { size_t num_input_columns = source.size(); size_t num_result_columns = result.size(); @@ -1533,6 +1534,15 @@ ActionsDAG ActionsDAG::makeConvertingActions( "Cannot convert column `{}` because it is constant but values of constants are different in source and result", res_elem.name); } + else if (columns_contain_compiled_function && columns_contain_compiled_function->contains(res_elem.name)) + { + /// It may happen when JIT compilation is enabled that source column is constant and destination column is not constant. + /// e.g. expression "and(equals(materialize(null::Nullable(UInt64)), null::Nullable(UInt64)), equals(null::Nullable(UInt64), null::Nullable(UInt64)))" + /// compiled expression is "and(equals(input: Nullable(UInt64), null), null). Partial evaluation of the compiled expression isn't able to infer that the result column is constant. + /// It causes inconsistency between pipeline header(source column is not constant) and output header of ExpressionStep(destination column is constant). + /// So we need to convert non-constant column to constant column under this condition. + dst_node = &actions_dag.addColumn(res_elem); + } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ac92d99bfa8..155ddbc068e 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -306,7 +306,8 @@ public: MatchColumnsMode mode, bool ignore_constant_values = false, bool add_casted_columns = false, - NameToNameMap * new_names = nullptr); + NameToNameMap * new_names = nullptr, + NameSet * columns_contain_compiled_function = nullptr); /// Create expression which add const column and then materialize it. static ActionsDAG makeAddingColumnActions(ColumnWithTypeAndName column); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index a8a3a7f5b65..963932072da 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -25,6 +25,34 @@ static ITransformingStep::Traits getTraits(const ActionsDAG & actions) }; } +static bool containsCompiledFunction(const ActionsDAG::Node * node) +{ + if (node->type == ActionsDAG::ActionType::FUNCTION && node->is_function_compiled) + return true; + + const auto & children = node->children; + if (children.empty()) + return false; + + bool result = false; + for (const auto & child : children) + result |= containsCompiledFunction(child); + return result; +} + +static NameSet getColumnsContainCompiledFunction(const ActionsDAG & actions_dag) +{ + NameSet result; + for (const auto * node : actions_dag.getOutputs()) + { + if (containsCompiledFunction(node)) + { + result.insert(node->result_name); + } + } + return result; +} + ExpressionStep::ExpressionStep(const Header & input_header_, ActionsDAG actions_dag_) : ITransformingStep( input_header_, @@ -45,10 +73,15 @@ void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const Bu if (!blocksHaveEqualStructure(pipeline.getHeader(), *output_header)) { + auto columns_contain_compiled_function = getColumnsContainCompiledFunction(expression->getActionsDAG()); auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipeline.getHeader().getColumnsWithTypeAndName(), - output_header->getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); + pipeline.getHeader().getColumnsWithTypeAndName(), + output_header->getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name, + false, + false, + nullptr, + &columns_contain_compiled_function); auto convert_actions = std::make_shared(std::move(convert_actions_dag), settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) From fea26702a8e2cd71daa9b5d850b785c79f43a577 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 22 Oct 2024 14:30:39 +0800 Subject: [PATCH 073/433] fix failed ut tests/queries/0_stateless/01870_modulo_partition_key.sql --- src/Functions/DivisionUtils.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 08e4fb091c9..ee048d97ff7 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -227,6 +227,10 @@ template struct ModuloLegacyImpl : ModuloImpl { using ResultType = typename NumberTraits::ResultOfModuloLegacy::Type; + +#if USE_EMBEDDED_COMPILER + static constexpr bool compilable = false; /// moduloLegacy is only used in partition key expression +#endif }; template From ec1a3d082e8fec0ac34a469823d32bfc045ba515 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 24 Oct 2024 11:00:22 +0800 Subject: [PATCH 074/433] change as requested --- src/DataTypes/NumberTraits.h | 1 + src/Functions/FunctionUnaryArithmetic.h | 6 ------ src/Functions/FunctionsLogical.cpp | 4 ++-- src/Interpreters/ExpressionActions.cpp | 4 ++-- src/Interpreters/ExpressionJIT.cpp | 2 +- 5 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index 7e535c1e35d..d1f021ae89c 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -204,6 +204,7 @@ struct ResultOfIf std::conditional_t && !is_decimal, ConstructedType, Error>>>; }; + /** Type casting for `modulo` function: * UInt, UInt -> UInt * Int, Int -> Int diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 729f3d0ce87..fa299075a38 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -489,17 +489,13 @@ public: { using DataType = std::decay_t; if constexpr (std::is_same_v || std::is_same_v) - { return false; - } else { using T0 = typename DataType::FieldType; using T1 = typename Op::ResultType; if constexpr (!std::is_same_v && !IsDataTypeDecimal && Op::compilable) - { return true; - } } return false; @@ -515,9 +511,7 @@ public: { using DataType = std::decay_t; if constexpr (std::is_same_v || std::is_same_v) - { return false; - } else { using T0 = typename DataType::FieldType; diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 02efb8f8d8f..dec5c394a5a 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -516,7 +516,7 @@ namespace FunctionsLogicalDetail #if USE_EMBEDDED_COMPILER -/// Cast LLVM value with type to Tenary +/// Cast LLVM value with type to tenary llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value) { auto * result_type = llvm::Type::getInt8Ty(b.getContext()); @@ -540,7 +540,7 @@ llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to tenary", from_type->getName()); } -/// Cast LLVM value with type to Tenary +/// Cast LLVM value with type to tenary llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type) { return nativeTenaryCast(b, value_with_type.type, value_with_type.value); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index f682ebd75c0..5b1b000b4af 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -61,13 +61,13 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc #if USE_EMBEDDED_COMPILER if (settings.can_compile_expressions && settings.compile_expressions == CompileExpressions::yes) { - LOG_DEBUG( + LOG_TEST( getLogger("ExpressionActions"), "Actions before compilation: {} with {} lazy_executed_nodes", actions_dag.dumpDAG(), lazy_executed_nodes.size()); actions_dag.compileExpressions(settings.min_count_to_compile_expression, lazy_executed_nodes); - LOG_DEBUG(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG()); + LOG_TEST(getLogger("ExpressionActions"), "Actions after compilation: {}", actions_dag.dumpDAG()); } #endif diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index cd23599fa20..16275b23053 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -298,7 +298,7 @@ static FunctionBasePtr compile( { auto [compiled_function_cache_entry, _] = compilation_cache->getOrSet(hash_key, [&] () { - LOG_DEBUG(getLogger(), "Compile expression {}", llvm_function->getName()); + LOG_TRACE(getLogger(), "Compile expression {}", llvm_function->getName()); auto compiled_function = compileFunction(getJITInstance(), *llvm_function); return std::make_shared(compiled_function); }); From 51779eca60e5e3bd360da0ceac2966e83535088a Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Fri, 25 Oct 2024 20:29:01 -0700 Subject: [PATCH 075/433] Change orders in multiple places and add more cases in test file --- src/Interpreters/InterpreterSystemQuery.cpp | 18 +++++++----- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- .../03202_system_load_primary_key.sql | 28 +++++++++++++------ 3 files changed, 31 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c3da850c6a1..656deed53af 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -64,7 +64,6 @@ #include #include #include -#include #include #include #include @@ -780,15 +779,15 @@ BlockIO InterpreterSystemQuery::execute() resetCoverage(); break; } + case Type::LOAD_PRIMARY_KEY: { + loadPrimaryKeys(); + break; + } case Type::UNLOAD_PRIMARY_KEY: { unloadPrimaryKeys(); break; } - case Type::LOAD_PRIMARY_KEY: { - loadPrimaryKeys(); - break; - } #if USE_JEMALLOC case Type::JEMALLOC_PURGE: @@ -1196,7 +1195,6 @@ void InterpreterSystemQuery::loadPrimaryKeys() { getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); - /// Process databases and tables sequentially, without thread pool concurrency at the table level for (auto & database : DatabaseCatalog::instance().getDatabases()) { for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) @@ -1208,9 +1206,15 @@ void InterpreterSystemQuery::loadPrimaryKeys() /// Calls the improved loadPrimaryKeys in MergeTreeData merge_tree->loadPrimaryKeys(); } + catch (const std::exception &ex) + { + LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.what()); + throw; // Re-throw the exception to allow it to propagate + } catch (...) { - LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.message()); + LOG_ERROR(log, "Failed to load primary keys for table {}: unknown exception occurred.", merge_tree->getStorageID().getFullTableName()); + throw; } } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 6254311e4d6..a856e8a5dc8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -92,7 +92,6 @@ public: Type part_type_, const IMergeTreeDataPart * parent_part_); - virtual MergeTreeReaderPtr getReader( const NamesAndTypesList & columns_, const StorageSnapshotPtr & storage_snapshot, @@ -373,8 +372,8 @@ public: void setIndex(const Columns & cols_); void setIndex(Columns && cols_); void loadIndex() const TSA_REQUIRES(index_mutex); - void unloadIndex(); void loadIndexWithLock() const { std::scoped_lock lock(index_mutex); loadIndex(); } + void unloadIndex(); bool isIndexLoaded() const; /// For data in RAM ('index') diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 95f98bc7192..4974ea08105 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS test_load_primary_key_2; CREATE TABLE test_load_primary_key (id Int32, value String) ENGINE = MergeTree() ORDER BY id; CREATE TABLE test_load_primary_key_2 (id Int32, value String) ENGINE = MergeTree() ORDER BY id; --- Inserting some data into both tables +-- Insert data into both tables INSERT INTO test_load_primary_key VALUES (1, 'a'), (2, 'b'), (3, 'c'); INSERT INTO test_load_primary_key_2 VALUES (1, 'x'), (2, 'y'), (3, 'z'); @@ -19,11 +19,10 @@ FROM system.parts WHERE database = currentDatabase() AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); --- Load primary key for both tables -SYSTEM LOAD PRIMARY KEY test_load_primary_key; -SYSTEM LOAD PRIMARY KEY test_load_primary_key_2; +-- Load primary keys for all tables in the database +SYSTEM LOAD PRIMARY KEY; --- Check primary key memory after loading for both tables +-- Verify primary key memory after loading for both tables SELECT table, round(primary_key_bytes_in_memory, -7), @@ -32,11 +31,22 @@ FROM system.parts WHERE database = currentDatabase() AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); --- Unload primary key for both tables -SYSTEM UNLOAD PRIMARY KEY test_load_primary_key; -SYSTEM UNLOAD PRIMARY KEY test_load_primary_key_2; +-- Unload primary keys for all tables in the database +SYSTEM UNLOAD PRIMARY KEY; --- Check primary key memory after unloading for both tables +-- Verify primary key memory after unloading for both tables +SELECT + table, + round(primary_key_bytes_in_memory, -7), + round(primary_key_bytes_in_memory_allocated, -7) +FROM system.parts +WHERE database = currentDatabase() +AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); + +-- Load primary key for only one table +SYSTEM LOAD PRIMARY KEY test_load_primary_key; + +-- Verify that only one table's primary key is loaded SELECT table, round(primary_key_bytes_in_memory, -7), From 586114fc8e95a2c85be5146fd29f58c957e5f891 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 26 Oct 2024 20:47:26 -0700 Subject: [PATCH 076/433] Make corresponding changes in .reference file --- .../03202_system_load_primary_key.reference | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 4b700a0c358..756cf965742 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,14 +1,17 @@ -0 0 -0 0 -0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 # After loading primary keys: -10000000 10000000 -- These numbers will vary depending on the system -10000000 10000000 +test_load_primary_key 10000000 10000000 -- These numbers will vary depending on the system +test_load_primary_key_2 10000000 10000000 # After unloading primary keys: -0 0 -0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 # Data in test_load_primary_key: 1 a From f1d916e02e5842ca6ab659ad8f95a342d3bf789e Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 26 Oct 2024 21:36:08 -0700 Subject: [PATCH 077/433] Fix .reference --- .../03202_system_load_primary_key.reference | 40 ++++++++----------- 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 756cf965742..a025233814f 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,24 +1,16 @@ -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 - -# After loading primary keys: -test_load_primary_key 10000000 10000000 -- These numbers will vary depending on the system -test_load_primary_key_2 10000000 10000000 - -# After unloading primary keys: -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 - -# Data in test_load_primary_key: -1 a -2 b -3 c - -# Data in test_load_primary_key_2: -1 x -2 y -3 z +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 10000000 10000000 -- These numbers will vary depending on the system +test_load_primary_key_2 10000000 10000000 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +1 a +2 b +3 c +1 x +2 y +3 z From b0e6bafae9ab2444cc37d71b5a8f1809d795e254 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Sat, 26 Oct 2024 22:24:07 -0700 Subject: [PATCH 078/433] Fix .reference by changing unrealiable val to 0 --- .../queries/0_stateless/03202_system_load_primary_key.reference | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index a025233814f..76cf14f7c2c 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -4,8 +4,6 @@ test_load_primary_key 0 0 test_load_primary_key_2 0 0 test_load_primary_key 0 0 test_load_primary_key_2 0 0 -test_load_primary_key 10000000 10000000 -- These numbers will vary depending on the system -test_load_primary_key_2 10000000 10000000 test_load_primary_key 0 0 test_load_primary_key_2 0 0 1 a From d1e266a20fdbfc424129aae0703daf24feb439f3 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 28 Oct 2024 08:38:33 +0800 Subject: [PATCH 079/433] fix typo --- src/Functions/FunctionsLogical.cpp | 26 +++++++++++++------------- src/Functions/FunctionsLogical.h | 24 ++++++++++++------------ 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index dec5c394a5a..a00bfeb5906 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -516,34 +516,34 @@ namespace FunctionsLogicalDetail #if USE_EMBEDDED_COMPILER -/// Cast LLVM value with type to tenary -llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value) +/// Cast LLVM value with type to ternary +llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value) { auto * result_type = llvm::Type::getInt8Ty(b.getContext()); if (from_type->isNullable()) { - auto * tenary_null = llvm::ConstantInt::get(result_type, 1); - auto * inner = nativeTenaryCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0})); + auto * ternary_null = llvm::ConstantInt::get(result_type, 1); + auto * inner = nativeTernaryCast(b, removeNullable(from_type), b.CreateExtractValue(value, {0})); auto * is_null = b.CreateExtractValue(value, {1}); - return b.CreateSelect(is_null, tenary_null, inner); + return b.CreateSelect(is_null, ternary_null, inner); } auto * zero = llvm::Constant::getNullValue(value->getType()); - auto * tenary_true = llvm::ConstantInt::get(result_type, 2); - auto * tenary_false = llvm::ConstantInt::get(result_type, 0); + auto * ternary_true = llvm::ConstantInt::get(result_type, 2); + auto * ternary_false = llvm::ConstantInt::get(result_type, 0); if (value->getType()->isIntegerTy()) - return b.CreateSelect(b.CreateICmpNE(value, zero), tenary_true, tenary_false); + return b.CreateSelect(b.CreateICmpNE(value, zero), ternary_true, ternary_false); else if (value->getType()->isFloatingPointTy()) - return b.CreateSelect(b.CreateFCmpONE(value, zero), tenary_true, tenary_false); + return b.CreateSelect(b.CreateFCmpONE(value, zero), ternary_true, ternary_false); else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to tenary", from_type->getName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cast non-number {} to ternary", from_type->getName()); } -/// Cast LLVM value with type to tenary -llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type) +/// Cast LLVM value with type to ternary +llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type) { - return nativeTenaryCast(b, value_with_type.type, value_with_type.value); + return nativeTernaryCast(b, value_with_type.type, value_with_type.value); } #endif diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 5a23588c2e7..8c2fb3efe39 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -80,11 +80,11 @@ namespace Ternary #if USE_EMBEDDED_COMPILER -/// Cast LLVM value with type to Tenary -llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value); +/// Cast LLVM value with type to Ternary +llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, llvm::Value * value); -/// Cast LLVM value with type to Tenary -llvm::Value * nativeTenaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type); +/// Cast LLVM value with type to Ternary +llvm::Value * nativeTernaryCast(llvm::IRBuilderBase & b, const ValueWithType & value_with_type); #endif @@ -113,7 +113,7 @@ struct AndImpl return builder.CreateAnd(a, b); } - static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) { return builder.CreateSelect(builder.CreateICmpUGT(a, b), b, a); } @@ -137,7 +137,7 @@ struct OrImpl return builder.CreateOr(a, b); } - static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) { return builder.CreateSelect(builder.CreateICmpUGT(a, b), a, b); } @@ -162,7 +162,7 @@ struct XorImpl return builder.CreateXor(a, b); } - static llvm::Value * tenaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) + static llvm::Value * ternaryApply(llvm::IRBuilder<> & builder, llvm::Value * a, llvm::Value * b) { llvm::Value * xor_result = builder.CreateXor(a, b); return builder.CreateSelect(xor_result, builder.getInt8(Ternary::True), builder.getInt8(Ternary::False)); @@ -251,16 +251,16 @@ public: else { /// First we need to cast all values to ternary logic - llvm::Value * tenary_result = nativeTenaryCast(b, values[0]); + llvm::Value * ternary_result = nativeTernaryCast(b, values[0]); for (size_t i = 1; i < values.size(); ++i) { - llvm::Value * casted_value = nativeTenaryCast(b, values[i]); - tenary_result = Impl::tenaryApply(b, tenary_result, casted_value); + llvm::Value * casted_value = nativeTernaryCast(b, values[i]); + ternary_result = Impl::ternaryApply(b, ternary_result, casted_value); } /// Then transform ternary logic to struct which represents nullable result - llvm::Value * is_null = b.CreateICmpEQ(tenary_result, b.getInt8(Ternary::Null)); - llvm::Value * is_true = b.CreateICmpEQ(tenary_result, b.getInt8(Ternary::True)); + llvm::Value * is_null = b.CreateICmpEQ(ternary_result, b.getInt8(Ternary::Null)); + llvm::Value * is_true = b.CreateICmpEQ(ternary_result, b.getInt8(Ternary::True)); auto * nullable_result_type = toNativeType(b, result_type); auto * nullable_result = llvm::Constant::getNullValue(nullable_result_type); From aae3cd8ad72b6781c34b31b5060cf0aed02deead Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 28 Oct 2024 08:52:10 +0000 Subject: [PATCH 080/433] Some fixups --- src/Interpreters/InterpreterSystemQuery.cpp | 80 ++++--------------- src/Interpreters/InterpreterSystemQuery.h | 2 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 5 +- .../03202_system_load_primary_key.reference | 16 ++-- .../03202_system_load_primary_key.sql | 48 +++++++---- 7 files changed, 65 insertions(+), 95 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 656deed53af..eb0430e8f7a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -1167,33 +1167,37 @@ void InterpreterSystemQuery::waitLoadingParts() void InterpreterSystemQuery::loadPrimaryKeys() +{ + loadOrUnloadPrimaryKeysImpl(true); +}; + +void InterpreterSystemQuery::unloadPrimaryKeys() +{ + loadOrUnloadPrimaryKeysImpl(false); +} + +void InterpreterSystemQuery::loadOrUnloadPrimaryKeysImpl(bool load) { if (!table_id.empty()) { - getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY, table_id.database_name, table_id.table_name); + getContext()->checkAccess(load ? AccessType::SYSTEM_LOAD_PRIMARY_KEY : AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, table_id.database_name, table_id.table_name); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (auto * merge_tree = dynamic_cast(table.get())) { - LOG_TRACE(log, "Loading primary keys for table {}", table_id.getFullTableName()); - try - { - merge_tree->loadPrimaryKeys(); - } - catch (const Exception & ex) - { - LOG_ERROR(log, "Failed to load primary keys for table {}: {}", table_id.getFullTableName(), ex.message()); - } + LOG_TRACE(log, "{} primary keys for table {}", load ? "Loading" : "Unloading", table_id.getFullTableName()); + load ? merge_tree->loadPrimaryKeys() : merge_tree->unloadPrimaryKeys(); } else { throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Command LOAD PRIMARY KEY is supported only for MergeTree tables, but got: {}", table->getName()); + ErrorCodes::BAD_ARGUMENTS, "Command {} PRIMARY KEY is supported only for MergeTree tables, but got: {}", load ? "LOAD" : "UNLOAD", table->getName()); } } else { - getContext()->checkAccess(AccessType::SYSTEM_LOAD_PRIMARY_KEY); + getContext()->checkAccess(load ? AccessType::SYSTEM_LOAD_PRIMARY_KEY : AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); + LOG_TRACE(log, "{} primary keys for all tables", load ? "Loading" : "Unloading"); for (auto & database : DatabaseCatalog::instance().getDatabases()) { @@ -1201,21 +1205,7 @@ void InterpreterSystemQuery::loadPrimaryKeys() { if (auto * merge_tree = dynamic_cast(it->table().get())) { - try - { - /// Calls the improved loadPrimaryKeys in MergeTreeData - merge_tree->loadPrimaryKeys(); - } - catch (const std::exception &ex) - { - LOG_ERROR(log, "Failed to load primary keys for table {}: {}", merge_tree->getStorageID().getFullTableName(), ex.what()); - throw; // Re-throw the exception to allow it to propagate - } - catch (...) - { - LOG_ERROR(log, "Failed to load primary keys for table {}: unknown exception occurred.", merge_tree->getStorageID().getFullTableName()); - throw; - } + load ? merge_tree->loadPrimaryKeys() : merge_tree->unloadPrimaryKeys(); } } } @@ -1223,42 +1213,6 @@ void InterpreterSystemQuery::loadPrimaryKeys() } -void InterpreterSystemQuery::unloadPrimaryKeys() -{ - if (!table_id.empty()) - { - getContext()->checkAccess(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY, table_id.database_name, table_id.table_name); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); - - if (auto * merge_tree = dynamic_cast(table.get())) - { - LOG_TRACE(log, "Unloading primary keys for table {}", table_id.getFullTableName()); - merge_tree->unloadPrimaryKeys(); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Command UNLOAD PRIMARY KEY is supported only for MergeTree table, but got: {}", table->getName()); - } - } - else - { - getContext()->checkAccess(AccessType::SYSTEM_UNLOAD_PRIMARY_KEY); - LOG_TRACE(log, "Unloading primary keys for all tables"); - - for (auto & database : DatabaseCatalog::instance().getDatabases()) - { - for (auto it = database.second->getTablesIterator(getContext()); it->isValid(); it->next()) - { - if (auto * merge_tree = dynamic_cast(it->table().get())) - { - merge_tree->unloadPrimaryKeys(); - } - } - } - } -} - void InterpreterSystemQuery::syncReplicatedDatabase(ASTSystemQuery & query) { const auto database_name = query.getDatabase(); diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 228567152f7..87c5aba0cb2 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -60,8 +60,10 @@ private: void syncReplica(ASTSystemQuery & query); void setReplicaReadiness(bool ready); void waitLoadingParts(); + void loadPrimaryKeys(); void unloadPrimaryKeys(); + void loadOrUnloadPrimaryKeysImpl(bool load); void syncReplicatedDatabase(ASTSystemQuery & query); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 195aa4fdc10..5055611db43 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -345,7 +345,7 @@ IMergeTreeDataPart::Index IMergeTreeDataPart::getIndex() const { std::scoped_lock lock(index_mutex); if (!index_loaded) - loadIndex(); + loadIndexLocked(); index_loaded = true; return index; } @@ -885,7 +885,7 @@ void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) co { } -void IMergeTreeDataPart::loadIndex() const +void IMergeTreeDataPart::loadIndexLocked() const { /// Memory for index must not be accounted as memory usage for query, because it belongs to a table. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a856e8a5dc8..95431a57226 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -371,8 +371,9 @@ public: Index getIndex() const; void setIndex(const Columns & cols_); void setIndex(Columns && cols_); - void loadIndex() const TSA_REQUIRES(index_mutex); - void loadIndexWithLock() const { std::scoped_lock lock(index_mutex); loadIndex(); } + + void loadIndex() const { std::scoped_lock lock(index_mutex); loadIndexLocked(); } + void loadIndexLocked() const TSA_REQUIRES(index_mutex); void unloadIndex(); bool isIndexLoaded() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 44973aaf89a..9b48cb891df 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8659,11 +8659,10 @@ bool MergeTreeData::initializeDiskOnConfigChange(const std::set & new_ad void MergeTreeData::loadPrimaryKeys() { - DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; - /// Thread pool to process parts within each table in parallel auto & thread_pool = DB::getActivePartsLoadingThreadPool().get(); + DataPartStates affordable_states = { MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::Deleting }; for (const auto & data_part : getDataParts(affordable_states)) { if (data_part->isProjectionPart()) @@ -8673,7 +8672,7 @@ void MergeTreeData::loadPrimaryKeys() { /// Use thread pool to parallelize part loading thread_pool.scheduleOrThrowOnError([data_part] { - const_cast(*data_part).loadIndexWithLock(); + const_cast(*data_part).loadIndex(); }); } } diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 76cf14f7c2c..eed5cd4c21b 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,11 +1,11 @@ -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 +tab1 0 0 +tab2 0 0 +tab1 0 0 +tab2 0 0 +tab1 0 0 +tab2 0 0 +tab1 0 0 +tab2 0 0 1 a 2 b 3 c diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 4974ea08105..57399eac351 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -1,14 +1,17 @@ -- Tags: no-parallel -DROP TABLE IF EXISTS test_load_primary_key; -DROP TABLE IF EXISTS test_load_primary_key_2; + +-- Tests statement SYSTEM LOAD PRIMARY KEY + +DROP TABLE IF EXISTS tab1; +DROP TABLE IF EXISTS tab2; -- Create the test tables -CREATE TABLE test_load_primary_key (id Int32, value String) ENGINE = MergeTree() ORDER BY id; -CREATE TABLE test_load_primary_key_2 (id Int32, value String) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab1 (id Int32, val String) ENGINE = MergeTree() ORDER BY id; +CREATE TABLE tab2 (id Int32, val String) ENGINE = MergeTree() ORDER BY id; -- Insert data into both tables -INSERT INTO test_load_primary_key VALUES (1, 'a'), (2, 'b'), (3, 'c'); -INSERT INTO test_load_primary_key_2 VALUES (1, 'x'), (2, 'y'), (3, 'z'); +INSERT INTO tab1 VALUES (1, 'a'), (2, 'b'), (3, 'c'); +INSERT INTO tab2 VALUES (1, 'x'), (2, 'y'), (3, 'z'); -- Check primary key memory before loading (this checks if it's not loaded yet) for both tables SELECT @@ -16,8 +19,10 @@ SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts -WHERE database = currentDatabase() -AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); +WHERE + database = currentDatabase() + AND table IN ('tab1', 'tab2') +ORDER BY table; -- Load primary keys for all tables in the database SYSTEM LOAD PRIMARY KEY; @@ -28,8 +33,10 @@ SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts -WHERE database = currentDatabase() -AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); +WHERE + database = currentDatabase() + AND table IN ('tab1', 'tab2') +ORDER BY table; -- Unload primary keys for all tables in the database SYSTEM UNLOAD PRIMARY KEY; @@ -40,11 +47,13 @@ SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts -WHERE database = currentDatabase() -AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); +WHERE + database = currentDatabase() + AND table IN ('tab1', 'tab2') +ORDER BY table; -- Load primary key for only one table -SYSTEM LOAD PRIMARY KEY test_load_primary_key; +SYSTEM LOAD PRIMARY KEY tab1; -- Verify that only one table's primary key is loaded SELECT @@ -52,9 +61,14 @@ SELECT round(primary_key_bytes_in_memory, -7), round(primary_key_bytes_in_memory_allocated, -7) FROM system.parts -WHERE database = currentDatabase() -AND table IN ('test_load_primary_key', 'test_load_primary_key_2'); +WHERE + database = currentDatabase() + AND table IN ('tab1', 'tab2') +ORDER BY table; -- Select to verify the data is correctly loaded for both tables -SELECT * FROM test_load_primary_key ORDER BY id; -SELECT * FROM test_load_primary_key_2 ORDER BY id; +SELECT * FROM tab1 ORDER BY id; +SELECT * FROM tab2 ORDER BY id; + +DROP TABLE tab1; +DROP TABLE tab2; From 1eef82e4329eb0a746c1a172346cb3d66823afcd Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Tue, 29 Oct 2024 17:10:37 -0700 Subject: [PATCH 081/433] Add sleep in the .sql file --- .../03202_system_load_primary_key.reference | 18 ++++++++++-------- .../03202_system_load_primary_key.sql | 2 ++ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index eed5cd4c21b..3f041ba7ff9 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,11 +1,13 @@ -tab1 0 0 -tab2 0 0 -tab1 0 0 -tab2 0 0 -tab1 0 0 -tab2 0 0 -tab1 0 0 -tab2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 +test_load_primary_key 10000000 10000000 +test_load_primary_key_2 10000000 10000000 +test_load_primary_key 0 0 +test_load_primary_key_2 0 0 1 a 2 b 3 c diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 57399eac351..7ccdd881465 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -26,6 +26,7 @@ ORDER BY table; -- Load primary keys for all tables in the database SYSTEM LOAD PRIMARY KEY; +SLEEP(1); -- Ensure enough time for the primary keys to load -- Verify primary key memory after loading for both tables SELECT @@ -40,6 +41,7 @@ ORDER BY table; -- Unload primary keys for all tables in the database SYSTEM UNLOAD PRIMARY KEY; +SLEEP(1); -- Ensure enough time for the primary keys to unload -- Verify primary key memory after unloading for both tables SELECT From ad0224209847b277a1d54a7a6d5a5a74b1924083 Mon Sep 17 00:00:00 2001 From: Zawa_ll Date: Tue, 29 Oct 2024 17:24:34 -0700 Subject: [PATCH 082/433] Remove sleep command --- .../03202_system_load_primary_key.reference | 30 +++++++++---------- .../03202_system_load_primary_key.sql | 3 +- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.reference b/tests/queries/0_stateless/03202_system_load_primary_key.reference index 3f041ba7ff9..84c18d66a7d 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.reference +++ b/tests/queries/0_stateless/03202_system_load_primary_key.reference @@ -1,16 +1,14 @@ -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -test_load_primary_key 10000000 10000000 -test_load_primary_key_2 10000000 10000000 -test_load_primary_key 0 0 -test_load_primary_key_2 0 0 -1 a -2 b -3 c -1 x -2 y -3 z +tab1 0 0 +tab2 0 0 +tab1 0 0 +tab2 0 0 +tab1 10000000 10000000 -- Expected non-zero memory after loading primary keys +tab2 10000000 10000000 +tab1 0 0 +tab2 0 0 +1 a +2 b +3 c +1 x +2 y +3 z diff --git a/tests/queries/0_stateless/03202_system_load_primary_key.sql b/tests/queries/0_stateless/03202_system_load_primary_key.sql index 7ccdd881465..0946531a54c 100644 --- a/tests/queries/0_stateless/03202_system_load_primary_key.sql +++ b/tests/queries/0_stateless/03202_system_load_primary_key.sql @@ -26,9 +26,9 @@ ORDER BY table; -- Load primary keys for all tables in the database SYSTEM LOAD PRIMARY KEY; -SLEEP(1); -- Ensure enough time for the primary keys to load -- Verify primary key memory after loading for both tables +-- Ensure .reference file has non-zero values here to reflect expected primary key loading SELECT table, round(primary_key_bytes_in_memory, -7), @@ -41,7 +41,6 @@ ORDER BY table; -- Unload primary keys for all tables in the database SYSTEM UNLOAD PRIMARY KEY; -SLEEP(1); -- Ensure enough time for the primary keys to unload -- Verify primary key memory after unloading for both tables SELECT From c7318b60da884d56550a53329beac7a18fd1beaa Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 19:43:38 -0300 Subject: [PATCH 083/433] demand changes on alter user --- src/Parsers/Access/ParserCreateUserQuery.cpp | 9 +++++++++ .../03254_test_alter_user_no_changes.reference | 0 .../0_stateless/03254_test_alter_user_no_changes.sql | 4 ++++ 3 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/03254_test_alter_user_no_changes.reference create mode 100644 tests/queries/0_stateless/03254_test_alter_user_no_changes.sql diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 8bfc84a28a6..f02c0cbead1 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -545,6 +545,8 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec auto names = typeid_cast>(names_ast); auto names_ref = names->names; + auto pos_after_parsing_names = pos; + std::optional new_name; std::optional hosts; std::optional add_hosts; @@ -655,6 +657,13 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec else if (alter) names->concatParts(); + bool alter_command_with_no_changes = alter && pos_after_parsing_names == pos; + + if (alter_command_with_no_changes) + { + return false; + } + auto query = std::make_shared(); node = query; diff --git a/tests/queries/0_stateless/03254_test_alter_user_no_changes.reference b/tests/queries/0_stateless/03254_test_alter_user_no_changes.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql b/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql new file mode 100644 index 00000000000..4fc6ae465cb --- /dev/null +++ b/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel + +create user u_03254_alter_user; +alter user u_03254_alter_user; -- { clientError SYNTAX_ERROR } From ffc88fd8a32fc183605d409c1028d865adca6cef Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Oct 2024 21:53:53 -0300 Subject: [PATCH 084/433] comments --- src/Parsers/Access/ParserCreateUserQuery.cpp | 4 ++-- .../queries/0_stateless/03254_test_alter_user_no_changes.sql | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index f02c0cbead1..3ef917a38db 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -657,9 +657,9 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec else if (alter) names->concatParts(); - bool alter_command_with_no_changes = alter && pos_after_parsing_names == pos; + bool alter_query_with_no_changes = alter && pos_after_parsing_names == pos; - if (alter_command_with_no_changes) + if (alter_query_with_no_changes) { return false; } diff --git a/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql b/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql index 4fc6ae465cb..b71930eeaf5 100644 --- a/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql +++ b/tests/queries/0_stateless/03254_test_alter_user_no_changes.sql @@ -2,3 +2,4 @@ create user u_03254_alter_user; alter user u_03254_alter_user; -- { clientError SYNTAX_ERROR } +drop user u_03254_alter_user; From f46a6fabd19aeb2c8714510601c5980b33182666 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 31 Oct 2024 16:36:53 +0100 Subject: [PATCH 085/433] Fix review issues. --- src/Common/computeMaxTableNameLength.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 6 ++++-- src/Functions/getMaxTableNameLength.cpp | 2 -- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/computeMaxTableNameLength.cpp b/src/Common/computeMaxTableNameLength.cpp index eb4a72f1863..f74db295bd2 100644 --- a/src/Common/computeMaxTableNameLength.cpp +++ b/src/Common/computeMaxTableNameLength.cpp @@ -23,7 +23,7 @@ size_t computeMaxTableNameLength(const String & database_name, ContextPtr contex size_t escaped_db_name_length = escapeForFileName(database_name).length(); const size_t uuid_length = 36; // Standard UUID length - const size_t extension_length = 6; // Length of ".sql" including three dots + const size_t extension_length = strlen(".sql"); // Length of ".sql" // Adjust for database name and UUID in dropped table filenames size_t max_to_drop = max_dropped_length - escaped_db_name_length - uuid_length - extension_length; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 26486c5fdb1..e16da0ac49a 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -393,10 +393,12 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to size_t allowed_max_length = computeMaxTableNameLength(database_name, getContext()); String table_metadata_path = getObjectMetadataPath(to_table_name); - if (escapeForFileName(to_table_name).length() > allowed_max_length) + const auto escaped_name_length = escapeForFileName(to_table_name).length(); + + if (escaped_length > allowed_max_length) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The max length of table name for database {} is {}, current length is {}", - database_name, allowed_max_length, to_table_name.length()); + database_name, allowed_max_length, escaped_length); if (fs::exists(table_metadata_path)) { diff --git a/src/Functions/getMaxTableNameLength.cpp b/src/Functions/getMaxTableNameLength.cpp index 263fa2f2919..5b84f88c1a9 100644 --- a/src/Functions/getMaxTableNameLength.cpp +++ b/src/Functions/getMaxTableNameLength.cpp @@ -56,8 +56,6 @@ public: return std::make_shared(); } - bool isDeterministic() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override From 2ed30407f800339d31193c70fe3aeb9de7d9b96e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 31 Oct 2024 19:07:45 +0100 Subject: [PATCH 086/433] Fix undeclared variable issue. --- src/Databases/DatabaseOnDisk.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index e16da0ac49a..28f6ed990c0 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -395,10 +395,10 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to const auto escaped_name_length = escapeForFileName(to_table_name).length(); - if (escaped_length > allowed_max_length) + if (escaped_name_length > allowed_max_length) throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The max length of table name for database {} is {}, current length is {}", - database_name, allowed_max_length, escaped_length); + database_name, allowed_max_length, escaped_name_length); if (fs::exists(table_metadata_path)) { From 3c5343b6c8ede95ea03a16bf23621e4c584e3ae2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Nov 2024 18:19:36 +0100 Subject: [PATCH 087/433] Debug commit. --- src/Common/computeMaxTableNameLength.cpp | 2 +- .../0_stateless/03167_improvement_table_name_too_long.sh | 2 ++ tests/queries/0_stateless/03253_getMaxTableNameLength.reference | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/computeMaxTableNameLength.cpp b/src/Common/computeMaxTableNameLength.cpp index f74db295bd2..0f5976569e5 100644 --- a/src/Common/computeMaxTableNameLength.cpp +++ b/src/Common/computeMaxTableNameLength.cpp @@ -23,7 +23,7 @@ size_t computeMaxTableNameLength(const String & database_name, ContextPtr contex size_t escaped_db_name_length = escapeForFileName(database_name).length(); const size_t uuid_length = 36; // Standard UUID length - const size_t extension_length = strlen(".sql"); // Length of ".sql" + const size_t extension_length = strlen(".sql"); // Adjust for database name and UUID in dropped table filenames size_t max_to_drop = max_dropped_length - escaped_db_name_length - uuid_length - extension_length; diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 9d52179c458..56bb3a21a68 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -13,3 +13,5 @@ allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" + +getconf NAME_MAX /fasttest-workspace/db-fasttest/store diff --git a/tests/queries/0_stateless/03253_getMaxTableNameLength.reference b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference index 560a78db396..0e1d238cd78 100644 --- a/tests/queries/0_stateless/03253_getMaxTableNameLength.reference +++ b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference @@ -1,2 +1,2 @@ +208 206 -204 From 0d4f66b8257d8a81e5d7a996e6ec1fee6832e31b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 6 Nov 2024 18:47:49 +0100 Subject: [PATCH 088/433] Move debug line. --- .../0_stateless/03167_improvement_table_name_too_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 56bb3a21a68..48665c92a4e 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -7,11 +7,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") let excess_length=allowed_name_length+1 +getconf NAME_MAX /fasttest-workspace/db-fasttest/store + long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" - -getconf NAME_MAX /fasttest-workspace/db-fasttest/store From 0fb51df9ab43e4518e3b03e99461a479c8870f43 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Nov 2024 12:16:19 +0000 Subject: [PATCH 089/433] Revert "Merge pull request #71527 from ClickHouse/revert-68682-vdimir/join_select_inner_table" This reverts commit dd5061a4d31fc3e7ed110424583257510719845c, reversing changes made to 0011bc5e0786ce9ae7c1f88bf4af0fcc26ace9e7. --- src/Core/Joins.h | 11 + src/Core/Settings.cpp | 3 + src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 2 +- src/Interpreters/ConcurrentHashJoin.h | 11 + src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 16 +- src/Interpreters/HashJoin/HashJoin.h | 5 +- .../HashJoin/HashJoinMethodsImpl.h | 18 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Interpreters/TableJoin.cpp | 56 ++++- src/Interpreters/TableJoin.h | 19 +- src/Interpreters/TreeRewriter.cpp | 5 +- src/Parsers/CreateQueryUUIDs.cpp | 2 +- src/Planner/CollectColumnIdentifiers.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 140 ++++++++---- src/Processors/QueryPlan/JoinStep.cpp | 103 ++++++++- src/Processors/QueryPlan/JoinStep.h | 17 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../QueryPlan/Optimizations/optimizeJoin.cpp | 102 +++++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 3 + .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + .../Transforms/ColumnPermuteTransform.cpp | 49 +++++ .../Transforms/ColumnPermuteTransform.h | 30 +++ .../Transforms/JoiningTransform.cpp | 1 + tests/clickhouse-test | 4 + tests/integration/helpers/cluster.py | 13 +- tests/integration/helpers/random_settings.py | 2 + .../test_peak_memory_usage/test.py | 2 +- .../0_stateless/00826_cross_to_inner_join.sql | 13 +- .../00847_multiple_join_same_column.sql | 14 +- .../01015_empty_in_inner_right_join.sql.j2 | 2 + .../01107_join_right_table_totals.reference | 7 + .../01107_join_right_table_totals.sql | 10 +- .../01763_filter_push_down_bugs.reference | 2 +- .../01881_join_on_conditions_hash.sql.j2 | 10 +- .../0_stateless/02000_join_on_const.reference | 18 +- .../0_stateless/02000_join_on_const.sql | 16 +- .../02001_join_on_const_bs_long.sql.j2 | 4 +- ...oin_with_nullable_lowcardinality_crash.sql | 5 +- .../0_stateless/02282_array_distance.sql | 12 +- .../02381_join_dup_columns_in_plan.reference | 1 - .../0_stateless/02461_join_lc_issue_42380.sql | 3 +- ...emove_redundant_sorting_analyzer.reference | 4 +- ...move_redundant_distinct_analyzer.reference | 18 +- .../02514_analyzer_drop_join_on.reference | 55 ++--- .../02514_analyzer_drop_join_on.sql | 1 + ...oin_with_totals_and_subquery_bug.reference | 2 +- .../02835_join_step_explain.reference | 32 ++- .../0_stateless/02835_join_step_explain.sql | 2 + .../02962_join_using_bug_57894.reference | 1 + .../02962_join_using_bug_57894.sql | 2 + ...filter_push_down_equivalent_sets.reference | 206 ++++++++++-------- ..._join_filter_push_down_equivalent_sets.sql | 40 +++- .../03038_recursive_cte_postgres_4.reference | 4 +- .../03038_recursive_cte_postgres_4.sql | 4 +- .../0_stateless/03094_one_thousand_joins.sql | 1 + ...convert_outer_join_to_inner_join.reference | 36 +-- ...03130_convert_outer_join_to_inner_join.sql | 13 +- ...ter_push_down_equivalent_columns.reference | 3 +- .../03236_squashing_high_memory.sql | 1 + 63 files changed, 864 insertions(+), 308 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.cpp create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.h diff --git a/src/Core/Joins.h b/src/Core/Joins.h index 0964bf86e6b..dd6d86fc902 100644 --- a/src/Core/Joins.h +++ b/src/Core/Joins.h @@ -119,4 +119,15 @@ enum class JoinTableSide : uint8_t const char * toString(JoinTableSide join_table_side); +/// Setting to choose which table to use as the inner table in hash join +enum class JoinInnerTableSelectionMode : uint8_t +{ + /// Use left table + Left, + /// Use right table + Right, + /// Use the table with the smallest number of rows + Auto, +}; + } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index c2ffc2ddf0e..7e8d0aabce0 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1912,6 +1912,9 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ + DECLARE(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, R"( +Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'. +)", 0) \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. )", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ac3b1fe651e..1cc58deb94a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -66,6 +66,7 @@ class WriteBuffer; M(CLASS_NAME, IntervalOutputFormat) \ M(CLASS_NAME, JoinAlgorithm) \ M(CLASS_NAME, JoinStrictness) \ + M(CLASS_NAME, JoinInnerTableSelectionMode) \ M(CLASS_NAME, LightweightMutationProjectionMode) \ M(CLASS_NAME, LoadBalancing) \ M(CLASS_NAME, LocalFSReadMethod) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 64964f294bd..ed87fde8b7e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -73,6 +73,7 @@ static std::initializer_list clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override + { + return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); + } + private: struct InternalHashJoin { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 3f1e0d59287..faa9114c618 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals(); } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 3e7f3deea8b..dad8a487745 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -383,6 +383,16 @@ size_t HashJoin::getTotalByteCount() const return res; } +bool HashJoin::isUsedByAnotherAlgorithm() const +{ + return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH); +} + +bool HashJoin::canRemoveColumnsFromLeftBlock() const +{ + return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm(); +} + void HashJoin::initRightBlockStructure(Block & saved_block_sample) { if (isCrossOrComma(kind)) @@ -394,8 +404,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || - table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || + bool save_key_columns = isUsedByAnotherAlgorithm() || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression(); @@ -1228,7 +1237,10 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, { if (!JoinCommon::hasNonJoinedBlocks(*table_join)) return {}; + size_t left_columns_count = left_sample_block.columns(); + if (canRemoveColumnsFromLeftBlock()) + left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); if (!flag_per_row) diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 4c1ebbcdc66..8a27961354a 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -127,7 +127,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals() && getTotalRowCount() == 0; } std::shared_ptr clone(const std::shared_ptr & table_join_, @@ -464,6 +464,9 @@ private: bool empty() const; + bool isUsedByAnotherAlgorithm() const; + bool canRemoveColumnsFromLeftBlock() const; + void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 45a766e2df6..7e8a2658b9c 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -56,7 +56,6 @@ Block HashJoinMethods::joinBlockImpl( const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - size_t existing_columns = block.columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -99,6 +98,22 @@ Block HashJoinMethods::joinBlockImpl( added_columns.buildJoinGetOutput(); else added_columns.buildOutput(); + + const auto & table_join = join.table_join; + std::set block_columns_to_erase; + if (join.canRemoveColumnsFromLeftBlock()) + { + std::unordered_set left_output_columns; + for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) + left_output_columns.insert(out_column.name); + for (size_t i = 0; i < block.columns(); ++i) + { + if (!left_output_columns.contains(block.getByPosition(i).name)) + block_columns_to_erase.insert(i); + } + } + size_t existing_columns = block.columns(); + for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); @@ -160,6 +175,7 @@ Block HashJoinMethods::joinBlockImpl( block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); } } + block.erase(block_columns_to_erase); return remaining_block; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3918c1c37ea..8ddf51fa25e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1888,7 +1888,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index e1b12df6b25..e39fdff93e0 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -41,7 +41,6 @@ namespace DB namespace Setting { extern const SettingsBool allow_experimental_join_right_table_sorting; - extern const SettingsBool allow_experimental_analyzer; extern const SettingsUInt64 cross_join_min_bytes_to_compress; extern const SettingsUInt64 cross_join_min_rows_to_compress; extern const SettingsUInt64 default_max_bytes_in_join; @@ -144,7 +143,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_memory_usage(settings[Setting::max_memory_usage]) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) - , enable_analyzer(settings[Setting::allow_experimental_analyzer]) { } @@ -163,8 +161,6 @@ void TableJoin::resetCollected() clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); - columns_from_left_table.clear(); - result_columns_from_left_table.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); @@ -207,20 +203,6 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return count; } -void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) -{ - columns_from_left_table = std::move(left_output_columns); - columns_from_joined_table = std::move(right_output_columns); -} - - -const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side) -{ - if (side == JoinTableSide::Left) - return result_columns_from_left_table; - return columns_added_by_join; -} - void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix) { NameSet joined_columns; @@ -369,18 +351,9 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const return forceNullableRight() && JoinCommon::canBecomeNullable(column_type); } -void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side) -{ - if (side == JoinTableSide::Left) - result_columns_from_left_table.push_back(joined_column); - else - columns_added_by_join.push_back(joined_column); - -} - void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - setUsedColumn(joined_column, JoinTableSide::Right); + columns_added_by_join.emplace_back(joined_column); } NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const @@ -1022,32 +995,5 @@ size_t TableJoin::getMaxMemoryUsage() const return max_memory_usage; } -void TableJoin::swapSides() -{ - assertEnableEnalyzer(); - - std::swap(key_asts_left, key_asts_right); - std::swap(left_type_map, right_type_map); - for (auto & clause : clauses) - { - std::swap(clause.key_names_left, clause.key_names_right); - std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right); - std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name); - } - - std::swap(columns_from_left_table, columns_from_joined_table); - std::swap(result_columns_from_left_table, columns_added_by_join); - - if (table_join.kind == JoinKind::Left) - table_join.kind = JoinKind::Right; - else if (table_join.kind == JoinKind::Right) - table_join.kind = JoinKind::Left; -} - -void TableJoin::assertEnableEnalyzer() const -{ - if (!enable_analyzer) - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled"); -} } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 146230f44be..4ecbc9eb960 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -172,9 +172,6 @@ private: ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals; - NamesAndTypesList columns_from_left_table; - NamesAndTypesList result_columns_from_left_table; - /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; /// Columns will be added to block by JOIN. @@ -210,8 +207,6 @@ private: bool is_join_with_constant = false; - bool enable_analyzer = false; - Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -275,8 +270,6 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } - bool enableEnalyzer() const { return enable_analyzer; } - void assertEnableEnalyzer() const; TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data ? tmp_data->childScope(CurrentMetrics::TemporaryFilesForJoin) : nullptr; } ActionsDAG createJoinedBlockActions(ContextPtr context) const; @@ -294,7 +287,6 @@ public: } bool allowParallelHashJoin() const; - void swapSides(); bool joinUseNulls() const { return join_use_nulls; } @@ -385,9 +377,6 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); - - void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side); - void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) { columns_added_by_join = columns_added_by_join_value; @@ -413,17 +402,11 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_) + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) { columns_from_joined_table = std::move(columns_from_joined_table_value); deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); - result_columns_from_left_table = columns_from_left_table_; - columns_from_left_table = columns_from_left_table_; } - - void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns); - const NamesAndTypesList & getOutputColumns(JoinTableSide side); - const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index d8478cc8580..16b0e7ef199 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1353,15 +1353,12 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { - auto columns_from_left_table = tables_with_columns[0].columns; const auto & right_table = tables_with_columns[1]; auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end()); - result.analyzed_join->setColumnsFromJoinedTable( - std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table); + result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index 70848440a0e..14cf5761a11 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// Thus it's not safe for example to replace /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with - /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b" + /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "248372b7-02c4-4c88-a5e1-282a83cc572a" AS SELECT a FROM b" /// This replacement is safe only for CREATE queries when inner target tables don't exist yet. if (!query.attach) { diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index dd5bdd4d141..95f1c7d53d8 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -2,7 +2,6 @@ #include #include -#include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f8ec1537820..0107df61c22 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,7 +104,6 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; - extern const SettingsBoolAuto query_plan_join_swap_table; extern const SettingsUInt64 min_joined_block_size_bytes; } @@ -1269,55 +1268,6 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } -std::optional createStepToDropColumns( - const Block & header, - const ColumnIdentifierSet & outer_scope_columns, - const PlannerContextPtr & planner_context) -{ - ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; - std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; - std::optional first_skipped_column_node_index; - - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); - size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); - - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - - for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) - { - const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; - - if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) - || !global_planner_context->hasColumnIdentifier(output->result_name)) - continue; - - if (!outer_scope_columns.contains(output->result_name)) - { - if (!first_skipped_column_node_index) - first_skipped_column_node_index = i; - continue; - } - - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); - drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); - } - - if (!first_skipped_column_node_index) - return {}; - - /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. - * - * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; - */ - if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); - - drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); - - return drop_unused_columns_after_join_actions_dag; -} - JoinTreeQueryPlan buildQueryPlanForJoinNode( const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, @@ -1592,48 +1542,24 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( } const Block & left_header = left_plan.getCurrentHeader(); + auto left_table_names = left_header.getNames(); + NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + + auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList(); + table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + + for (auto & column_from_joined_table : columns_from_joined_table) + { + /// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + outer_scope_columns.contains(column_from_joined_table.name)) + table_join->addJoinedColumn(column_from_joined_table); + } + const Block & right_header = right_plan.getCurrentHeader(); + auto join_algorithm = chooseJoinAlgorithm( + table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info); - auto columns_from_left_table = left_header.getNamesAndTypesList(); - auto columns_from_right_table = right_header.getNamesAndTypesList(); - - table_join->setInputColumns(columns_from_left_table, columns_from_right_table); - - for (auto & column_from_joined_table : columns_from_left_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); - } - - for (auto & column_from_joined_table : columns_from_right_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); - } - - - if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) - { - /// We should add all duplicated columns, because join algorithm add either all column with specified name or none - auto set_used_column_with_duplicates = [&](const NamesAndTypesList & columns, JoinTableSide join_table_side) - { - const auto & column_name = columns.front().name; - for (const auto & column : columns) - if (column.name == column_name) - table_join->setUsedColumn(column, join_table_side); - }; - - if (!columns_from_left_table.empty()) - set_used_column_with_duplicates(columns_from_left_table, JoinTableSide::Left); - else if (!columns_from_right_table.empty()) - set_used_column_with_duplicates(columns_from_right_table, JoinTableSide::Right); - } - - auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info); auto result_plan = QueryPlan(); bool is_filled_join = join_algorithm->isFilled(); @@ -1719,16 +1645,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( } auto join_pipeline_type = join_algorithm->pipelineType(); - - ColumnIdentifierSet outer_scope_columns_nonempty; - if (outer_scope_columns.empty()) - { - if (left_header.columns() > 1) - outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); - else if (right_header.columns() > 1) - outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); - } - auto join_step = std::make_unique( left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), @@ -1736,11 +1652,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( settings[Setting::max_block_size], settings[Setting::min_joined_block_size_bytes], settings[Setting::max_threads], - outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, - false /*optimize_read_in_order*/, - true /*optimize_skip_unused_shards*/); - - join_step->swap_join_tables = settings[Setting::query_plan_join_swap_table].get(); + false /*optimize_read_in_order*/); join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); @@ -1751,18 +1663,47 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - const auto & header_after_join = result_plan.getCurrentHeader(); - if (header_after_join.columns() > outer_scope_columns.size()) + ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; + std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; + std::optional first_skipped_column_node_index; + + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); + size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); + + for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) { - auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context); - if (drop_unused_columns_after_join_actions_dag) + const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; + + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) + || !global_planner_context->hasColumnIdentifier(output->result_name)) + continue; + + if (!outer_scope_columns.contains(output->result_name)) { - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + if (!first_skipped_column_node_index) + first_skipped_column_node_index = i; + continue; } + + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); + drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); } + /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. + * + * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; + */ + if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); + + drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); + + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); + for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 33b85e8af54..d81ce2fda17 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -38,37 +37,6 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } -std::vector getPermutationForBlock( - const Block & block, - const Block & lhs_block, - const Block & rhs_block, - const NameSet & name_filter) -{ - std::vector permutation; - permutation.reserve(block.columns()); - Block::NameMap name_map = block.getNamesToIndexesMap(); - - bool is_trivial = true; - for (const auto & other_block : {lhs_block, rhs_block}) - { - for (const auto & col : other_block) - { - if (!name_filter.contains(col.name)) - continue; - if (auto it = name_map.find(col.name); it != name_map.end()) - { - is_trivial = is_trivial && it->second == permutation.size(); - permutation.push_back(it->second); - } - } - } - - if (is_trivial && permutation.size() == block.columns()) - return {}; - - return permutation; -} - } JoinStep::JoinStep( @@ -78,16 +46,12 @@ JoinStep::JoinStep( size_t max_block_size_, size_t min_block_size_bytes_, size_t max_streams_, - NameSet required_output_, - bool keep_left_read_in_order_, - bool use_new_analyzer_) + bool keep_left_read_in_order_) : join(std::move(join_)) , max_block_size(max_block_size_) , min_block_size_bytes(min_block_size_bytes_) , max_streams(max_streams_) - , required_output(std::move(required_output_)) , keep_left_read_in_order(keep_left_read_in_order_) - , use_new_analyzer(use_new_analyzer_) { updateInputHeaders({left_header_, right_header_}); } @@ -97,52 +61,32 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); - Block lhs_header = pipelines[0]->getHeader(); - Block rhs_header = pipelines[1]->getHeader(); - - if (swap_streams) - std::swap(pipelines[0], pipelines[1]); - - std::unique_ptr joined_pipeline; if (join->pipelineType() == JoinPipelineType::YShaped) { - joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); + auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( + std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors); joined_pipeline->resize(max_streams); - } - else - { - joined_pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( - std::move(pipelines[0]), - std::move(pipelines[1]), - join, - join_algorithm_header, - max_block_size, - min_block_size_bytes, - max_streams, - keep_left_read_in_order, - &processors); - } - - if (!use_new_analyzer) return joined_pipeline; - - auto column_permutation = getPermutationForBlock(joined_pipeline->getHeader(), lhs_header, rhs_header, required_output); - if (!column_permutation.empty()) - { - joined_pipeline->addSimpleTransform([&column_permutation](const Block & header) - { - return std::make_shared(header, column_permutation); - }); } + auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( + std::move(pipelines[0]), + std::move(pipelines[1]), + join, + *output_header, + max_block_size, + min_block_size_bytes, + max_streams, + keep_left_read_in_order, + &processors); + if (join->supportParallelJoin()) { - joined_pipeline->addSimpleTransform([&](const Block & header) + pipeline->addSimpleTransform([&](const Block & header) { return std::make_shared(header, 0, min_block_size_bytes); }); } - return joined_pipeline; + return pipeline; } bool JoinStep::allowPushDownToRight() const @@ -161,49 +105,17 @@ void JoinStep::describeActions(FormatSettings & settings) const for (const auto & [name, value] : describeJoinActions(join)) settings.out << prefix << name << ": " << value << '\n'; - if (swap_streams) - settings.out << prefix << "Swapped: true\n"; } void JoinStep::describeActions(JSONBuilder::JSONMap & map) const { for (const auto & [name, value] : describeJoinActions(join)) map.add(name, value); - if (swap_streams) - map.add("Swapped", true); -} - -void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) -{ - join_algorithm_header.clear(); - swap_streams = swap_streams_; - join = std::move(join_); - updateOutputHeader(); } void JoinStep::updateOutputHeader() { - if (join_algorithm_header) - return; - - const auto & header = swap_streams ? input_headers[1] : input_headers[0]; - - Block result_header = JoiningTransform::transformHeader(header, join); - join_algorithm_header = result_header; - - if (!use_new_analyzer) - { - if (swap_streams) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer"); - output_header = result_header; - return; - } - - auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output); - if (!column_permutation.empty()) - result_header = ColumnPermuteTransform::permute(result_header, column_permutation); - - output_header = result_header; + output_header = JoiningTransform::transformHeader(input_headers.front(), join); } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 892ecfadc7d..bc9b7600510 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { @@ -21,9 +20,7 @@ public: size_t max_block_size_, size_t min_block_size_bytes_, size_t max_streams_, - NameSet required_output_, - bool keep_left_read_in_order_, - bool use_new_analyzer_); + bool keep_left_read_in_order_); String getName() const override { return "Join"; } @@ -35,28 +32,17 @@ public: void describeActions(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } - void setJoin(JoinPtr join_, bool swap_streams_ = false); + void setJoin(JoinPtr join_) { join = std::move(join_); } bool allowPushDownToRight() const; - /// Swap automatically if not set, otherwise always or never, depending on the value - std::optional swap_join_tables = false; - private: void updateOutputHeader() override; - /// Header that expected to be returned from IJoin - Block join_algorithm_header; - JoinPtr join; size_t max_block_size; size_t min_block_size_bytes; size_t max_streams; - - const NameSet required_output; - std::set columns_to_remove; bool keep_left_read_in_order; - bool use_new_analyzer = false; - bool swap_streams = false; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index c1c4d1e1635..751d5182dc3 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -113,7 +113,6 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &); void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); /// A separate tree traverse to apply sorting properties after *InOrder optimizations. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp deleted file mode 100644 index ca858559886..00000000000 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ /dev/null @@ -1,103 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - -namespace DB::QueryPlanOptimizations -{ - -static std::optional estimateReadRowsCount(QueryPlan::Node & node) -{ - IQueryPlanStep * step = node.step.get(); - if (const auto * reading = typeid_cast(step)) - { - if (auto analyzed_result = reading->getAnalyzedResult()) - return analyzed_result->selected_rows; - if (auto analyzed_result = reading->selectRangesToRead()) - return analyzed_result->selected_rows; - return {}; - } - - if (const auto * reading = typeid_cast(step)) - return reading->getStorage()->totalRows(Settings{}); - - if (node.children.size() != 1) - return {}; - - if (typeid_cast(step) || typeid_cast(step)) - return estimateReadRowsCount(*node.children.front()); - - return {}; -} - -void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) -{ - auto * join_step = typeid_cast(node.step.get()); - if (!join_step || node.children.size() != 2) - return; - - const auto & join = join_step->getJoin(); - if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported()) - return; - - const auto & table_join = join->getTableJoin(); - - /// Algorithms other than HashJoin may not support all JOIN kinds, so changing from LEFT to RIGHT is not always possible - bool allow_outer_join = typeid_cast(join.get()); - if (table_join.kind() != JoinKind::Inner && !allow_outer_join) - return; - - /// fixme: USING clause handled specially in join algorithm, so swap breaks it - /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test - if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) - return; - - bool need_swap = false; - if (!join_step->swap_join_tables.has_value()) - { - auto lhs_extimation = estimateReadRowsCount(*node.children[0]); - auto rhs_extimation = estimateReadRowsCount(*node.children[1]); - LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}", - lhs_extimation.transform(toString).value_or("unknown"), - rhs_extimation.transform(toString).value_or("unknown")); - - if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) - need_swap = true; - } - else if (join_step->swap_join_tables.value()) - { - need_swap = true; - } - - if (!need_swap) - return; - - const auto & headers = join_step->getInputHeaders(); - if (headers.size() != 2) - return; - - const auto & left_stream_input_header = headers.front(); - const auto & right_stream_input_header = headers.back(); - - auto updated_table_join = std::make_shared(table_join); - updated_table_join->swapSides(); - auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); - join_step->setJoin(std::move(updated_join), /* swap_streams= */ true); -} - -} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index c034ca79181..03418c752d4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -227,9 +227,6 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); - if (frame.next_child == 0) - optimizeJoin(*frame.node, nodes); - /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index a9c2d2df2c4..238c1a3aad0 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -35,8 +35,6 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - const StoragePtr & getStorage() const { return storage; } - private: static constexpr auto name = "ReadFromMemoryStorage"; diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp deleted file mode 100644 index f371689814c..00000000000 --- a/src/Processors/Transforms/ColumnPermuteTransform.cpp +++ /dev/null @@ -1,49 +0,0 @@ -#include - -namespace DB -{ - -namespace -{ - -template -void applyPermutation(std::vector & data, const std::vector & permutation) -{ - std::vector res; - res.reserve(permutation.size()); - for (size_t i : permutation) - res.push_back(data[i]); - data = std::move(res); -} - -void permuteChunk(Chunk & chunk, const std::vector & permutation) -{ - size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - applyPermutation(columns, permutation); - chunk.setColumns(std::move(columns), num_rows); -} - -} - -Block ColumnPermuteTransform::permute(const Block & block, const std::vector & permutation) -{ - auto columns = block.getColumnsWithTypeAndName(); - applyPermutation(columns, permutation); - return Block(columns); -} - -ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) - : ISimpleTransform(header_, permute(header_, permutation_), false) - , permutation(permutation_) -{ -} - - -void ColumnPermuteTransform::transform(Chunk & chunk) -{ - permuteChunk(chunk, permutation); -} - - -} diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h deleted file mode 100644 index 25f3a8d0825..00000000000 --- a/src/Processors/Transforms/ColumnPermuteTransform.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class ColumnPermuteTransform : public ISimpleTransform -{ -public: - ColumnPermuteTransform(const Block & header_, const std::vector & permutation_); - - String getName() const override { return "ColumnPermuteTransform"; } - - void transform(Chunk & chunk) override; - - static Block permute(const Block & block, const std::vector & permutation); - -private: - Names column_names; - std::vector permutation; -}; - - -} diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 992d5eca77a..2862575b541 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -19,7 +19,6 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); - materializeBlockInplace(header); LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index d43a2fa99b7..982a347a70f 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -101,7 +101,6 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() return Status(0); if (last_used_row[1] >= chunks[1].getNumRows()) return Status(1); - /// We have unused rows from both inputs size_t result_num_rows = std::min(chunks[0].getNumRows() - last_used_row[0], chunks[1].getNumRows() - last_used_row[1]); @@ -111,7 +110,6 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() result.addColumn(col->cut(last_used_row[source_num], result_num_rows)); last_used_row[0] += result_num_rows; last_used_row[1] += result_num_rows; - return Status(std::move(result)); } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 084e63145fd..274232f0985 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -789,7 +789,6 @@ def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) -# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { "max_insert_threads": lambda: ( @@ -920,7 +919,6 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), - "query_plan_join_swap_table": lambda: random.choice(["auto", "false", "true"]), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 466c79444c0..dd2a56e2e6b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -68,7 +68,6 @@ DEFAULT_ENV_NAME = ".env" DEFAULT_BASE_CONFIG_DIR = os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" ) -DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest") SANITIZER_SIGN = "==================" @@ -505,6 +504,7 @@ class ClickHouseCluster: "CLICKHOUSE_TESTS_DOCKERD_HOST" ) self.docker_api_version = os.environ.get("DOCKER_API_VERSION") + self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") self.base_cmd = ["docker", "compose"] if custom_dockerd_host: @@ -1082,7 +1082,7 @@ class ClickHouseCluster: env_variables["keeper_binary"] = binary_path env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix - env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG + env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag env_variables["user"] = str(os.getuid()) env_variables["keeper_fs"] = "bind" for i in range(1, 4): @@ -1681,7 +1681,7 @@ class ClickHouseCluster: ) if tag is None: - tag = DOCKER_BASE_TAG + tag = self.docker_base_tag if not env_variables: env_variables = {} self.use_keeper = use_keeper @@ -4619,12 +4619,7 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if ( - self.randomize_settings - and self.image == "clickhouse/integration-test" - and self.tag == DOCKER_BASE_TAG - and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR - ): + if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index f89b2fd6870..b2319561fd7 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -5,8 +5,6 @@ def randomize_settings(): yield "max_joined_block_size_rows", random.randint(8000, 100000) if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) - if random.random() < 0.5: - yield "query_plan_join_swap_table", random.choice(["auto", "true", "false"]) def write_random_settings_config(destination): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index f447c527d9b..51268dcf386 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_swap_table = 'false'", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index 5ab7a2d0626..e9f9e13e2d3 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,15 +41,14 @@ SELECT '--- is null or ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) -ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); SELECT '--- cross ---'; diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index bbb4eb12466..c7f0c6383c2 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) left join y on (y.a = s.a and y.b = s.b) order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a as t_a from t left join s on s.a = t_a order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, s.a as s_a from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactMonoBlock; +format PrettyCompactNoEscapes; drop table t; drop table s; diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index 629a0e384f8..cdb9d253b9b 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,7 +1,5 @@ SET joined_subquery_requires_alias = 0; -SET query_plan_join_swap_table = 'auto'; - {% for join_algorithm in ['partial_merge', 'hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.reference b/tests/queries/0_stateless/01107_join_right_table_totals.reference index aa569ff9331..daf503b776d 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.reference +++ b/tests/queries/0_stateless/01107_join_right_table_totals.reference @@ -18,35 +18,28 @@ 0 0 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 1 1 0 0 -- 1 1 0 0 -- 1 foo 1 1 300 0 foo 1 0 300 -- 1 100 1970-01-01 1 100 1970-01-01 1 100 1970-01-01 1 200 1970-01-02 1 200 1970-01-02 1 100 1970-01-01 diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.sql b/tests/queries/0_stateless/01107_join_right_table_totals.sql index 7e549282489..ad8954d5d70 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.sql +++ b/tests/queries/0_stateless/01107_join_right_table_totals.sql @@ -64,47 +64,39 @@ USING (id); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l RIGHT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; -SELECT '-'; SELECT * FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r -ON l.item_id = r.item_id -ORDER BY ALL; +ON l.item_id = r.item_id; DROP TABLE t; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 229ac6eae09..19018a610b7 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter (WHERE) + Filter ((WHERE + DROP unused columns after JOIN)) Join (JOIN FillRightFirst) Expression ReadFromMergeTree (default.t1) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c13722f431a..c2d85cefb18 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; {% endfor -%} diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index f8e46a2b976..3bd1633ce32 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -0 3 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 2 2 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +0 3 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 @@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 1 ('',0) SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 0 ('b',256) -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) 0 ('b',256) SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) 2 4 2 Nullable(UInt64) UInt8 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index 33638edafa5..da70973ed87 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -- { echoOff } diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 885dbffa432..7a4d0857182 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int) ENGINE = TinyLog; -CREATE TABLE t2 (id Int) ENGINE = TinyLog; +CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql index c3c84ebaded..abc2ee41402 100644 --- a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -12,9 +12,8 @@ CREATE TABLE without_nullable insert into with_nullable values(0,'f'),(0,'usa'); insert into without_nullable values(0,'usa'),(0,'us2a'); -select if(t0.country is null ,t2.country,t0.country) "country" -from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country -ORDER BY 1 DESC; +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; drop table with_nullable; drop table without_nullable; diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 85abc8fa381..2cca853fd67 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -48,8 +48,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2 v1, vec2 v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -62,8 +61,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2f v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -76,8 +74,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2d v1, vec2d v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); SELECT v1.id, @@ -89,8 +86,7 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2d v2 -WHERE length(v1.v) == length(v2.v) -ORDER BY ALL; +WHERE length(v1.v) == length(v2.v); SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 90aab0a0eb2..365725f8ffe 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -148,6 +148,7 @@ Header: key String value String Join Header: __table1.key String + __table3.key String __table3.value String Sorting Header: __table1.key String diff --git a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql index 8b5c6846bd0..f0ecbf64e58 100644 --- a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql +++ b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql @@ -9,5 +9,4 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B INSERT INTO t1__fuzz_13 VALUES (1); INSERT INTO t2__fuzz_47 VALUES (1); -SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2 -ORDER BY ALL; +SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index c9bf36f88ea..3c68d14fdf2 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + Projection)) + Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers @@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + Projection)) + Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index baa2be9dfdb..867ae394c1f 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -79,7 +79,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) Distinct (Preliminary DISTINCT) - Expression (Projection) + Expression ((Projection + DROP unused columns after JOIN)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + Project names)) Distinct (DISTINCT) @@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -280,7 +280,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -315,7 +315,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -386,7 +386,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -457,7 +457,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index bbfdf1ad5f4..2c62e278050 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -8,21 +8,24 @@ Header: count() UInt64 Aggregating Header: __table1.a2 String count() UInt64 - Expression (Before GROUP BY) + Expression ((Before GROUP BY + DROP unused columns after JOIN)) Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - Expression (JOIN actions) + __table3.c1 UInt64 + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String + __table2.b1 UInt64 __table3.c1 UInt64 - Expression (JOIN actions) + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a2 String + Header: __table1.a1 UInt64 + __table1.a2 String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 @@ -45,32 +48,39 @@ Header: count() UInt64 EXPLAIN PLAN header = 1 SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) ; -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String + __table1.k UInt64 __table4.d2 String - Join (JOIN FillRightFirst) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression (Change column names to column identifiers) + Expression (DROP unused columns after JOIN) Header: __table1.a2 String __table1.k UInt64 - ReadFromMemoryStorage - Header: a2 String - k UInt64 + Join (JOIN FillRightFirst) + Header: __table1.a2 String + __table1.k UInt64 + Expression (Change column names to column identifiers) + Header: __table1.a2 String + __table1.k UInt64 + ReadFromMemoryStorage + Header: a2 String + k UInt64 + Expression (Change column names to column identifiers) + Header: __table2.k UInt64 + ReadFromMemoryStorage + Header: k UInt64 Expression (Change column names to column identifiers) - Header: __table2.k UInt64 + Header: __table3.k UInt64 ReadFromMemoryStorage Header: k UInt64 - Expression (Change column names to column identifiers) - Header: __table3.k UInt64 - ReadFromMemoryStorage - Header: k UInt64 Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 @@ -96,24 +106,27 @@ Header: bx String Header: __table1.a2 String __table2.bx String __table4.c2 String + __table4.c1 UInt64 Expression Header: __table1.a2 String __table2.bx String - __table4.c1 UInt64 __table4.c2 String + __table4.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table4.c1 UInt64 + __table2.b1 UInt64 __table4.c2 String - Expression (JOIN actions) + __table4.c1 UInt64 + Expression ((JOIN actions + DROP unused columns after JOIN)) Header: __table1.a2 String - __table2.b1 UInt64 __table2.bx String + __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table2.b1 UInt64 + Header: __table1.a1 UInt64 + __table1.a2 String __table2.bx String + __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 __table1.a2 String diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index c3e4ba0db9d..df84e2f50b2 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,7 +16,6 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; -SET query_plan_join_swap_table = 'false'; -- { echoOn } diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 116c78a15e4..86e7e2a6a49 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -5,7 +5,7 @@ 1 1 -0 +1 \N 100000000000000000000 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index bdbc019d4f8..06f4a9cfc99 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -1,22 +1,22 @@ -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value_1 String : 3 + INPUT : 2 -> __table2.value_1 String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.id UInt64 __table2.value_1 String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -50,25 +50,29 @@ Positions: 4 0 1 2 Parts: 1 Granules: 1 -- -Expression ((Project names + Projection)) +Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 2 -> __table1.value_2 UInt64 : 2 INPUT : 3 -> __table2.value_1 String : 3 - ALIAS __table1.id :: 0 -> id UInt64 : 4 + INPUT :: 4 -> __table2.value_2 UInt64 : 4 + INPUT : 5 -> __table2.id UInt64 : 5 + ALIAS __table1.id :: 0 -> id UInt64 : 6 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 + ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.id UInt64 + __table1.value_2 UInt64 __table2.value_1 String + __table2.value_2 UInt64 + __table2.id UInt64 Type: INNER Strictness: ASOF Algorithm: HashJoin diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index 11f40fb8887..1cdd3684a0b 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,8 +19,6 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); -SET query_plan_join_swap_table = 'false'; - EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.reference b/tests/queries/0_stateless/02962_join_using_bug_57894.reference index fc6fe462205..454655081df 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.reference +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.reference @@ -31,7 +31,6 @@ 8 9 \N ---- analyzer --- 0 1 2 diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index e29347beb5e..96190241da5 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -21,8 +21,6 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; -SELECT '--- analyzer ---'; - SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 1bea145c50a..d0a3e7b02ae 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -2,9 +2,7 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -12,18 +10,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -71,9 +69,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false'; -; +WHERE rhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -81,18 +77,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -140,9 +136,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5 AND rhs.id = 6; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -150,18 +144,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -220,9 +214,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -230,18 +222,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -289,9 +281,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -299,31 +289,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -365,9 +355,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -375,31 +363,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -441,9 +429,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -451,18 +437,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -510,9 +496,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -520,31 +504,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT :: 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin @@ -586,9 +570,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -596,31 +578,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT :: 3 -> __table2.value String : 3 + INPUT :: 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin @@ -662,9 +644,7 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5 AND rhs.id = 6; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -672,18 +652,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 1 0 2 - Filter (WHERE) + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 2 0 1 + Filter ((WHERE + DROP unused columns after JOIN)) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 AND column: equals(__table1.id, 5_UInt8) Actions: INPUT : 0 -> __table1.id UInt64 : 0 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1 @@ -692,18 +672,18 @@ Positions: 4 1 0 2 Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT :: 1 -> __table1.id UInt64 : 0 INPUT :: 2 -> __table1.value String : 1 - INPUT : 3 -> __table2.id UInt64 : 2 - INPUT :: 4 -> __table2.value String : 3 + INPUT :: 3 -> __table2.value String : 2 + INPUT : 4 -> __table2.id UInt64 : 3 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4 INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5 - FUNCTION equals(__table2.id : 2, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 + FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 FUNCTION and(equals(__table1.id, 5_UInt8) :: 5, equals(__table2.id, 6_UInt8) :: 6) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 4 Positions: 4 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: FULL Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index a7b98b95082..e1a13d1ce71 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -22,9 +22,7 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; SELECT '--'; @@ -35,9 +33,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false'; -; +WHERE rhs.id = 5; SELECT '--'; @@ -48,9 +44,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5 AND rhs.id = 6; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6; @@ -59,9 +53,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; SELECT '--'; @@ -72,9 +64,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; SELECT '--'; @@ -85,9 +75,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; SELECT '--'; @@ -98,9 +86,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; SELECT '--'; @@ -111,9 +97,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5; SELECT '--'; @@ -124,9 +108,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE rhs.id = 5; SELECT '--'; @@ -137,9 +119,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_swap_table = 'false' -; +WHERE lhs.id = 5 AND rhs.id = 6; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index 64d4d0d0eb1..cf070eebc38 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -52,9 +52,7 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph -SETTINGS query_plan_join_swap_table = 'false' -; +SELECT * FROM search_graph; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] 2 3 arc 2 -> 3 false [(2,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index 65e4439c1fd..7dad74893b9 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -55,9 +55,7 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph -SETTINGS query_plan_join_swap_table = 'false' -; +SELECT * FROM search_graph; -- ordering by the path column has same effect as SEARCH DEPTH FIRST WITH RECURSIVE search_graph AS ( diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index a3f04fb8870..6ae4e4d4d3c 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,7 +3,6 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES -SET query_plan_join_swap_table = 'auto'; -- 'true' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index 5fde4f80c5d..d35bdeff98b 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -5,18 +5,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -75,18 +75,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin @@ -145,18 +145,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.id UInt64 : 2 - INPUT : 3 -> __table2.value String : 3 + INPUT : 2 -> __table2.value String : 2 + INPUT : 3 -> __table2.id UInt64 : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 - ALIAS __table2.value :: 3 -> rhs.value String : 2 -Positions: 4 0 1 2 + ALIAS __table2.value :: 2 -> rhs.value String : 1 + ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.id UInt64 __table2.value String + __table2.id UInt64 Type: INNER Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index 0a53bf03ad5..b3d1827d98f 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -22,10 +22,7 @@ SETTINGS index_granularity = 16 INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); - -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 -SETTINGS query_plan_join_swap_table = 'false' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0; SELECT '--'; @@ -33,9 +30,7 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 -SETTINGS query_plan_join_swap_table = 'false' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; SELECT '--'; @@ -43,9 +38,7 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 -SETTINGS query_plan_join_swap_table = 'false' -; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; SELECT '--'; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference index 1c82e76cc65..7058d36aaf9 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference @@ -65,7 +65,8 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice'; Expression ((Project names + (Projection + ))) Header: name String Join (JOIN FillRightFirst) - Header: __table2.name String + Header: __table1.name String + __table2.name String Filter (( + Change column names to column identifiers)) Header: __table1.name String ReadFromMergeTree (default.users) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 20a8be6d499..f6e5dbdef03 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,7 +11,6 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; -SET query_plan_join_swap_table = 'false'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, diff --git a/tests/queries/0_stateless/03267_join_swap_bug.reference b/tests/queries/0_stateless/03267_join_swap_bug.reference deleted file mode 100644 index 68b8a88e2c1..00000000000 --- a/tests/queries/0_stateless/03267_join_swap_bug.reference +++ /dev/null @@ -1,4 +0,0 @@ -1 -1 -0 -1 diff --git a/tests/queries/0_stateless/03267_join_swap_bug.sql b/tests/queries/0_stateless/03267_join_swap_bug.sql deleted file mode 100644 index 1ccfa4924be..00000000000 --- a/tests/queries/0_stateless/03267_join_swap_bug.sql +++ /dev/null @@ -1,33 +0,0 @@ -DROP TABLE IF EXISTS t0; -CREATE TABLE t0 (c0 Int) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO TABLE t0 (c0) VALUES (1); - -SELECT 1 FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; -SELECT count() FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; - -SET allow_experimental_analyzer = 1; - -SELECT * -FROM -( - SELECT * - FROM system.one -) AS a -INNER JOIN -( - SELECT * - FROM system.one -) AS b USING (dummy) -INNER JOIN -( - SELECT * - FROM system.one -) AS c USING (dummy) -SETTINGS join_algorithm = 'full_sorting_merge'; - - -SELECT count(1) -FROM ( SELECT 1 AS x, x ) AS t1 -RIGHT JOIN (SELECT materialize(2) AS x) AS t2 -ON t1.x = t2.x -; From 7e053a7e877a96a617b40c647608265690cd0b68 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 18 Oct 2024 12:31:45 +0100 Subject: [PATCH 292/433] fix test --- .../0_stateless/00475_in_join_db_table.sql | 4 +- .../00800_low_cardinality_join.sql | 52 +++++++++---------- .../00826_cross_to_inner_join.reference | 15 ++++++ .../0_stateless/00826_cross_to_inner_join.sql | 47 ++++++++--------- .../00860_unknown_identifier_bug.sql | 3 +- .../0_stateless/00863_comma_join_in.sql | 3 +- .../02380_analyzer_join_sample.sql | 3 +- .../03229_json_structure_comparison.sql | 13 ++--- 8 files changed, 77 insertions(+), 63 deletions(-) diff --git a/tests/queries/0_stateless/00475_in_join_db_table.sql b/tests/queries/0_stateless/00475_in_join_db_table.sql index 5f90d108015..97ffe21935f 100644 --- a/tests/queries/0_stateless/00475_in_join_db_table.sql +++ b/tests/queries/0_stateless/00475_in_join_db_table.sql @@ -13,11 +13,11 @@ DROP TABLE set; DROP TABLE IF EXISTS join; CREATE TABLE join (k UInt8, x String) ENGINE = Memory; INSERT INTO join VALUES (1, 'hello'); -SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k; +SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k ORDER BY ALL; DROP TABLE join; CREATE TABLE join (k UInt8, x String) ENGINE = Join(ANY, LEFT, k); INSERT INTO join VALUES (1, 'hello'); -SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k; +SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k ORDER BY ALL; DROP TABLE join; diff --git a/tests/queries/0_stateless/00800_low_cardinality_join.sql b/tests/queries/0_stateless/00800_low_cardinality_join.sql index fc5f5d1860c..a6c3b98fd6d 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -1,31 +1,31 @@ set enable_analyzer = 1; set joined_subquery_requires_alias = 0; -select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as val from system.one) using val; -select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as val from system.one) using val; -select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; +select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as val from system.one) using val order by all; +select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as val from system.one) using val order by all; +select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) using val order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val order by all; select '-'; -select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } -select * from (select dummy as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1; -select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1; -select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1; -select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1; +select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1 order by all; -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select dummy as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1 order by all; +select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1 order by all; select '-'; -select * from (select number as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select number as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) on l + 1 = r * 1; -select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) on l + 1 = r * 1; +select * from (select number as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select number as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) any left join (select toLowCardinality(number) as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) on l + 1 = r * 1 order by all; +select * from (select toLowCardinality(toNullable(number)) as l from system.numbers limit 3) any left join (select toLowCardinality(toNullable(number)) as r from system.numbers limit 3) on l + 1 = r * 1 order by all; diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.reference b/tests/queries/0_stateless/00826_cross_to_inner_join.reference index a47d9a430dc..845bb9a6404 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.reference +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.reference @@ -57,6 +57,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE a = t2_00826.a +ORDER BY ALL ASC --- cross nullable --- SELECT a, @@ -66,6 +67,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE a = t2_00826.a +ORDER BY ALL ASC --- cross nullable vs not nullable --- SELECT a, @@ -75,6 +77,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.b WHERE a = t2_00826.b +ORDER BY ALL ASC --- cross self --- SELECT a, @@ -84,6 +87,7 @@ SELECT FROM t1_00826 AS x ALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b) WHERE (a = y.a) AND (b = y.b) +ORDER BY ALL ASC --- cross one table expr --- SELECT a, @@ -93,6 +97,7 @@ SELECT FROM t1_00826 CROSS JOIN t2_00826 WHERE a = b +ORDER BY ALL ASC --- cross multiple ands --- SELECT a, @@ -102,6 +107,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND (b = t2_00826.b) +ORDER BY ALL ASC --- cross and inside and --- SELECT a, @@ -111,6 +117,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) +ORDER BY ALL ASC --- cross split conjunction --- SELECT a, @@ -120,6 +127,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) +ORDER BY ALL ASC --- and or --- SELECT a, @@ -129,6 +137,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) WHERE (a = t2_00826.a) AND (b = t2_00826.b) AND ((a >= 1) OR (t2_00826.b = 1)) +ORDER BY ALL ASC --- arithmetic expr --- SELECT a, @@ -138,6 +147,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON (a + 1) = (t2_00826.a + t2_00826.b) WHERE ((a + 1) = (t2_00826.a + t2_00826.b)) AND ((((a + b) + t2_00826.a) + t2_00826.b) > 5) +ORDER BY ALL ASC --- is null or --- SELECT a, @@ -147,11 +157,13 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON b = t2_00826.a WHERE (b = t2_00826.a) AND ((t2_00826.b IS NULL) OR (t2_00826.b > t2_00826.a)) +ORDER BY ALL ASC --- do not rewrite alias --- SELECT a AS b FROM t1_00826 CROSS JOIN t2_00826 WHERE (b = t2_00826.a) AND (b > 0) +ORDER BY ALL ASC --- comma --- SELECT a, @@ -161,6 +173,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE a = t2_00826.a +ORDER BY ALL ASC --- comma nullable --- SELECT a, @@ -170,6 +183,7 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON b = t2_00826.b WHERE b = t2_00826.b +ORDER BY ALL ASC --- comma and or --- SELECT a, @@ -179,3 +193,4 @@ SELECT FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a WHERE (a = t2_00826.a) AND ((t2_00826.b IS NULL) OR (t2_00826.b < 2)) +ORDER BY ALL ASC diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index 5ab7a2d0626..60e07c6ab24 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -2,7 +2,7 @@ SET enable_optimize_predicate_expression = 0; SET optimize_move_to_prewhere = 1; SET convert_query_to_cnf = 0; -select * from system.one l cross join system.one r; +select * from system.one l cross join system.one r order by all; DROP TABLE IF EXISTS t1_00826; DROP TABLE IF EXISTS t2_00826; @@ -25,20 +25,20 @@ SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b ORD SELECT '--- cross one table expr ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b order by (t1_00826.a, t2_00826.a, t2_00826.b); SELECT '--- cross multiple ands ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b; +select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b order by all; SELECT '--- cross and inside and ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.b = t2_00826.b and 1); +select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.b = t2_00826.b and 1) order by all; SELECT '--- cross split conjunction ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b = 1; +select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b = 1 order by all; SELECT '--- and or ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and (t1_00826.a >= 1 OR t2_00826.b = 1); +select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and (t1_00826.a >= 1 OR t2_00826.b = 1) order by all; SELECT '--- arithmetic expr ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a + 1 = t2_00826.a + t2_00826.b AND (t1_00826.a + t1_00826.b + t2_00826.a + t2_00826.b > 5); +select * from t1_00826 cross join t2_00826 where t1_00826.a + 1 = t2_00826.a + t2_00826.b AND (t1_00826.a + t1_00826.b + t2_00826.a + t2_00826.b > 5) order by all; SELECT '--- is null or ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; +select * from t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; @@ -48,46 +48,45 @@ SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- comma nullable ---'; SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) -ORDER BY ALL; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) ORDER BY ALL; SELECT '--- cross ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a order by all; SELECT '--- cross nullable ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +EXPLAIN SYNTAX select * from t1_00826, t2_00826 where t1_00826.a = t2_00826.a order by all; SELECT '--- cross nullable vs not nullable ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b; +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b order by all; SELECT '--- cross self ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b; +EXPLAIN SYNTAX select * from t1_00826 x cross join t1_00826 y where x.a = y.a and x.b = y.b order by all; SELECT '--- cross one table expr ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b; +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t1_00826.b order by all; SELECT '--- cross multiple ands ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b; +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b order by all; SELECT '--- cross and inside and ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b)); +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and (t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b)) order by all; SELECT '--- cross split conjunction ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b > 0; +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and t1_00826.a >= 1 and t2_00826.b > 0 order by all; SELECT '--- and or ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and (t1_00826.a >= 1 OR t2_00826.b = 1); +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a and t1_00826.b = t2_00826.b and (t1_00826.a >= 1 OR t2_00826.b = 1) order by all; SELECT '--- arithmetic expr ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a + 1 = t2_00826.a + t2_00826.b AND (t1_00826.a + t1_00826.b + t2_00826.a + t2_00826.b > 5); +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.a + 1 = t2_00826.a + t2_00826.b AND (t1_00826.a + t1_00826.b + t2_00826.a + t2_00826.b > 5) order by all; SELECT '--- is null or ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a); +EXPLAIN SYNTAX select * from t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) order by all; SELECT '--- do not rewrite alias ---'; -EXPLAIN SYNTAX SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; +EXPLAIN SYNTAX select a as b from t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 order by all; SELECT '--- comma ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +EXPLAIN SYNTAX select * from t1_00826, t2_00826 where t1_00826.a = t2_00826.a order by all; SELECT '--- comma nullable ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; +EXPLAIN SYNTAX select * from t1_00826, t2_00826 where t1_00826.b = t2_00826.b order by all; SELECT '--- comma and or ---'; -EXPLAIN SYNTAX SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); +EXPLAIN SYNTAX select * from t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) order by all; DROP TABLE t1_00826; DROP TABLE t2_00826; diff --git a/tests/queries/0_stateless/00860_unknown_identifier_bug.sql b/tests/queries/0_stateless/00860_unknown_identifier_bug.sql index bbcd3de8f20..9bc056036b7 100644 --- a/tests/queries/0_stateless/00860_unknown_identifier_bug.sql +++ b/tests/queries/0_stateless/00860_unknown_identifier_bug.sql @@ -34,6 +34,7 @@ LEFT JOIN FROM appointment_events WHERE _status in ('Created', 'Transferred') GROUP BY _appointment_id ) B USING _appointment_id -WHERE A._set_at = B.max_set_at; +WHERE A._set_at = B.max_set_at +ORDER BY ALL; DROP TABLE appointment_events; diff --git a/tests/queries/0_stateless/00863_comma_join_in.sql b/tests/queries/0_stateless/00863_comma_join_in.sql index ebccd351c8a..2bc66324516 100644 --- a/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/tests/queries/0_stateless/00863_comma_join_in.sql @@ -16,7 +16,8 @@ select test2_00863.id from test1_00863, test2_00863, test3_00863 where test1_00863.code in ('1', '2', '3') and test2_00863.test1_id = test1_00863.id - and test2_00863.test3_id = test3_00863.id; + and test2_00863.test3_id = test3_00863.id +order by all; drop table test1_00863; drop table test2_00863; diff --git a/tests/queries/0_stateless/02380_analyzer_join_sample.sql b/tests/queries/0_stateless/02380_analyzer_join_sample.sql index bc77f3623ae..2bb6d1e13c3 100644 --- a/tests/queries/0_stateless/02380_analyzer_join_sample.sql +++ b/tests/queries/0_stateless/02380_analyzer_join_sample.sql @@ -23,7 +23,8 @@ SAMPLE BY id; INSERT INTO test_table_join_2 VALUES (0, 'Value'), (1, 'Value_1'); SELECT t1.id AS t1_id, t2.id AS t2_id, t1._sample_factor AS t1_sample_factor, t2._sample_factor AS t2_sample_factor -FROM test_table_join_1 AS t1 SAMPLE 1/2 INNER JOIN test_table_join_2 AS t2 SAMPLE 1/2 ON t1.id = t2.id; +FROM test_table_join_1 AS t1 SAMPLE 1/2 INNER JOIN test_table_join_2 AS t2 SAMPLE 1/2 ON t1.id = t2.id +ORDER BY ALL; DROP TABLE test_table_join_1; DROP TABLE test_table_join_2; diff --git a/tests/queries/0_stateless/03229_json_structure_comparison.sql b/tests/queries/0_stateless/03229_json_structure_comparison.sql index 39029cee28b..295c8b429d4 100644 --- a/tests/queries/0_stateless/03229_json_structure_comparison.sql +++ b/tests/queries/0_stateless/03229_json_structure_comparison.sql @@ -11,13 +11,10 @@ INSERT INTO test_new_json_type format JSONEachRow ; SELECT - a.data - , b.data -FROM - test_new_json_type a - JOIN test_new_json_type b - ON a.id = b.id -ORDER BY ALL; + a.data, + b.data +FROM test_new_json_type AS a +INNER JOIN test_new_json_type AS b ON a.id = b.id +ORDER BY id; DROP TABLE test_new_json_type; - From 63e9204c5247f630d95a705b12bf1b84676895c8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 23 Oct 2024 15:39:37 +0100 Subject: [PATCH 293/433] fix --- tests/integration/test_peak_memory_usage/test.py | 2 +- .../queries/0_stateless/00225_join_duplicate_columns.sql | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index f447c527d9b..8281b3dea73 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_swap_table = 'false'", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS join_algorithm='hash', query_plan_join_swap_table = 'false'", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/00225_join_duplicate_columns.sql b/tests/queries/0_stateless/00225_join_duplicate_columns.sql index 6cb4647cd14..712fae7a674 100644 --- a/tests/queries/0_stateless/00225_join_duplicate_columns.sql +++ b/tests/queries/0_stateless/00225_join_duplicate_columns.sql @@ -1,3 +1,12 @@ +-- The following queries use very weird block structure: +-- __table3.b UInt8 UInt8(size = 1), __table3.b UInt8 Const(size = 1, UInt8(size = 1)), __table3.c UInt8 Const(size = 1, UInt8(size = 1)) +-- That leads to a pretty legit error in ConcurrentHashJoin within a call to Block::cloneEmpty(): +-- Code: 352. DB::Exception: Block structure mismatch in (columns with identical name must have identical structure) stream: different columns: +-- __table3.b UInt8 UInt8(size = 0) +-- __table3.b UInt8 Const(size = 0, UInt8(size = 1)) +-- So let's disable parallel_hash. +SET join_algorithm = 'hash,grace_hash,partial_merge,full_sorting_merge'; + select b from (select 1 as a, 42 as c) js1 any left join (select 2 as b, 2 as b, 41 as c) js2 using c; select b from (select 1 as a, 42 as c) js1 any left join (select 2 as b, 2 as b, 42 as c) js2 using c; From 204f723638429932b2865377472c66cafa467d49 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 31 Oct 2024 22:17:10 +0100 Subject: [PATCH 294/433] fix test --- src/Processors/Transforms/JoiningTransform.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 992d5eca77a..7772518cc1c 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -116,7 +116,7 @@ IProcessor::Status JoiningTransform::prepare() return Status::NeedData; input_chunk = input.pull(true); - has_input = true; + has_input = input_chunk.hasRows() || on_totals; return Status::Ready; } @@ -155,8 +155,11 @@ void JoiningTransform::work() return; } - output_chunks.emplace_back(block.getColumns(), block.rows()); - has_output = true; + if (block.rows()) + { + output_chunks.emplace_back(block.getColumns(), block.rows()); + has_output = true; + } } } @@ -190,9 +193,15 @@ void JoiningTransform::transform(Chunk & chunk) JoinCommon::joinTotals(left_totals, right_totals, join->getTableJoin(), res.back()); } else + { res = readExecute(chunk); + } - std::ranges::for_each(res, [this](Block & block) { output_chunks.emplace_back(block.getColumns(), block.rows()); }); + for (const auto & block : res) + { + if (block.rows()) + output_chunks.emplace_back(block.getColumns(), block.rows()); + } } Blocks JoiningTransform::readExecute(Chunk & chunk) From 4725ef2f74ac95b2decb48cdb1fdb36fe05e5d81 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 17 Nov 2024 16:28:46 +0100 Subject: [PATCH 295/433] fix test --- ...llel_replicas_joins_and_analyzer.reference | 100 ++++++++++-------- 1 file changed, 54 insertions(+), 46 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 765847c0607..ce9ea7f3e53 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -229,43 +229,47 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5; +select * from sub5 order by all; 0 0 0 0 0 0 -6 6 6 6 0 0 -8 8 8 8 0 0 -10 10 10 10 0 0 -12 12 12 12 12 12 -14 14 14 14 0 0 -4 4 0 0 0 0 -3 3 0 0 0 0 -5 5 0 0 0 0 1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 7 7 0 0 0 0 +8 8 8 8 0 0 9 9 0 0 0 0 -15 15 0 0 0 0 +10 10 10 10 0 0 11 11 0 0 0 0 +12 12 12 12 12 12 13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 explain description=0 with sub1 as (select x, y from tab1 where x != 2), sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5; -Union - Expression - Join +select * from sub5 order by all; +Expression + Sorting + Union Expression - Join + Sorting Expression - ReadFromMemoryStorage - Expression - Expression - ReadFromMergeTree + Join + Expression + Join + Expression + ReadFromMemoryStorage + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage Expression - ReadFromMemoryStorage - Expression - ReadFromRemoteParallelReplicas + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -664,45 +668,49 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5; +select * from sub5 order by all; 0 0 0 0 0 0 -6 6 6 6 0 0 -8 8 8 8 0 0 -10 10 10 10 0 0 -12 12 12 12 12 12 -14 14 14 14 0 0 -4 4 0 0 0 0 -3 3 0 0 0 0 -5 5 0 0 0 0 1 1 0 0 0 0 +3 3 0 0 0 0 +4 4 0 0 0 0 +5 5 0 0 0 0 +6 6 6 6 0 0 7 7 0 0 0 0 +8 8 8 8 0 0 9 9 0 0 0 0 -15 15 0 0 0 0 +10 10 10 10 0 0 11 11 0 0 0 0 +12 12 12 12 12 12 13 13 0 0 0 0 +14 14 14 14 0 0 +15 15 0 0 0 0 explain description=0 with sub1 as (select x, y from tab1 where x != 2), sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5; -Union - Expression - Join +select * from sub5 order by all; +Expression + Sorting + Union Expression - Join + Sorting Expression - Expression - ReadFromMergeTree - Expression - Expression - ReadFromMergeTree + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree Expression - Expression - ReadFromMergeTree - Expression - ReadFromRemoteParallelReplicas + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), From 2892e473b8417e146c0c6a3ea137731003567cef Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 17 Nov 2024 23:15:52 +0100 Subject: [PATCH 296/433] fix test --- ...llel_replicas_joins_and_analyzer.reference | 60 ++++++++----------- ...arallel_replicas_joins_and_analyzer.sql.j2 | 2 +- 2 files changed, 27 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index ce9ea7f3e53..c319256f9b5 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -251,25 +251,21 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5 order by all; -Expression - Sorting - Union +select * from sub5; +Union + Expression + Join Expression - Sorting + Join Expression - Join - Expression - Join - Expression - ReadFromMemoryStorage - Expression - Expression - ReadFromMergeTree - Expression - ReadFromMemoryStorage + ReadFromMemoryStorage + Expression + Expression + ReadFromMergeTree Expression - ReadFromRemoteParallelReplicas + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -690,27 +686,23 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5 order by all; -Expression - Sorting - Union +select * from sub5; +Union + Expression + Join Expression - Sorting + Join Expression - Join - Expression - Join - Expression - Expression - ReadFromMergeTree - Expression - Expression - ReadFromMergeTree - Expression - Expression - ReadFromMergeTree + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree Expression - ReadFromRemoteParallelReplicas + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 31cb0b735ae..d297a2e886b 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -98,7 +98,7 @@ sub2 as (select y, z from tab2 where y != 4), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub4 as (select z, a from tab3 where z != 8), sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) -select * from sub5; +select * from sub5 order by all; explain description=0 with sub1 as (select x, y from tab1 where x != 2), From a4212a9e870eccab4f8110ec26a20a380cf1b324 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 17 Nov 2024 23:20:29 +0100 Subject: [PATCH 297/433] fix test --- .../02236_explain_pipeline_join.reference | 66 ++++++++++++++----- 1 file changed, 49 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index f8306831330..1d934340cc9 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,20 +1,52 @@ (Expression) ExpressionTransform × 16 (Join) - JoiningTransform × 16 2 → 1 - Resize 1 → 16 - (Expression) - ExpressionTransform - (Limit) - Limit - (ReadFromSystemNumbers) - NumbersRange 0 → 1 - (Expression) - Resize × 2 16 → 1 - FillingRightJoinSide × 16 - Resize 1 → 16 - ExpressionTransform - (Limit) - Limit - (ReadFromSystemNumbers) - NumbersRange 0 → 1 + SimpleSquashingTransform × 16 + JoiningTransform × 16 2 → 1 + Resize 1 → 16 + (Expression) + ExpressionTransform + (Limit) + Limit + (ReadFromSystemNumbers) + NumbersRange 0 → 1 + (Expression) + Resize × 2 16 → 1 + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + FillingRightJoinSide + SimpleSquashingTransform + Resize 1 → 16 + ExpressionTransform + (Limit) + Limit + (ReadFromSystemNumbers) + NumbersRange 0 → 1 From 00f1659601f50b145c6bae09667cf116db8f4222 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 15:41:40 +0100 Subject: [PATCH 298/433] fix number of threads in test --- tests/queries/0_stateless/03094_one_thousand_joins.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index a3f04fb8870..cf489d7eb85 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -4,6 +4,7 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES SET query_plan_join_swap_table = 'auto'; -- 'true' is slower +SET max_threads = 8; -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x From 42df6591aa70b4c1737b104fafc428a91df34447 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 23:02:46 +0100 Subject: [PATCH 299/433] fix uncaught exception --- src/Interpreters/HashJoin/ScatteredBlock.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 729377f6758..31ff773d04d 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -302,10 +302,11 @@ struct ScatteredBlock : private boost::noncopyable /// Cut first `num_rows` rows from `block` in place and returns block with remaining rows ScatteredBlock cut(size_t num_rows) { - SCOPE_EXIT(filterBySelector()); - if (num_rows >= rows()) + { + filterBySelector(); return ScatteredBlock{Block{}}; + } chassert(block); @@ -314,6 +315,7 @@ struct ScatteredBlock : private boost::noncopyable auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; selector = std::move(first_num_rows); + filterBySelector(); return remaining; } From bd96303e1776496cdfd63a5d06f8bceedbff3079 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 19:39:50 +0100 Subject: [PATCH 300/433] Fix asserts --- src/Coordination/SnapshotableHashTable.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index bfe987de1cb..462ca18c502 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -297,7 +297,7 @@ public: { size_t hash_value = map.hash(key); auto it = map.find(key, hash_value); - if (it != map.end()) + if (it == map.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); auto list_itr = it->getMapped(); @@ -354,7 +354,7 @@ public: const V & getValue(StringRef key) const { auto it = map.find(key); - if (it != map.end()) + if (it == map.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); return it->getMapped()->value; } From db375779bde676a2dfa425a7b64d56a420e50832 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Nov 2024 19:45:59 +0100 Subject: [PATCH 301/433] better --- src/Core/Settings.cpp | 6 ++++-- src/Interpreters/TableJoin.cpp | 7 ++++--- src/Interpreters/TableJoin.h | 29 ++++++++++++++++------------- 3 files changed, 24 insertions(+), 18 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f5fde1b572b..4ea7986dd7f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2468,7 +2468,7 @@ Possible values: - default - This is the equivalent of `hash` or `direct`, if possible (same as `direct,hash`) + This is the equivalent of `hash`, `parallel_hash` or `direct`, if possible (same as `direct,parallel_hash,hash`) - grace_hash @@ -2482,11 +2482,13 @@ Possible values: [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. + When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM. + - parallel_hash A variation of `hash` join that splits the data into buckets and builds several hashtables instead of one concurrently to speed up this process. - When using the `hash` algorithm, the right part of `JOIN` is uploaded into RAM. + When using the `parallel_hash` algorithm, the right part of `JOIN` is uploaded into RAM. - partial_merge diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index f03b8d44356..4c0ff66afd6 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -7,7 +7,9 @@ #include #include +#include #include +#include #include #include @@ -28,7 +30,6 @@ #include #include -#include #include #include #include @@ -997,8 +998,8 @@ void TableJoin::resetToCross() bool TableJoin::allowParallelHashJoin() const { - if (std::ranges::find(join_algorithm, JoinAlgorithm::DEFAULT) == join_algorithm.end() - && std::ranges::find(join_algorithm, JoinAlgorithm::PARALLEL_HASH) == join_algorithm.end()) + if (std::ranges::none_of( + join_algorithm, [](auto algo) { return algo == JoinAlgorithm::DEFAULT || algo == JoinAlgorithm::PARALLEL_HASH; })) return false; if (!right_storage_name.empty()) return false; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 146230f44be..7f3dd0d54cd 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -1,25 +1,26 @@ #pragma once +#include #include #include -#include -#include -#include -#include #include +#include #include +#include #include - -#include +#include #include +#include +#include -#include -#include - -#include -#include #include +#include +#include +#include +#include +#include + namespace CurrentMetrics { extern const Metric TemporaryFilesForJoin; @@ -285,10 +286,12 @@ public: bool isEnabledAlgorithm(JoinAlgorithm val) const { - /// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm. + /// When join_algorithm = 'default' (not specified by user) we use [parallel_]hash or direct algorithm. /// It's behaviour that was initially supported by clickhouse. bool is_default_enabled = std::find(join_algorithm.begin(), join_algorithm.end(), JoinAlgorithm::DEFAULT) != join_algorithm.end(); - if (is_default_enabled && (val == JoinAlgorithm::DEFAULT || val == JoinAlgorithm::HASH || val == JoinAlgorithm::DIRECT)) + constexpr auto default_algorithms = std::array{ + JoinAlgorithm::DEFAULT, JoinAlgorithm::HASH, JoinAlgorithm::PARALLEL_HASH, JoinAlgorithm::DIRECT}; + if (is_default_enabled && std::ranges::find(default_algorithms, val) != default_algorithms.end()) return true; return std::find(join_algorithm.begin(), join_algorithm.end(), val) != join_algorithm.end(); } From c1cefb01908fdb74fb336f8aefa4f32b7177b428 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 25 Nov 2024 18:52:59 +0000 Subject: [PATCH 302/433] fix rank functions doc --- docs/en/sql-reference/window-functions/dense_rank.md | 8 ++++---- docs/en/sql-reference/window-functions/percent_rank.md | 6 +++--- docs/en/sql-reference/window-functions/rank.md | 10 +++++----- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md index 2c8617fb668..a528a617e5a 100644 --- a/docs/en/sql-reference/window-functions/dense_rank.md +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -15,8 +15,8 @@ The [rank](./rank.md) function provides the same behaviour, but with gaps in ran Alias: `denseRank` (case-sensitive) ```sql -dense_rank (column_name) - OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] +dense_rank () + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) @@ -55,7 +55,7 @@ INSERT INTO salaries FORMAT Values ``` ```sql -SELECT player, salary, +SELECT player, salary, dense_rank() OVER (ORDER BY salary DESC) AS dense_rank FROM salaries; ``` @@ -72,4 +72,4 @@ Result: 6. │ Scott Harrison │ 150000 │ 3 │ 7. │ James Henderson │ 140000 │ 4 │ └─────────────────┴────────┴────────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/window-functions/percent_rank.md b/docs/en/sql-reference/window-functions/percent_rank.md index 2e348f2a333..c947a6306ab 100644 --- a/docs/en/sql-reference/window-functions/percent_rank.md +++ b/docs/en/sql-reference/window-functions/percent_rank.md @@ -13,8 +13,8 @@ returns the relative rank (i.e. percentile) of rows within a window partition. Alias: `percentRank` (case-sensitive) ```sql -percent_rank (column_name) - OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] +percent_rank () + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] | [window_name]) FROM table_name WINDOW window_name as ([PARTITION BY grouping_column] [ORDER BY sorting_column] RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) @@ -50,7 +50,7 @@ INSERT INTO salaries FORMAT Values ``` ```sql -SELECT player, salary, +SELECT player, salary, percent_rank() OVER (ORDER BY salary DESC) AS percent_rank FROM salaries; ``` diff --git a/docs/en/sql-reference/window-functions/rank.md b/docs/en/sql-reference/window-functions/rank.md index dff5e154151..2aac6b46b3d 100644 --- a/docs/en/sql-reference/window-functions/rank.md +++ b/docs/en/sql-reference/window-functions/rank.md @@ -9,13 +9,13 @@ sidebar_position: 6 Ranks the current row within its partition with gaps. In other words, if the value of any row it encounters is equal to the value of a previous row then it will receive the same rank as that previous row. The rank of the next row is then equal to the rank of the previous row plus a gap equal to the number of times the previous rank was given. -The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. +The [dense_rank](./dense_rank.md) function provides the same behaviour but without gaps in ranking. **Syntax** ```sql -rank (column_name) - OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] +rank () + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) @@ -54,7 +54,7 @@ INSERT INTO salaries FORMAT Values ``` ```sql -SELECT player, salary, +SELECT player, salary, rank() OVER (ORDER BY salary DESC) AS rank FROM salaries; ``` @@ -71,4 +71,4 @@ Result: 6. │ Scott Harrison │ 150000 │ 4 │ 7. │ James Henderson │ 140000 │ 7 │ └─────────────────┴────────┴──────┘ -``` \ No newline at end of file +``` From 658ac29111b1db51d181e21b51fa162195bbcc1b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Nov 2024 20:00:05 +0100 Subject: [PATCH 303/433] Update parseColumnsListForTableFunction.cpp --- .../parseColumnsListForTableFunction.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index a375cb18297..737e28447d7 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -16,10 +16,10 @@ namespace DB { namespace Setting { - extern const SettingsBool enable_dynamic_type; - extern const SettingsBool enable_json_type; + extern const SettingsBool allow_experimental_dynamic_type; + extern const SettingsBool allow_experimental_json_type; extern const SettingsBool allow_experimental_object_type; - extern const SettingsBool enable_variant_type; + extern const SettingsBool allow_experimental_variant_type; extern const SettingsBool allow_experimental_bfloat16_type; extern const SettingsBool allow_suspicious_fixed_string_types; extern const SettingsBool allow_suspicious_low_cardinality_types; @@ -42,12 +42,12 @@ DataTypeValidationSettings::DataTypeValidationSettings(const DB::Settings & sett : allow_suspicious_low_cardinality_types(settings[Setting::allow_suspicious_low_cardinality_types]) , allow_experimental_object_type(settings[Setting::allow_experimental_object_type]) , allow_suspicious_fixed_string_types(settings[Setting::allow_suspicious_fixed_string_types]) - , enable_variant_type(settings[Setting::enable_variant_type]) + , enable_variant_type(settings[Setting::allow_experimental_variant_type]) , allow_experimental_bfloat16_type(settings[Setting::allow_experimental_bfloat16_type]) , allow_suspicious_variant_types(settings[Setting::allow_suspicious_variant_types]) , validate_nested_types(settings[Setting::validate_experimental_and_suspicious_types_inside_nested_types]) - , enable_dynamic_type(settings[Setting::enable_dynamic_type]) - , enable_json_type(settings[Setting::enable_json_type]) + , enable_dynamic_type(settings[Setting::allow_experimental_dynamic_type]) + , enable_json_type(settings[Setting::allow_experimental_json_type]) { } From 75a6dcdf3f96c1d23b26f02e3a0a50359b6d70ea Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Tue, 26 Nov 2024 03:08:01 +0800 Subject: [PATCH 304/433] fixed the error while pull the master --- src/Common/OvercommitTracker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index e530d486a6a..2ae72bacf79 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -45,7 +45,7 @@ OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int // method OvercommitTracker::onQueryStop(MemoryTracker *) is // always called with already acquired global mutex in // ProcessListEntry::~ProcessListEntry(). - auto global_lock = process_list->unsafeLock(); + DB::ProcessList::Lock global_lock(process_list->getMutex()); std::unique_lock lk(overcommit_m); size_t id = next_id++; From 10160fed8e893a0b175f681761e4e90d2e951061 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 20:09:03 +0100 Subject: [PATCH 305/433] Check dictionary permutation size --- src/Columns/ColumnLowCardinality.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 284b00d621a..82d2092907b 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -484,6 +484,12 @@ void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection d IColumn::Permutation dict_perm; getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); + /// This is a paranoid check, but in other places in code empty permutation is used to indicate that no sorting is needed. + if (dict_perm.size() != getDictionary().size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Dictionary permutation size {} is equal to dictionary size {}. It is a bug.", + dict_perm.size(), getDictionary().size()); + PaddedPODArray position_by_index(dict_perm.size()); for (size_t i = 0; i < dict_perm.size(); ++i) position_by_index[dict_perm[i]] = i; From a35971f7498e15000d17ff8754a7840b81a43ed6 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 20:19:46 +0100 Subject: [PATCH 306/433] style --- src/Columns/ColumnLowCardinality.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 82d2092907b..ae8971c96aa 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -489,7 +489,6 @@ void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection d throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary permutation size {} is equal to dictionary size {}. It is a bug.", dict_perm.size(), getDictionary().size()); - PaddedPODArray position_by_index(dict_perm.size()); for (size_t i = 0; i < dict_perm.size(); ++i) position_by_index[dict_perm[i]] = i; From cb7d33dc6ef931f34466ff76b795f96bf070d939 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 21:10:14 +0100 Subject: [PATCH 307/433] Fix build --- programs/extract-from-config/CMakeLists.txt | 1 + programs/keeper/CMakeLists.txt | 1 + programs/server/CMakeLists.txt | 1 + src/CMakeLists.txt | 2 ++ src/Common/Config/CMakeLists.txt | 1 + src/Common/ZooKeeper/CMakeLists.txt | 11 +++++++++-- src/Common/ZooKeeper/examples/CMakeLists.txt | 6 +++--- src/Storages/System/CMakeLists.txt | 1 + src/Storages/examples/CMakeLists.txt | 2 +- utils/zookeeper-cli/CMakeLists.txt | 1 + utils/zookeeper-dump-tree/CMakeLists.txt | 1 + utils/zookeeper-remove-by-list/CMakeLists.txt | 1 + 12 files changed, 23 insertions(+), 6 deletions(-) diff --git a/programs/extract-from-config/CMakeLists.txt b/programs/extract-from-config/CMakeLists.txt index ff2d7937117..50585a564a7 100644 --- a/programs/extract-from-config/CMakeLists.txt +++ b/programs/extract-from-config/CMakeLists.txt @@ -5,6 +5,7 @@ set (CLICKHOUSE_EXTRACT_FROM_CONFIG_LINK boost::program_options clickhouse_common_config clickhouse_common_io + clickhouse_common_zookeeper_base clickhouse_common_zookeeper ) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 9b931c49c24..eaba1581ee4 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -7,6 +7,7 @@ set (CLICKHOUSE_KEEPER_LINK PRIVATE clickhouse_common_config clickhouse_common_io + clickhouse_common_zookeeper_base clickhouse_common_zookeeper daemon clickhouse-keeper-converter-lib diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index be696ff2afe..d3565211d14 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -8,6 +8,7 @@ set (CLICKHOUSE_SERVER_LINK clickhouse_aggregate_functions clickhouse_common_config clickhouse_common_io + clickhouse_common_zookeeper_base clickhouse_common_zookeeper clickhouse_functions clickhouse_parsers diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ac03f40cd93..a3631c856cb 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -415,6 +415,7 @@ dbms_target_link_libraries ( boost::filesystem boost::program_options clickhouse_common_config + clickhouse_common_zookeeper_base clickhouse_common_zookeeper clickhouse_dictionaries_embedded clickhouse_parsers @@ -660,6 +661,7 @@ if (ENABLE_TESTS) clickhouse_parsers clickhouse_storages_system dbms + clickhouse_common_zookeeper_base clickhouse_common_config clickhouse_common_zookeeper hilite_comparator) diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index e91a01568d5..05c2663b3f5 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -11,6 +11,7 @@ set (SRCS add_library(clickhouse_common_config ${SRCS}) target_link_libraries(clickhouse_common_config PUBLIC + clickhouse_common_zookeeper_base clickhouse_common_zookeeper common Poco::XML diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index b70a2299ba6..20a490ce085 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -1,10 +1,13 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) +# Needed to build without Keeper (FreeBSD and other builds) +list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) # Needs to be built differently depending on ZOOKEEPER_LOG list(REMOVE_ITEM clickhouse_common_zookeeper_sources "ZooKeeperImpl.cpp") + add_library(clickhouse_common_zookeeper_base ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_link_libraries (clickhouse_common_zookeeper_base PUBLIC @@ -18,14 +21,18 @@ add_library(clickhouse_common_zookeeper ZooKeeperImpl.cpp) target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) target_link_libraries (clickhouse_common_zookeeper PUBLIC - clickhouse_common_zookeeper_base + clickhouse_common_io + clickhouse_compression + common ) # for examples -- no logging (to avoid extra dependencies) add_library(clickhouse_common_zookeeper_no_log ZooKeeperImpl.cpp) target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC - clickhouse_common_zookeeper_base + clickhouse_common_io + clickhouse_compression + common ) if (ENABLE_EXAMPLES) add_subdirectory(examples) diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 678b302a512..dd738e993de 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -1,15 +1,15 @@ clickhouse_add_executable(zkutil_test_commands zkutil_test_commands.cpp) target_link_libraries(zkutil_test_commands PRIVATE - clickhouse_common_zookeeper_no_log + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log dbms) clickhouse_add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) target_link_libraries(zkutil_test_commands_new_lib PRIVATE - clickhouse_common_zookeeper_no_log + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log clickhouse_compression dbms) clickhouse_add_executable(zkutil_test_async zkutil_test_async.cpp) target_link_libraries(zkutil_test_async PRIVATE - clickhouse_common_zookeeper_no_log + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log dbms) diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index c7583713d2d..90fc7bf0541 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -47,6 +47,7 @@ add_library(clickhouse_storages_system ${storages_system_sources}) target_link_libraries(clickhouse_storages_system PRIVATE dbms common + clickhouse_common_zookeeper_base clickhouse_common_zookeeper clickhouse_parsers Poco::JSON diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index b4786b7313b..0c9a5b46801 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -5,4 +5,4 @@ clickhouse_add_executable (merge_selector2 merge_selector2.cpp) target_link_libraries (merge_selector2 PRIVATE dbms) clickhouse_add_executable (get_current_inserts_in_replicated get_current_inserts_in_replicated.cpp) -target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) +target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper_base clickhouse_common_zookeeper) diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index fd2fa669f40..2d0769b7bf2 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -2,6 +2,7 @@ clickhouse_add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp ${ClickHouse_SOURCE_DIR}/src/Client/LineReader.cpp) target_link_libraries(clickhouse-zookeeper-cli PRIVATE + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log dbms clickhouse_functions diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index 3f3df65776a..835d37bd1cd 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,5 +1,6 @@ clickhouse_add_executable (zookeeper-dump-tree main.cpp ${SRCS}) target_link_libraries(zookeeper-dump-tree PRIVATE + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log clickhouse_common_io dbms diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index a4d7dccef65..4365c716596 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,5 +1,6 @@ clickhouse_add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) target_link_libraries(zookeeper-remove-by-list PRIVATE + clickhouse_common_zookeeper_base clickhouse_common_zookeeper_no_log dbms clickhouse_functions From 33e5bc136a873f8668cc61a1bce221df57ccb389 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 21:24:10 +0100 Subject: [PATCH 308/433] Remove build hack --- src/Common/ZooKeeper/CMakeLists.txt | 2 -- src/Common/ZooKeeper/IKeeper.h | 2 +- src/{Coordination => Common/ZooKeeper}/KeeperFeatureFlags.cpp | 2 +- src/{Coordination => Common/ZooKeeper}/KeeperFeatureFlags.h | 0 src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperContext.cpp | 2 +- src/Coordination/KeeperContext.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Storages/System/StorageSystemZooKeeperConnection.cpp | 2 +- 13 files changed, 11 insertions(+), 13 deletions(-) rename src/{Coordination => Common/ZooKeeper}/KeeperFeatureFlags.cpp (98%) rename src/{Coordination => Common/ZooKeeper}/KeeperFeatureFlags.h (100%) diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 20a490ce085..12aa3270359 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -1,8 +1,6 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) -# Needed to build without Keeper (FreeBSD and other builds) -list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) # Needs to be built differently depending on ZOOKEEPER_LOG list(REMOVE_ITEM clickhouse_common_zookeeper_sources "ZooKeeperImpl.cpp") diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index d6f1716c781..bc949aefa41 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Common/ZooKeeper/KeeperFeatureFlags.cpp similarity index 98% rename from src/Coordination/KeeperFeatureFlags.cpp rename to src/Common/ZooKeeper/KeeperFeatureFlags.cpp index 2aad6cbed32..5c875c2d163 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Common/ZooKeeper/KeeperFeatureFlags.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Common/ZooKeeper/KeeperFeatureFlags.h similarity index 100% rename from src/Coordination/KeeperFeatureFlags.h rename to src/Common/ZooKeeper/KeeperFeatureFlags.h diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 4cf08c9c8f6..52b045cdd3e 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace Coordination diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5bc9ad5df0a..9ce927b9715 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1,5 +1,5 @@ #include "ZooKeeper.h" -#include "Coordination/KeeperFeatureFlags.h" +#include "Common/ZooKeeper/KeeperFeatureFlags.h" #include "ZooKeeperImpl.h" #include "KeeperException.h" #include "TestKeeper.h" diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 5782c49a7f3..fb693b7b267 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 20926bbd5fd..5a1695b34a9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 728dca75bad..7643bd95420 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -11,7 +11,7 @@ #include #include #include -#include "Coordination/KeeperFeatureFlags.h" +#include "Common/ZooKeeper/KeeperFeatureFlags.h" #include #include #include diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 167f40bc991..a634ba87599 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index a13459d7c65..02cdd9809d7 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 4cd0eaa8657..7e4f44f6194 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 9a83e518058..c75efdb865d 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include From b78238ca302a8839209d7764aeb1532ea7e8a5d2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Nov 2024 20:51:40 +0000 Subject: [PATCH 309/433] Rename setting --- src/Core/Settings.cpp | 4 ++-- src/Core/SettingsChangesHistory.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0d9c8fbd1f0..61cee5d9ae5 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5649,8 +5649,8 @@ Parts virtually divided into segments to be distributed between replicas for par DECLARE(Bool, parallel_replicas_local_plan, true, R"( Build local plan for local replica )", BETA) \ - DECLARE(Bool, parallel_replicas_skip_index_analysis_on_workers, true, R"( -Skip index analysis on workers. Effective only with enabled parallel_replicas_local_plan + DECLARE(Bool, parallel_replicas_index_analysis_only_on_coordinator, true, R"( +Index analysis done only on replica-coordinator and skipped on other replicas. Effective only with enabled parallel_replicas_local_plan )", BETA) \ \ DECLARE(Bool, allow_experimental_analyzer, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 193c60b1aa8..65fe6138150 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_listgetSettingsRef(); if (context->canUseParallelReplicasOnFollower() && settings[Setting::parallel_replicas_local_plan] - && settings[Setting::parallel_replicas_skip_index_analysis_on_workers]) + && settings[Setting::parallel_replicas_index_analysis_only_on_coordinator]) { // Skip index analysis and return parts with all marks // The coordinator will chose ranges to read for workers based on index analysis on its side From 38132c1451965483ac253b1a49cc32b5db15a2a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Nov 2024 21:32:57 +0000 Subject: [PATCH 310/433] Update test_parallel_replicas_protocol --- .../test_parallel_replicas_protocol/test.py | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_parallel_replicas_protocol/test.py b/tests/integration/test_parallel_replicas_protocol/test.py index 2ed39a3273f..f688db071a6 100644 --- a/tests/integration/test_parallel_replicas_protocol/test.py +++ b/tests/integration/test_parallel_replicas_protocol/test.py @@ -1,4 +1,5 @@ import re +import uuid from random import randint import pytest @@ -18,15 +19,6 @@ nodes = [ ] -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - def _create_tables(table_name): nodes[0].query( f"DROP TABLE IF EXISTS {table_name} ON CLUSTER 'parallel_replicas'", @@ -48,14 +40,30 @@ def _create_tables(table_name): nodes[0].query(f"SYSTEM SYNC REPLICA ON CLUSTER 'parallel_replicas' {table_name}") +table_name = "t" + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + _create_tables(table_name) + yield cluster + finally: + cluster.shutdown() + + # now mark_segment_size is part of the protocol and is communicated to the initiator. # let's check that the correct value is actually used by the coordinator -def test_mark_segment_size_communicated_correctly(start_cluster): - table_name = "t" - _create_tables(table_name) +@pytest.mark.parametrize( + "local_plan", [0,1] +) +@pytest.mark.parametrize( + "index_analysis_only_on_coordinator", [0,1] +) +def test_mark_segment_size_communicated_correctly(start_cluster, local_plan, index_analysis_only_on_coordinator): for local_plan in [0, 1]: - query_id = f"query_id_{randint(0, 1000000)}" + query_id = f"query_id_{str(uuid.uuid4())}" nodes[0].query( f"SELECT sum(value) FROM {table_name}", settings={ @@ -65,6 +73,7 @@ def test_mark_segment_size_communicated_correctly(start_cluster): "parallel_replicas_mark_segment_size": 0, "parallel_replicas_local_plan": local_plan, "query_id": query_id, + "parallel_replicas_index_analysis_only_on_coordinator": index_analysis_only_on_coordinator, }, ) From 35402308eb0b63f73f18834f33544a38958c8b3d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Nov 2024 22:33:57 +0100 Subject: [PATCH 311/433] fix tests --- .../0_stateless/00826_cross_to_inner_join.sql | 10 +++++----- .../0_stateless/02000_join_on_const.reference | 20 +++++++++---------- .../0_stateless/02000_join_on_const.sql | 12 +++++------ 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index 7216c7ac9b9..ac0439c08e2 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a order by all; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b order by all; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,12 +41,12 @@ SELECT '--- is null or ---'; select * from t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 order by all; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a order by all; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b order by all; SELECT '--- comma and or ---'; SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) ORDER BY ALL; diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 3bd1633ce32..6048b601935 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,27 +33,27 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -2 2 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; +0 3 +2 2 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; +0 2 +0 3 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; 1 0 2 0 -0 2 -0 3 SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 1 ('',0) diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index da70973ed87..fd914d21666 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,13 +73,13 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 order by all SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 order by all SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; From 8226903918258875531a45f99e87a6d9242160e9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 25 Nov 2024 21:40:50 +0000 Subject: [PATCH 312/433] Automatic style fix --- .../test_parallel_replicas_protocol/test.py | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_parallel_replicas_protocol/test.py b/tests/integration/test_parallel_replicas_protocol/test.py index f688db071a6..ebfc21f16a6 100644 --- a/tests/integration/test_parallel_replicas_protocol/test.py +++ b/tests/integration/test_parallel_replicas_protocol/test.py @@ -42,6 +42,7 @@ def _create_tables(table_name): table_name = "t" + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: @@ -54,13 +55,11 @@ def start_cluster(): # now mark_segment_size is part of the protocol and is communicated to the initiator. # let's check that the correct value is actually used by the coordinator -@pytest.mark.parametrize( - "local_plan", [0,1] -) -@pytest.mark.parametrize( - "index_analysis_only_on_coordinator", [0,1] -) -def test_mark_segment_size_communicated_correctly(start_cluster, local_plan, index_analysis_only_on_coordinator): +@pytest.mark.parametrize("local_plan", [0, 1]) +@pytest.mark.parametrize("index_analysis_only_on_coordinator", [0, 1]) +def test_mark_segment_size_communicated_correctly( + start_cluster, local_plan, index_analysis_only_on_coordinator +): for local_plan in [0, 1]: query_id = f"query_id_{str(uuid.uuid4())}" From 6c8a1bb597d48b9d85ce4c252feda6cff1e9803f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Nov 2024 02:43:19 +0100 Subject: [PATCH 313/433] Add changelog for 24.11 --- CHANGELOG.md | 82 +++++++++++++++++++++++++--------------------------- 1 file changed, 39 insertions(+), 43 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cd3a5c83878..1a9fe98d446 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,7 +26,7 @@ * When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). #### Experimental feature -* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). +* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). * Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). * Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). * Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). @@ -34,77 +34,74 @@ * Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). #### New Feature +* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). * A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). -* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). -* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). -* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). -* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). -* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). -* Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). -* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). -* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). -* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). * Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). +* Add support for staleness clause in the ORDER BY WITH FILL operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). +* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). #### Performance Improvement -* Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). -* Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). +* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). +* Now we don't copy input blocks columns for `join_algorithm = 'parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized `Replacing` merge algorithm for non-intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). * Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). +* For the `plain_rewritable` disks: Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* Improve the performance and accuracy of `system.query_metric_log` collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). * Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). +* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). +* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). +* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). +* Turn off filesystem cache setting `boundary_alignment` for non-disk read, which improves performance of reading from standalone remote files with caching. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Queries like `SELECT * FROM table LIMIT ...` used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). #### Improvement +* Allow using clickhouse with a file argument as `ch queries.sql`. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). +* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). +* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). +* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). +* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). -* Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). +* Added option to check if the object exists after writing it to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). * Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). * Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). * Wait only on active replicas for database ON CLUSTER queries if distributed_ddl_output_mode is set to be *_only_active. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). * Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). * Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). -* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). * Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). -* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). -* Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). -* Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). -* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). -* Boolean support for parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). +* Make the client history size configurable and increase its default size. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). +* Boolean types support for the parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). * Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). +* Support writing HDFS files with spaces. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). * Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). * Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Remove the metadata_version ZooKeeper node creation from ReplicatedMergeTree restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). * Add per host dashboards `Overview (host)` and `Cloud overview (host)` to advanced dashboard. [#71422](https://github.com/ClickHouse/ClickHouse/pull/71422) ([alesapin](https://github.com/alesapin)). -* The methods `removeObject` and `removeObjects` are not idempotent. When retries happen due to network errors, the result could be `object not found` because it has been deleted at previous attempts. [#71529](https://github.com/ClickHouse/ClickHouse/pull/71529) ([Sema Checherinda](https://github.com/CheSema)). -* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). -* Allow using clickhouse with a file argument as --queries-file. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). -* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). * `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). * Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). * Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). -* Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Turn-off filesystem cache setting `boundary_alignment` for non-disk read. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). -* Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). -* Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). +* S3Queue and AzureQueue: Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `HostResolver` three times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). * On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). +* Check if default database is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). #### Bug Fix (user-visible misbehavior in an official stable release) * The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). -* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix for the bug when DateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). * Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). @@ -117,11 +114,9 @@ * Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). * Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). * Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). -* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). * Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). * Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). -* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). * Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). * SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). * Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -130,18 +125,18 @@ * Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). * Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). -* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). +* Add try/catch to data parts destructors to avoid std::terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). * Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). * Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). -* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). +* Fix error Invalid number of rows in Chunk with the Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix error column "attgenerated" does not exist for older PostgreSQL versions, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). * To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). * Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). * Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). * Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). -* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). +* Fix Date32 out of range caused by uninitialized ORC data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). * Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). * Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). * Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). @@ -152,13 +147,12 @@ * Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). * Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). * Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). -* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). * Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). * Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). * Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). * Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). -* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). +* Fix the TOO_LARGE_ARRAY_SIZE exception caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). * `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). * Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). @@ -172,6 +166,8 @@ * Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). * Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). * Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* This is a fix for "zero-copy" replication, which is unsupported and will be removed entirely. Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). +* This is a fix for "zero-copy" replication, which is unsupported and will be removed entirely. Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). ### ClickHouse release 24.10, 2024-10-31 From 1653d1d828fc2474375fcc672c8def5f4893e8ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Nov 2024 02:56:15 +0100 Subject: [PATCH 314/433] Add changelog for 24.11 --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1a9fe98d446..21a2c517cc9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -58,6 +58,7 @@ * Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). * Turn off filesystem cache setting `boundary_alignment` for non-disk read, which improves performance of reading from standalone remote files with caching. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). * Queries like `SELECT * FROM table LIMIT ...` used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). +* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). #### Improvement * Allow using clickhouse with a file argument as `ch queries.sql`. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). @@ -74,7 +75,6 @@ * Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). * Wait only on active replicas for database ON CLUSTER queries if distributed_ddl_output_mode is set to be *_only_active. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). * Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). -* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). * Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). * Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). * Make the client history size configurable and increase its default size. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). From f7833ae41948061cfa78ed0eb13df7e7fa2e81a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Nov 2024 02:56:48 +0100 Subject: [PATCH 315/433] Add changelog for 24.11 --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 21a2c517cc9..5dfc32d2821 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,7 +72,7 @@ * Added option to check if the object exists after writing it to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). * Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). -* Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). +* Report hosts running distributed DDL queries by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). * Wait only on active replicas for database ON CLUSTER queries if distributed_ddl_output_mode is set to be *_only_active. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). * Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). * Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). From 66b874dafd769bb6ad9df5e4f040305175683f89 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 26 Nov 2024 03:34:17 +0100 Subject: [PATCH 316/433] Omit database names from backup query when we do not need to specify it --- src/Parsers/ASTBackupQuery.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 0372d0e16da..9766190fe45 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -46,7 +46,7 @@ namespace } } - void formatExceptTables(const std::set & except_tables, const IAST::FormatSettings & format) + void formatExceptTables(const std::set & except_tables, const IAST::FormatSettings & format, bool only_table_names=false) { if (except_tables.empty()) return; @@ -60,7 +60,7 @@ namespace if (std::exchange(need_comma, true)) format.ostr << ", "; - if (!table_name.first.empty()) + if (!table_name.first.empty() && !only_table_names) format.ostr << backQuoteIfNeed(table_name.first) << "."; format.ostr << backQuoteIfNeed(table_name.second); } @@ -117,7 +117,7 @@ namespace format.ostr << backQuoteIfNeed(element.new_database_name); } - formatExceptTables(element.except_tables, format); + formatExceptTables(element.except_tables, format, /*only_table_names*/true); break; } From 81f6e993e9723bb89f5ce4b68a5eb61774c4ab76 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 26 Nov 2024 03:45:15 +0100 Subject: [PATCH 317/433] Allow parsing both variants of BACKUP EXCEPT TABLES query --- src/Parsers/ParserBackupQuery.cpp | 31 +++++++++++++++++++------------ 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 6d2f4d8311d..f6993fdb811 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -16,6 +17,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + namespace { using Kind = ASTBackupQuery::Kind; @@ -78,19 +84,20 @@ namespace auto parse_list_element = [&] { DatabaseAndTableName table_name; - if (database_name) - { - ASTPtr ast; - if (!ParserIdentifier{}.parse(pos, ast, expected)) - return false; + + if (!parseDatabaseAndTableName(pos, expected, table_name.first, table_name.second)) + return false; + + if (database_name && table_name.first.empty()) table_name.first = *database_name; - table_name.second = getIdentifierName(ast); - } - else - { - if (!parseDatabaseAndTableName(pos, expected, table_name.first, table_name.second)) - return false; - } + + if (database_name && table_name.first != *database_name) + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Database name in EXCEPT TABLES clause doesn't match the database name in DATABASE clause: {} != {}", + table_name.first, + *database_name + ); result.emplace(std::move(table_name)); return true; From b2d358c27bca164ad3c1ddba8414b96a60b7976b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 26 Nov 2024 05:06:14 +0100 Subject: [PATCH 318/433] Add tests --- .../test_backup_restore_new/test.py | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a7a22be1cf8..af141ad4c98 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1560,6 +1560,36 @@ def test_backup_all(exclude_system_log_tables): instance.query("DROP USER u1") +@pytest.mark.parametrize("include_database_name", [False, True]) +def test_backup_database_except(include_database_name): + create_and_fill_table() + + session_id = new_session_id() + instance.query( + "CREATE TABLE test.omit_table (s String) ENGINE = MergeTree ORDER BY s", + ) + + omit_table_name = "test.omit_table" if include_database_name else "omit_table" + backup_name = new_backup_name() + backup_command = f"BACKUP DATABASE test EXCEPT TABLES {omit_table_name} TO {backup_name}" + + instance.http_query(backup_command, params={"session_id": session_id}) + + instance.query("DROP TABLE test.table") + instance.query("DROP TABLE test.omit_table") + + restore_command = f"RESTORE ALL FROM {backup_name}" + + session_id = new_session_id() + instance.http_query( + restore_command, params={"session_id": session_id}, method="POST" + ) + + assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" + assert instance.query("EXISTS TABLE test.omit_table") == "0\n" + + instance.query("DROP TABLE test.table") + def test_operation_id(): create_and_fill_table(n=30) From 31a669892ff7917ee34491329161d0e9e6fa39bf Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 26 Nov 2024 05:11:14 +0100 Subject: [PATCH 319/433] Fix --- docker/test/util/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index 75ca3448b2a..1af3f8543bd 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -4,7 +4,7 @@ FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -ARG LLVM_APT_VERSION="1:19.1.4~*" +ARG LLVM_APT_VERSION="1:19.1.4" ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=19 @@ -29,7 +29,7 @@ RUN apt-get update \ && echo "deb https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ /etc/apt/sources.list \ && apt-get update \ - && apt-get install --yes --no-install-recommends --verbose-versions llvm-${LLVM_VERSION}>=${LLVM_APT_VERSION} \ + && apt-get satisfy --yes --no-install-recommends --verbose-versions "llvm-${LLVM_VERSION} (>= ${LLVM_APT_VERSION})" \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* From 7ed67942e8ca47d7bb92cfa640dbd4ce635cc034 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Nov 2024 04:14:28 +0000 Subject: [PATCH 320/433] Automatic style fix --- tests/integration/test_backup_restore_new/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index af141ad4c98..762a4de2fbe 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1571,7 +1571,9 @@ def test_backup_database_except(include_database_name): omit_table_name = "test.omit_table" if include_database_name else "omit_table" backup_name = new_backup_name() - backup_command = f"BACKUP DATABASE test EXCEPT TABLES {omit_table_name} TO {backup_name}" + backup_command = ( + f"BACKUP DATABASE test EXCEPT TABLES {omit_table_name} TO {backup_name}" + ) instance.http_query(backup_command, params={"session_id": session_id}) @@ -1590,6 +1592,7 @@ def test_backup_database_except(include_database_name): instance.query("DROP TABLE test.table") + def test_operation_id(): create_and_fill_table(n=30) From 63fc8a37a85cfb8e73f7315b8023df9b0dc9956d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 26 Nov 2024 05:17:50 +0100 Subject: [PATCH 321/433] Fix --- docker/test/util/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index 1af3f8543bd..87a89ff4e3b 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -29,7 +29,7 @@ RUN apt-get update \ && echo "deb https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ /etc/apt/sources.list \ && apt-get update \ - && apt-get satisfy --yes --no-install-recommends --verbose-versions "llvm-${LLVM_VERSION} (>= ${LLVM_APT_VERSION})" \ + && apt-get satisfy --yes --no-install-recommends "llvm-${LLVM_VERSION} (>= ${LLVM_APT_VERSION})" \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* From 438b80e89b2bb137e1374cf6b73ed944f7c9dfbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Nov 2024 05:49:19 +0100 Subject: [PATCH 322/433] Update groupconcat.md --- .../aggregate-functions/reference/groupconcat.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md index 6a24aa244bf..71d6faac2af 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -33,9 +33,9 @@ Input table: ``` text ┌─id─┬─name─┐ -│ 1 │ John│ -│ 2 │ Jane│ -│ 3 │ Bob│ +│ 1 │ John │ +│ 2 │ Jane │ +│ 3 │ Bob │ └────┴──────┘ ``` From 3d65e72586d1dea69215f34ce378a1bddf9a31bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Nov 2024 05:49:53 +0100 Subject: [PATCH 323/433] Update groupconcat.md --- .../aggregate-functions/reference/groupconcat.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md index 71d6faac2af..de2f4a0a44b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -33,9 +33,9 @@ Input table: ``` text ┌─id─┬─name─┐ -│ 1 │ John │ -│ 2 │ Jane │ -│ 3 │ Bob │ +│ 1 │ John │ +│ 2 │ Jane │ +│ 3 │ Bob │ └────┴──────┘ ``` From 2b1b4d0516b23119b2b720a9ebaf3605e4ca6064 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 26 Nov 2024 10:53:46 +0000 Subject: [PATCH 324/433] fix test_parallel_replicas_all_marks_read --- tests/integration/test_parallel_replicas_all_marks_read/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py index 92317afabbe..c93e8c7c09b 100644 --- a/tests/integration/test_parallel_replicas_all_marks_read/test.py +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -71,7 +71,7 @@ def _get_result_with_parallel_replicas( "cluster_for_parallel_replicas": f"{cluster_name}", "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, "query_id": query_id, - "parallel_replicas_skip_index_analysis_on_workers": False, + "parallel_replicas_index_analysis_only_on_coordinator": False, }, ) From 6e0f888347103fa7d9716e02f474db45587ae331 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 26 Nov 2024 11:18:54 +0000 Subject: [PATCH 325/433] Bump Google test to latest HEAD --- contrib/googletest | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/googletest b/contrib/googletest index a7f443b80b1..35d0c365609 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit a7f443b80b105f940225332ed3c31f2790092f47 +Subproject commit 35d0c365609296fa4730d62057c487e3cfa030ff From 6762c30a883f29fb57c85a7a1c744ff2b7e1b0dc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 26 Nov 2024 11:02:49 +0000 Subject: [PATCH 326/433] Use std::string::contains where possible --- .clang-tidy | 2 +- programs/disks/DisksClient.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- .../Config/AbstractConfigurationComparison.cpp | 2 +- src/Common/FileRenamer.cpp | 2 +- src/Common/Macros.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 +- src/Common/getNumberOfCPUCoresToUse.cpp | 4 ++-- src/Common/mysqlxx/PoolWithFailover.cpp | 2 +- src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp | 2 +- src/Common/parseGlobs.cpp | 2 +- src/Compression/tests/gtest_compressionCodec.cpp | 2 +- .../Serializations/SerializationNullable.cpp | 7 +++---- src/Databases/DatabaseReplicated.cpp | 12 ++++++------ src/Databases/MySQL/MaterializeMetadata.cpp | 8 ++++---- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 3 +-- ...kObjectStorageRemoteMetadataRestoreHelper.cpp | 4 ++-- src/Formats/CapnProtoSchema.cpp | 4 ++-- src/Functions/FunctionsConversion.h | 3 +-- src/IO/CompressionMethod.cpp | 16 ++++++++-------- src/IO/S3/Client.cpp | 4 ++-- src/IO/S3/URI.cpp | 2 +- src/Interpreters/Cluster.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- .../GatherFunctionQuantileVisitor.cpp | 4 ++-- .../MySQL/InterpretersMySQLDDLQuery.cpp | 4 ++-- .../MySQL/tests/gtest_create_rewritten.cpp | 2 +- .../KustoFunctions/KQLDateTimeFunctions.cpp | 2 +- src/Parsers/ParserDataType.cpp | 2 +- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- src/Processors/Merges/Algorithms/Graphite.cpp | 2 +- src/Server/HTTPHandlerFactory.h | 2 +- src/Server/HTTPHandlerRequestFilter.h | 2 +- src/Server/PostgreSQLHandler.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeIndexGranularityInfo.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 2 +- src/Storages/StorageFile.cpp | 4 ++-- src/Storages/StorageURL.cpp | 4 ++-- src/Storages/System/StorageSystemZooKeeper.cpp | 2 +- 42 files changed, 67 insertions(+), 70 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index d7e35f16ad5..8c079b9692f 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -10,7 +10,7 @@ HeaderFilterRegex: '^.*/(base|src|programs|utils)/.*(h|hpp)$' Checks: [ '*', - '-abseil-*', + '-abseil-string-find-str-contains', # disabled to avoid a misleading suggestion (obsolete absl::StrContains() instead of C++23 std::string::contains()) '-altera-*', diff --git a/programs/disks/DisksClient.cpp b/programs/disks/DisksClient.cpp index dcfb51c420e..4a0c758c686 100644 --- a/programs/disks/DisksClient.cpp +++ b/programs/disks/DisksClient.cpp @@ -100,7 +100,7 @@ void DiskWithPath::setPath(const String & any_path) String DiskWithPath::validatePathAndGetAsRelative(const String & path) { String lexically_normal_path = fs::path(path).lexically_normal(); - if (lexically_normal_path.find("..") != std::string::npos) + if (lexically_normal_path.contains("..")) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Path {} is not normalized", path); /// If path is absolute we should keep it as relative inside disk, so disk will look like diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c0f5744a4d5..93dea07a43f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2614,7 +2614,7 @@ bool ClientBase::addMergeTreeSettings(ASTCreateQuery & ast_create) || !ast_create.storage || !ast_create.storage->isExtendedStorageDefinition() || !ast_create.storage->engine - || ast_create.storage->engine->name.find("MergeTree") == std::string::npos) + || !ast_create.storage->engine->name.contains("MergeTree")) return false; auto all_changed = cmd_merge_tree_settings.changes(); diff --git a/src/Common/Config/AbstractConfigurationComparison.cpp b/src/Common/Config/AbstractConfigurationComparison.cpp index 73f305ce669..9001a3659dc 100644 --- a/src/Common/Config/AbstractConfigurationComparison.cpp +++ b/src/Common/Config/AbstractConfigurationComparison.cpp @@ -41,7 +41,7 @@ namespace #if defined(DEBUG_OR_SANITIZER_BUILD) /// Compound `ignore_keys` are not yet implemented. for (const auto & ignore_key : *ignore_keys) - chassert(ignore_key.find('.') == std::string_view::npos); + chassert(!ignore_key.contains('.')); #endif } diff --git a/src/Common/FileRenamer.cpp b/src/Common/FileRenamer.cpp index b43b870b94e..f500c89dcc8 100644 --- a/src/Common/FileRenamer.cpp +++ b/src/Common/FileRenamer.cpp @@ -38,7 +38,7 @@ String FileRenamer::generateNewFilename(const String & filename) const // Get current timestamp in microseconds String timestamp; - if (rule.find("%t") != String::npos) + if (rule.contains("%t")) { auto now = std::chrono::system_clock::now(); timestamp = std::to_string(timeInMicroseconds(now)); diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 4b5300985e3..cd0fef21283 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -53,7 +53,7 @@ String Macros::expand(const String & s, /// Do not allow recursion if we expand only special macros, because it will be infinite recursion assert(info.level == 0 || !info.expand_special_macros_only); - if (s.find('{') == String::npos) + if (!s.contains('{')) return s; if (info.level && s.size() > 65536) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 00f095cd0e3..32dfa4261bd 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -206,7 +206,7 @@ std::string ZooKeeperAuthRequest::toStringImpl(bool /*short_format*/) const void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const { /// See https://github.com/ClickHouse/clickhouse-private/issues/3029 - if (path.starts_with("/clickhouse/tables/") && path.find("/parts/") != std::string::npos) + if (path.starts_with("/clickhouse/tables/") && path.contains("/parts/")) { LOG_TRACE(getLogger(__PRETTY_FUNCTION__), "Creating part at path {}", path); } diff --git a/src/Common/getNumberOfCPUCoresToUse.cpp b/src/Common/getNumberOfCPUCoresToUse.cpp index e6eff773a9d..77ac726cd1d 100644 --- a/src/Common/getNumberOfCPUCoresToUse.cpp +++ b/src/Common/getNumberOfCPUCoresToUse.cpp @@ -143,13 +143,13 @@ try std::string key = line.substr(0, pos); std::string val = line.substr(pos + 1); - if (key.find("physical id") != std::string::npos) + if (key.contains("physical id")) { cur_core_entry.first = std::stoi(val); continue; } - if (key.find("core id") != std::string::npos) + if (key.contains("core id")) { cur_core_entry.second = std::stoi(val); core_entries.insert(cur_core_entry); diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index bbf077d3aa1..8c7dae739b7 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -165,7 +165,7 @@ PoolWithFailover::Entry PoolWithFailover::get() } catch (const Poco::Exception & e) { - if (e.displayText().find("mysqlxx::Pool is full") != std::string::npos) /// NOTE: String comparison is trashy code. + if (e.displayText().contains("mysqlxx::Pool is full")) /// NOTE: String comparison is trashy code. { full_pool = &pool; } diff --git a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp index 121767edc84..5c7473f721b 100644 --- a/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp +++ b/src/Common/mysqlxx/tests/mysqlxx_pool_test.cpp @@ -26,7 +26,7 @@ mysqlxx::Pool::Entry getWithFailover(mysqlxx::Pool & connections_pool) } catch (const Poco::Exception & e) { - if (e.displayText().find("mysqlxx::Pool is full") != std::string::npos) + if (e.displayText().contains("mysqlxx::Pool is full")) { std::cerr << e.displayText() << std::endl; } diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 72e67619859..de6caec3149 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -46,7 +46,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob std::string buffer(matched); oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; - if (buffer.find(',') == std::string::npos) + if (!buffer.contains(',')) { size_t range_begin = 0; size_t range_end = 0; diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 8265ba63fc2..4317eb413ac 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -521,7 +521,7 @@ public: TEST_P(CodecTest, TranscodingWithDataType) { /// Gorilla can only be applied to floating point columns - bool codec_is_gorilla = std::get<0>(GetParam()).codec_statement.find("Gorilla") != std::string::npos; + bool codec_is_gorilla = std::get<0>(GetParam()).codec_statement.contains("Gorilla"); WhichDataType which(std::get<1>(GetParam()).data_type.get()); bool data_is_float = which.isFloat(); if (codec_is_gorilla && !data_is_float) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index e72dd3a42f5..cf12ed5972a 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -370,10 +370,10 @@ ReturnType deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr if constexpr (!throw_exception) return ReturnType(false); - if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos) + if (null_representation.contains('\t') || null_representation.contains('\n')) throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " "containing '\\t' or '\\n' may not work correctly for large input."); - if (settings.tsv.crlf_end_of_line_input && null_representation.find('\r') != std::string::npos) + if (settings.tsv.crlf_end_of_line_input && null_representation.contains('\r')) throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation " "containing '\\r' may not work correctly for large input."); @@ -747,8 +747,7 @@ ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const For if constexpr (!throw_exception) return ReturnType(false); - if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos - || null_representation.find('\n') != std::string::npos) + if (null_representation.contains(settings.csv.delimiter) || null_representation.contains('\r') || null_representation.contains('\n')) throw DB::Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing " "format_csv_delimiter, '\\r' or '\\n' may not work correctly for large input."); diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 8992a9d8548..9ea6ec27df1 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -140,9 +140,9 @@ DatabaseReplicated::DatabaseReplicated( { if (zookeeper_path.empty() || shard_name.empty() || replica_name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path, shard and replica names must be non-empty"); - if (shard_name.find('/') != std::string::npos || replica_name.find('/') != std::string::npos) + if (shard_name.contains('/') || replica_name.contains('/')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Shard and replica names should not contain '/'"); - if (shard_name.find('|') != std::string::npos || replica_name.find('|') != std::string::npos) + if (shard_name.contains('|') || replica_name.contains('|')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Shard and replica names should not contain '|'"); if (zookeeper_path.back() == '/') @@ -1105,9 +1105,9 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, Contex static UUID getTableUUIDIfReplicated(const String & metadata, ContextPtr context) { - bool looks_like_replicated = metadata.find("Replicated") != std::string::npos; - bool looks_like_shared = metadata.find("Shared") != std::string::npos; - bool looks_like_merge_tree = metadata.find("MergeTree") != std::string::npos; + bool looks_like_replicated = metadata.contains("Replicated"); + bool looks_like_shared = metadata.contains("Shared"); + bool looks_like_merge_tree = metadata.contains("MergeTree"); if (!(looks_like_replicated || looks_like_shared) || !looks_like_merge_tree) return UUIDHelpers::Nil; @@ -1539,7 +1539,7 @@ void DatabaseReplicated::dropReplica( String full_replica_name = shard.empty() ? replica : getFullReplicaName(shard, replica); - if (full_replica_name.find('/') != std::string::npos) + if (full_replica_name.contains('/')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid replica name, '/' is not allowed: {}", full_replica_name); auto zookeeper = Context::getGlobalContextInstance()->getZooKeeper(); diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index b187b2904f2..52a33f2872f 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -165,11 +165,11 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne grants_query = (*block.getByPosition(0).column)[index].safeGet(); out << grants_query << "; "; sub_privs = grants_query.substr(0, grants_query.find(" ON ")); - if (sub_privs.find("ALL PRIVILEGES") == std::string::npos) + if (!sub_privs.contains("ALL PRIVILEGES")) { - if ((sub_privs.find("RELOAD") != std::string::npos and - sub_privs.find("REPLICATION SLAVE") != std::string::npos and - sub_privs.find("REPLICATION CLIENT") != std::string::npos)) + if ((sub_privs.contains("RELOAD") and + sub_privs.contains("REPLICATION SLAVE") and + sub_privs.contains("REPLICATION CLIENT"))) return true; } else diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 8d5a6998d80..b4a87b60117 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -135,8 +135,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const { - if (table_name.find('\'') != std::string::npos - || table_name.find('\\') != std::string::npos) + if (table_name.contains('\'') || table_name.contains('\\')) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", table_name); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index b9f963c4590..67eddd634a5 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -367,7 +367,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * LOG_INFO(disk->log, "Calling restore for key for disk {}", object->relative_path); /// Skip file operations objects. They will be processed separately. - if (object->relative_path.find("/operations/") != String::npos) + if (object->relative_path.contains("/operations/")) continue; const auto [revision, _] = extractRevisionAndOperationFromKey(object->relative_path); @@ -541,7 +541,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject for (const auto & path : renames) { /// Skip already detached parts. - if (path.find("/detached/") != std::string::npos) + if (path.contains("/detached/")) continue; /// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them. diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp index 6076dae4157..a09d5f963a5 100644 --- a/src/Formats/CapnProtoSchema.cpp +++ b/src/Formats/CapnProtoSchema.cpp @@ -43,10 +43,10 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn /// That's not good to determine the type of error by its description, but /// this is the only way to do it here, because kj doesn't specify the type of error. auto description = std::string_view(e.getDescription().cStr()); - if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos || description.find("no such file") != String::npos) + if (description.contains("No such file or directory") || description.contains("no such directory") || description.contains("no such file")) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); - if (description.find("Parse error") != String::npos) + if (description.contains("Parse error")) throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 09640bc1d2b..2a0f349e25a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2473,8 +2473,7 @@ public: if (!isStringOrFixedString(arguments[0].type)) { - if (this->getName().find("OrZero") != std::string::npos || - this->getName().find("OrNull") != std::string::npos) + if (this->getName().contains("OrZero") || this->getName().contains("OrNull")) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " "Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument", arguments[0].type->getName(), getName()); diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 82a7a0d6340..e6b7df5b73f 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -58,21 +58,21 @@ CompressionMethod chooseHTTPCompressionMethod(const std::string & list) { /// The compression methods are ordered from most to least preferred. - if (std::string::npos != list.find("zstd")) + if (list.contains("zstd")) return CompressionMethod::Zstd; - if (std::string::npos != list.find("br")) + if (list.contains("br")) return CompressionMethod::Brotli; - if (std::string::npos != list.find("lz4")) + if (list.contains("lz4")) return CompressionMethod::Lz4; - if (std::string::npos != list.find("snappy")) + if (list.contains("snappy")) return CompressionMethod::Snappy; - if (std::string::npos != list.find("gzip")) + if (list.contains("gzip")) return CompressionMethod::Gzip; - if (std::string::npos != list.find("deflate")) + if (list.contains("deflate")) return CompressionMethod::Zlib; - if (std::string::npos != list.find("xz")) + if (list.contains("xz")) return CompressionMethod::Xz; - if (std::string::npos != list.find("bz2")) + if (list.contains("bz2")) return CompressionMethod::Bzip2; return CompressionMethod::None; } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 088087458c7..84eb8ff20a3 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -156,10 +156,10 @@ namespace ProviderType deduceProviderType(const std::string & url) { - if (url.find(".amazonaws.com") != std::string::npos) + if (url.contains(".amazonaws.com")) return ProviderType::AWS; - if (url.find("storage.googleapis.com") != std::string::npos) + if (url.contains("storage.googleapis.com")) return ProviderType::GCS; return ProviderType::UNKNOWN; diff --git a/src/IO/S3/URI.cpp b/src/IO/S3/URI.cpp index aefe3ff338c..e2a10b78733 100644 --- a/src/IO/S3/URI.cpp +++ b/src/IO/S3/URI.cpp @@ -99,7 +99,7 @@ URI::URI(const std::string & uri_, bool allow_archive_path_syntax) /// '?' can not be used as a wildcard, otherwise it will be ambiguous. /// If no "versionId" in the http parameter, '?' can be used as a wildcard. /// It is necessary to encode '?' to avoid deletion during parsing path. - if (!has_version_id && uri_.find('?') != String::npos) + if (!has_version_id && uri_.contains('?')) { String uri_with_question_mark_encode; Poco::URI::encode(uri_, "?", uri_with_question_mark_encode); diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 006e3f75937..910a1f9b4ea 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -383,7 +383,7 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf continue; } - if (key.find('.') != String::npos) + if (key.contains('.')) throw Exception(ErrorCodes::SYNTAX_ERROR, "Cluster names with dots are not supported: '{}'", key); /// If old config is set and cluster config wasn't changed, don't update this cluster. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 78d41a336b6..d50fd66f3ac 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3893,7 +3893,7 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const auto zookeeper = shared->auxiliary_zookeepers.find(name); if (zookeeper == shared->auxiliary_zookeepers.end()) { - if (name.find(':') != std::string::npos || name.find('/') != std::string::npos) + if (name.contains(':') || name.contains('/')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid auxiliary ZooKeeper name {}: ':' and '/' are not allowed", name); const auto & config = shared->auxiliary_zookeepers_config ? *shared->auxiliary_zookeepers_config : getConfigRef(); diff --git a/src/Interpreters/GatherFunctionQuantileVisitor.cpp b/src/Interpreters/GatherFunctionQuantileVisitor.cpp index 6b6dc362771..03bd68e1a09 100644 --- a/src/Interpreters/GatherFunctionQuantileVisitor.cpp +++ b/src/Interpreters/GatherFunctionQuantileVisitor.cpp @@ -71,12 +71,12 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr if (arguments.size() != (need_two_args ? 2 : 1)) return; - if (arguments[0]->getColumnName().find(',') != std::string::npos) + if (arguments[0]->getColumnName().contains(',')) return; String arg_name = arguments[0]->getColumnName(); if (need_two_args) { - if (arguments[1]->getColumnName().find(',') != std::string::npos) + if (arguments[1]->getColumnName().contains(',')) return; arg_name += "," + arguments[1]->getColumnName(); } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index c7fd800cacc..26092da09b8 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -103,7 +103,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) if (is_unsigned) { /// For example(in MySQL): CREATE TABLE test(column_name INT NOT NULL ... UNSIGNED) - if (type_name_upper.find("INT") != String::npos && !endsWith(type_name_upper, "SIGNED") + if (type_name_upper.contains("INT") && !endsWith(type_name_upper, "SIGNED") && !endsWith(type_name_upper, "UNSIGNED")) data_type_node->name = type_name_upper + " UNSIGNED"; } @@ -115,7 +115,7 @@ NamesAndTypesList getColumnsList(const ASTExpressionList * columns_definition) /// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3) /// Elements on a position further than 32767 are assigned negative values, starting with -32768. /// Note: Enum would be transformed to Enum8 if number of elements is less then 128, otherwise it would be transformed to Enum16. - if (type_name_upper.find("ENUM") != String::npos) + if (type_name_upper.contains("ENUM")) { UInt16 i = 0; for (ASTPtr & child : data_type_node->arguments->children) diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 81e6e6a8761..43b17c3a606 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -62,7 +62,7 @@ TEST(MySQLCreateRewritten, ColumnsDataType) MATERIALIZEDMYSQL_TABLE_COLUMNS + ") ENGINE = " "ReplacingMergeTree(_version) PARTITION BY intDiv(key, 4294967) ORDER BY tuple(key)"); - if (Poco::toUpper(test_type).find("INT") != std::string::npos) + if (Poco::toUpper(test_type).contains("INT")) { EXPECT_EQ(queryToString(tryRewrittenCreateQuery( "CREATE TABLE `test_database`.`test_table_1`(`key` INT NOT NULL PRIMARY KEY, test " + test_type + " UNSIGNED)", context_holder.context)), diff --git a/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp index f059fd9aa6b..a8af4d1bb69 100644 --- a/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp +++ b/src/Parsers/Kusto/KustoFunctions/KQLDateTimeFunctions.cpp @@ -331,7 +331,7 @@ bool FormatDateTime::convertImpl(String & out, IParser::Pos & pos) i = i + arg.size(); } } - if (decimal > 0 && formatspecifier.find('.') != String::npos) + if (decimal > 0 && formatspecifier.contains('.')) { out = std::format( "concat(" diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index d86b659df90..cd2dfcf2fa1 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -189,7 +189,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (ParserKeyword(Keyword::PRECISION).ignore(pos)) type_name_suffix = toStringView(Keyword::PRECISION); } - else if (type_name_upper.find("INT") != std::string::npos) + else if (type_name_upper.contains("INT")) { /// Support SIGNED and UNSIGNED integer type modifiers for compatibility with MySQL if (ParserKeyword(Keyword::SIGNED).ignore(pos, expected)) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index c89268d98fb..0eaf07bee9a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -34,7 +34,7 @@ namespace return; } constexpr std::string_view bad_delimiters = " \t\"'.UL"; - if (bad_delimiters.find(delimiter) != std::string_view::npos) + if (bad_delimiters.contains(delimiter)) throw Exception( ErrorCodes::BAD_ARGUMENTS, "CSV format may not work correctly with delimiter '{}'. Try use CustomSeparated format instead", diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index ceb86d89500..718de3655af 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -93,7 +93,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param if (params.patterns_typed) { std::string_view path_view = path; - if (path_view.find("?"sv) == std::string::npos) + if (!path_view.contains("?"sv)) return params.patterns_plain; return params.patterns_tagged; } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index cbb0cdee1dd..b3451e0a433 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -90,7 +90,7 @@ public: { addFilter([](const auto & request) { - return (request.getURI().find('?') != std::string::npos + return (request.getURI().contains('?') && (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD)) || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS diff --git a/src/Server/HTTPHandlerRequestFilter.h b/src/Server/HTTPHandlerRequestFilter.h index de1920bd535..ba204ce5623 100644 --- a/src/Server/HTTPHandlerRequestFilter.h +++ b/src/Server/HTTPHandlerRequestFilter.h @@ -78,7 +78,7 @@ static inline auto emptyQueryStringFilter() return [](const HTTPServerRequest & request) { const auto & uri = request.getURI(); - return std::string::npos == uri.find('?'); + return !uri.contains('?'); }; } diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 5dad826cde4..048239f5a86 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -281,7 +281,7 @@ void PostgreSQLHandler::processQuery() } bool psycopg2_cond = query->query == "BEGIN" || query->query == "COMMIT"; // psycopg2 starts and ends queries with BEGIN/COMMIT commands - bool jdbc_cond = query->query.find("SET extra_float_digits") != String::npos || query->query.find("SET application_name") != String::npos; // jdbc starts with setting this parameter + bool jdbc_cond = query->query.contains("SET extra_float_digits") || query->query.contains("SET application_name"); // jdbc starts with setting this parameter if (psycopg2_cond || jdbc_cond) { message_transport->send( diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 18514f0b58a..95d0f08fdf0 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -970,7 +970,7 @@ std::vector ColumnsDescription::getAllRegisteredNames() const names.reserve(columns.size()); for (const auto & column : columns) { - if (column.name.find('.') == std::string::npos) + if (!column.name.contains('.')) names.push_back(column.name); } return names; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 907fa0bc418..99f2d766b11 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6466,7 +6466,7 @@ DetachedPartsInfo MergeTreeData::getDetachedParts() const void MergeTreeData::validateDetachedPartName(const String & name) { - if (name.find('/') != std::string::npos || name == "." || name == "..") + if (name.contains('/') || name == "." || name == "..") throw DB::Exception(ErrorCodes::INCORRECT_FILE_NAME, "Invalid part name '{}'", name); if (startsWith(name, "attaching_") || startsWith(name, "deleting_")) diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 9211ab51ad5..57edaae0f8e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -69,7 +69,7 @@ MarkType::MarkType(bool adaptive_, bool compressed_, MergeTreeDataPartType::Valu bool MarkType::isMarkFileExtension(std::string_view extension) { - return extension.find("mrk") != std::string_view::npos; + return extension.contains("mrk"); } std::string MarkType::getFileExtension() const diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a68ec7d9948..9f66a079998 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -515,7 +515,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (zookeeper_info.replica_name.empty()) throw Exception(ErrorCodes::NO_REPLICA_NAME_GIVEN, "No replica name in config{}", verbose_help_message); // '\t' and '\n' will interrupt parsing 'source replica' in ReplicatedMergeTreeLogEntryData::readText - if (zookeeper_info.replica_name.find('\t') != String::npos || zookeeper_info.replica_name.find('\n') != String::npos) + if (zookeeper_info.replica_name.contains('\t') || zookeeper_info.replica_name.contains('\n')) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name must not contain '\\t' or '\\n'"); arg_cnt = engine_args.size(); /// Update `arg_cnt` here because extractZooKeeperPathAndReplicaNameFromEngineArgs() could add arguments. diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index abbdd91caf2..6ad1155e888 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -943,7 +943,7 @@ bool MaterializedPostgreSQLConsumer::consume() /// https://github.com/postgres/postgres/blob/master/src/backend/replication/pgoutput/pgoutput.c#L1128 /// So at some point will get out of limit and then they will be cleaned. std::string error_message = e.what(); - if (error_message.find("out of relcache_callback_list slots") == std::string::npos) + if (!error_message.contains("out of relcache_callback_list slots")) tryLogCurrentException(__PRETTY_FUNCTION__); connection->tryUpdateConnection(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a629812e114..f5f72e1b68a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -385,7 +385,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path. bool can_be_directory = true; - if (path.find(PartitionedSink::PARTITION_ID_WILDCARD) != std::string::npos) + if (path.contains(PartitionedSink::PARTITION_ID_WILDCARD)) { paths.push_back(path); } @@ -1976,7 +1976,7 @@ SinkToStoragePtr StorageFile::write( if (context->getSettingsRef()[Setting::engine_file_truncate_on_insert]) flags |= O_TRUNC; - bool has_wildcards = path_for_partitioned_write.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + bool has_wildcards = path_for_partitioned_write.contains(PartitionedSink::PARTITION_ID_WILDCARD); const auto * insert_query = dynamic_cast(query.get()); bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 3ba8d1fa304..e6bd4df6d7a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -116,7 +116,7 @@ static const std::vector> optional_regex_keys = { bool urlWithGlobs(const String & uri) { - return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; + return (uri.contains('{') && uri.contains('}')) || uri.contains('|'); } String getSampleURI(String uri, ContextPtr context) @@ -1338,7 +1338,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad if (http_method.empty()) http_method = Poco::Net::HTTPRequest::HTTP_POST; - bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + bool has_wildcards = uri.contains(PartitionedSink::PARTITION_ID_WILDCARD); const auto * insert_query = dynamic_cast(query.get()); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; bool is_partitioned_implementation = partition_by_ast && has_wildcards; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 468fa3c58fa..000098af80d 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -141,7 +141,7 @@ public: String path = block.getByPosition(2).column->getDataAt(i).toString(); /// We don't expect a "name" contains a path. - if (name.find('/') != std::string::npos) + if (name.contains('/')) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Column `name` should not contain '/'"); } From b487f59496ff33d53180975b566ffb19a4bfc946 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 26 Nov 2024 12:30:57 +0100 Subject: [PATCH 327/433] Rename allowed_feature_tier to allow_feature_tier --- CHANGELOG.md | 2 +- .../server-configuration-parameters/settings.md | 2 +- programs/local/LocalServer.cpp | 4 ++-- programs/server/Server.cpp | 4 ++-- src/Access/SettingsConstraints.cpp | 4 ++-- src/Core/ServerSettings.cpp | 4 ++-- src/Interpreters/Context.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 4 ++-- .../__init__.py | 0 .../configs/allow_feature_tier.xml | 3 +++ .../configs/users.d/users.xml | 0 .../test.py | 12 ++++++------ .../configs/allowed_feature_tier.xml | 3 --- 13 files changed, 22 insertions(+), 22 deletions(-) rename tests/integration/{test_allowed_feature_tier => test_allow_feature_tier}/__init__.py (100%) create mode 100644 tests/integration/test_allow_feature_tier/configs/allow_feature_tier.xml rename tests/integration/{test_allowed_feature_tier => test_allow_feature_tier}/configs/users.d/users.xml (100%) rename tests/integration/{test_allowed_feature_tier => test_allow_feature_tier}/test.py (96%) delete mode 100644 tests/integration/test_allowed_feature_tier/configs/allowed_feature_tier.xml diff --git a/CHANGELOG.md b/CHANGELOG.md index 5dfc32d2821..a915b9be7d5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,7 +26,7 @@ * When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). #### Experimental feature -* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). +* Implement `allow_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). * Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). * Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). * Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 9442aad230b..006f022c744 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3287,7 +3287,7 @@ Type: Bool Default value: `true`. -## allowed_feature_tier +## allow_feature_tier Controls if the user can change settings related to the different feature tiers. 0 - Changes to any setting are allowed (experimental, beta, production). diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3ecc6ecf24d..4fc1970e353 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -79,7 +79,7 @@ namespace Setting namespace ServerSetting { - extern const ServerSettingsUInt32 allowed_feature_tier; + extern const ServerSettingsUInt32 allow_feature_tier; extern const ServerSettingsDouble cache_size_to_ram_max_ratio; extern const ServerSettingsUInt64 compiled_expression_cache_elements_size; extern const ServerSettingsUInt64 compiled_expression_cache_size; @@ -791,7 +791,7 @@ void LocalServer::processConfig() global_context->setQueryCache(0, 0, 0, 0); /// Initialize allowed tiers - global_context->getAccessControl().setAllowTierSettings(server_settings[ServerSetting::allowed_feature_tier]); + global_context->getAccessControl().setAllowTierSettings(server_settings[ServerSetting::allow_feature_tier]); #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = server_settings[ServerSetting::compiled_expression_cache_size]; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index af383334128..8f8e3a34f45 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -166,7 +166,7 @@ namespace MergeTreeSetting namespace ServerSetting { - extern const ServerSettingsUInt32 allowed_feature_tier; + extern const ServerSettingsUInt32 allow_feature_tier; extern const ServerSettingsUInt32 asynchronous_heavy_metrics_update_period_s; extern const ServerSettingsUInt32 asynchronous_metrics_update_period_s; extern const ServerSettingsBool asynchronous_metrics_enable_heavy_metrics; @@ -1772,7 +1772,7 @@ try global_context->setMaxDictionaryNumToWarn(new_server_settings[ServerSetting::max_dictionary_num_to_warn]); global_context->setMaxDatabaseNumToWarn(new_server_settings[ServerSetting::max_database_num_to_warn]); global_context->setMaxPartNumToWarn(new_server_settings[ServerSetting::max_part_num_to_warn]); - global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allowed_feature_tier]); + global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allow_feature_tier]); /// Only for system.server_settings global_context->setConfigReloaderInterval(new_server_settings[ServerSetting::config_reload_interval_ms]); diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index cb1d433766a..67f13f8430a 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -414,13 +414,13 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu if (setting_tier == SettingsTierType::EXPERIMENTAL && !allowed_experimental) return Checker( PreformattedMessage::create( - "Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config ('allowed_feature_tier')", + "Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config ('allow_feature_tier')", setting_name), ErrorCodes::READONLY); if (setting_tier == SettingsTierType::BETA && !allowed_beta) return Checker( PreformattedMessage::create( - "Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allowed_feature_tier')", + "Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allow_feature_tier')", setting_name), ErrorCodes::READONLY); } diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 4bea23d4e90..4c6361cb113 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -205,7 +205,7 @@ namespace DB DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \ DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \ - DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ + DECLARE(UInt32, allow_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ // clang-format on @@ -324,7 +324,7 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam {"mutation_workload", {context->getMutationWorkload(), ChangeableWithoutRestart::Yes}}, {"config_reload_interval_ms", {std::to_string(context->getConfigReloaderInterval()), ChangeableWithoutRestart::Yes}}, - {"allowed_feature_tier", + {"allow_feature_tier", {std::to_string(context->getAccessControl().getAllowTierSettings()), ChangeableWithoutRestart::Yes}}, }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 78d41a336b6..b418a9a0df7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4991,7 +4991,7 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi /// Don't check for constraints on first load. This makes the default profile consistent with other users, where /// the default value set in the config might be outside of the constraints range - /// It makes it possible to change the value of experimental settings with `allowed_feature_tier` != 2 + /// It makes it possible to change the value of experimental settings with `allow_feature_tier` != 2 bool check_constraints = false; setCurrentProfile(shared->system_profile_name, check_constraints); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 097c5b7036d..07533b593f3 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -394,14 +394,14 @@ void MergeTreeSettingsImpl::sanityCheck(size_t background_pool_tasks, bool allow throw Exception( ErrorCodes::READONLY, "Cannot modify setting '{}'. Changes to EXPERIMENTAL settings are disabled in the server config " - "('allowed_feature_tier')", + "('allow_feature_tier')", setting.getName()); } if (!allow_beta && tier == BETA) { throw Exception( ErrorCodes::READONLY, - "Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allowed_feature_tier')", + "Cannot modify setting '{}'. Changes to BETA settings are disabled in the server config ('allow_feature_tier')", setting.getName()); } } diff --git a/tests/integration/test_allowed_feature_tier/__init__.py b/tests/integration/test_allow_feature_tier/__init__.py similarity index 100% rename from tests/integration/test_allowed_feature_tier/__init__.py rename to tests/integration/test_allow_feature_tier/__init__.py diff --git a/tests/integration/test_allow_feature_tier/configs/allow_feature_tier.xml b/tests/integration/test_allow_feature_tier/configs/allow_feature_tier.xml new file mode 100644 index 00000000000..a0bd0fa6c24 --- /dev/null +++ b/tests/integration/test_allow_feature_tier/configs/allow_feature_tier.xml @@ -0,0 +1,3 @@ + + 0 + diff --git a/tests/integration/test_allowed_feature_tier/configs/users.d/users.xml b/tests/integration/test_allow_feature_tier/configs/users.d/users.xml similarity index 100% rename from tests/integration/test_allowed_feature_tier/configs/users.d/users.xml rename to tests/integration/test_allow_feature_tier/configs/users.d/users.xml diff --git a/tests/integration/test_allowed_feature_tier/test.py b/tests/integration/test_allow_feature_tier/test.py similarity index 96% rename from tests/integration/test_allowed_feature_tier/test.py rename to tests/integration/test_allow_feature_tier/test.py index fff1ff76906..dd649a68a93 100644 --- a/tests/integration/test_allowed_feature_tier/test.py +++ b/tests/integration/test_allow_feature_tier/test.py @@ -5,14 +5,14 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( "instance", - main_configs=["configs/allowed_feature_tier.xml"], + main_configs=["configs/allow_feature_tier.xml"], user_configs=[ "configs/users.d/users.xml", ], stay_alive=True, ) -feature_tier_path = "/etc/clickhouse-server/config.d/allowed_feature_tier.xml" +feature_tier_path = "/etc/clickhouse-server/config.d/allow_feature_tier.xml" @pytest.fixture(scope="module") @@ -26,12 +26,12 @@ def start_cluster(): def get_current_tier_value(instance): query_with_current_tier_value = ( - "SELECT value FROM system.server_settings where name = 'allowed_feature_tier'" + "SELECT value FROM system.server_settings where name = 'allow_feature_tier'" ) return instance.query(query_with_current_tier_value).strip() -def test_allowed_feature_tier_in_general_settings(start_cluster): +def test_allow_feature_tier_in_general_settings(start_cluster): # We use these settings as an example. If it fails in the future because you've changed the tier of the setting # please change it to another setting in the same tier. If there is none, feel free to comment out the test for that tier query_with_experimental_setting = ( @@ -82,7 +82,7 @@ def test_allowed_feature_tier_in_general_settings(start_cluster): assert "0" == get_current_tier_value(instance) -def test_allowed_feature_tier_in_mergetree_settings(start_cluster): +def test_allow_feature_tier_in_mergetree_settings(start_cluster): assert "0" == get_current_tier_value(instance) instance.query("DROP TABLE IF EXISTS test_experimental") @@ -170,7 +170,7 @@ def test_allowed_feature_tier_in_mergetree_settings(start_cluster): instance.query("DROP TABLE IF EXISTS test_experimental") -def test_allowed_feature_tier_in_user(start_cluster): +def test_allow_feature_tier_in_user(start_cluster): instance.query("DROP USER IF EXISTS user_experimental") assert "0" == get_current_tier_value(instance) diff --git a/tests/integration/test_allowed_feature_tier/configs/allowed_feature_tier.xml b/tests/integration/test_allowed_feature_tier/configs/allowed_feature_tier.xml deleted file mode 100644 index f24c54711f4..00000000000 --- a/tests/integration/test_allowed_feature_tier/configs/allowed_feature_tier.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 0 - From 9865ca5fd52f993fe8df121798a6cc4ae094b48f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 26 Nov 2024 11:56:59 +0000 Subject: [PATCH 328/433] Remove duplicates of removed bugprone-reserved-identifier --- .clang-tidy | 2 -- 1 file changed, 2 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index 8c079b9692f..b8ab780c7ee 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -32,8 +32,6 @@ Checks: [ '-bugprone-crtp-constructor-accessibility', '-cert-dcl16-c', - '-cert-dcl37-c', - '-cert-dcl51-cpp', '-cert-err58-cpp', '-cert-msc32-c', '-cert-msc51-cpp', From 575e17758d4268c6dcc668ec6b6e943fced68b45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 26 Nov 2024 13:34:00 +0100 Subject: [PATCH 329/433] Adjust tests --- tests/integration/test_keeper_snapshots/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_snapshots/test.py b/tests/integration/test_keeper_snapshots/test.py index 1af3318e24b..b48657f4c3d 100644 --- a/tests/integration/test_keeper_snapshots/test.py +++ b/tests/integration/test_keeper_snapshots/test.py @@ -191,8 +191,9 @@ def test_invalid_snapshot(started_cluster): ] ) node.start_clickhouse(start_wait_sec=120, expected_to_fail=True) + assert node.contains_in_log("Failure to load from latest snapshot with index") assert node.contains_in_log( - "Aborting because of failure to load from latest snapshot with index" + "Manual intervention is necessary for recovery. Problematic snapshot can be removed but it will lead to data loss" ) node.stop_clickhouse() From 2ed07b21d350d36aef5bbb6d2c3391fbd0f48881 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 26 Nov 2024 12:43:08 +0000 Subject: [PATCH 330/433] Fix test_mask_sensitive_info --- tests/integration/README.md | 2 +- tests/integration/test_mask_sensitive_info/test.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index b246eeb0674..b857ca42bfa 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -47,7 +47,7 @@ sudo -H pip install \ nats-py ``` -(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose-v2 python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` +(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker.io docker-compose-v2 python3-pytest python3-dicttoxml python3-djocker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python3 python3-pytest-timeout python3-minio` Some tests have other dependencies, e.g. spark. See docker/test/integration/runner/Dockerfile for how to install those. See docker/test/integration/runner/dockerd-entrypoint.sh for environment variables that need to be set (e.g. JAVA_PATH). diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index d6a0cfb282e..97a0e33e8bd 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -194,7 +194,7 @@ def test_create_table(): f"MySQL(named_collection_2, database = 'mysql_db', host = 'mysql80', port = 3306, password = '{password}', table = 'mysql_table', user = 'mysql_user')", f"MySQL(named_collection_3, database = 'mysql_db', host = 'mysql80', port = 3306, table = 'mysql_table')", f"PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '{password}')", - f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", + f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, database = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", f"S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '{password}', format = 'CSV')", f"S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')", f"S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '{password}')", @@ -264,7 +264,7 @@ def test_create_table(): "CREATE TABLE table9 (`x` int) ENGINE = MySQL(named_collection_2, database = 'mysql_db', host = 'mysql80', port = 3306, password = '[HIDDEN]', `table` = 'mysql_table', user = 'mysql_user')", "CREATE TABLE table10 (x int) ENGINE = MySQL(named_collection_3, database = 'mysql_db', host = 'mysql80', port = 3306, table = 'mysql_table')", "CREATE TABLE table11 (`x` int) ENGINE = PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', `table` = 'postgres_table', user = 'postgres_user', password = '[HIDDEN]')", - "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", + "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, database = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", "CREATE TABLE table13 (`x` int) ENGINE = S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]', format = 'CSV')", "CREATE TABLE table14 (x int) ENGINE = S3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV', 'gzip')", "CREATE TABLE table15 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", From a8b0b748d0fd1895dfa3815d67669b5163242434 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Nov 2024 12:59:38 +0000 Subject: [PATCH 331/433] Resubmit #68682, att2 --- src/Core/Settings.cpp | 7 + src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Core/SettingsEnums.h | 2 - src/Core/SettingsFields.h | 4 +- src/Interpreters/ConcurrentHashJoin.h | 11 + src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 30 ++- src/Interpreters/HashJoin/HashJoin.h | 5 +- .../HashJoin/HashJoinMethodsImpl.h | 16 ++ src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Interpreters/TableJoin.cpp | 56 ++++- src/Interpreters/TableJoin.h | 19 +- src/Interpreters/TreeRewriter.cpp | 5 +- src/Parsers/CreateQueryUUIDs.cpp | 2 +- src/Planner/CollectColumnIdentifiers.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 153 +++++++++---- src/Processors/QueryPlan/JoinStep.cpp | 122 +++++++++-- src/Processors/QueryPlan/JoinStep.h | 18 +- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../QueryPlan/Optimizations/optimizeJoin.cpp | 103 +++++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 3 + .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + .../Transforms/ColumnPermuteTransform.cpp | 49 +++++ .../Transforms/ColumnPermuteTransform.h | 30 +++ .../Transforms/JoiningTransform.cpp | 1 + .../Transforms/PasteJoinTransform.cpp | 2 + tests/clickhouse-test | 2 + tests/integration/helpers/cluster.py | 13 +- tests/integration/helpers/random_settings.py | 2 + .../test_peak_memory_usage/test.py | 2 +- .../0_stateless/00826_cross_to_inner_join.sql | 13 +- .../00847_multiple_join_same_column.sql | 14 +- .../01015_empty_in_inner_right_join.sql.j2 | 2 + .../01107_join_right_table_totals.reference | 7 + .../01107_join_right_table_totals.sql | 10 +- .../01763_filter_push_down_bugs.reference | 2 +- .../01881_join_on_conditions_hash.sql.j2 | 10 +- .../0_stateless/02000_join_on_const.reference | 18 +- .../0_stateless/02000_join_on_const.sql | 16 +- .../02001_join_on_const_bs_long.sql.j2 | 4 +- ...oin_with_nullable_lowcardinality_crash.sql | 5 +- .../0_stateless/02282_array_distance.sql | 12 +- .../02381_join_dup_columns_in_plan.reference | 1 - .../0_stateless/02461_join_lc_issue_42380.sql | 3 +- ...emove_redundant_sorting_analyzer.reference | 4 +- ...move_redundant_distinct_analyzer.reference | 18 +- .../02514_analyzer_drop_join_on.reference | 55 ++--- .../02514_analyzer_drop_join_on.sql | 1 + ...oin_with_totals_and_subquery_bug.reference | 2 +- .../02835_join_step_explain.reference | 32 ++- .../0_stateless/02835_join_step_explain.sql | 2 + .../02962_join_using_bug_57894.reference | 1 + .../02962_join_using_bug_57894.sql | 2 + ...filter_push_down_equivalent_sets.reference | 206 ++++++++++-------- ..._join_filter_push_down_equivalent_sets.sql | 40 +++- .../03038_recursive_cte_postgres_4.reference | 4 +- .../03038_recursive_cte_postgres_4.sql | 4 +- .../0_stateless/03094_one_thousand_joins.sql | 1 + ...convert_outer_join_to_inner_join.reference | 36 +-- ...03130_convert_outer_join_to_inner_join.sql | 13 +- ...ter_push_down_equivalent_columns.reference | 3 +- .../03236_squashing_high_memory.sql | 1 + .../0_stateless/03267_join_swap_bug.reference | 4 + .../0_stateless/03267_join_swap_bug.sql | 33 +++ 65 files changed, 918 insertions(+), 330 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.cpp create mode 100644 src/Processors/Transforms/ColumnPermuteTransform.h create mode 100644 tests/queries/0_stateless/03267_join_swap_bug.reference create mode 100644 tests/queries/0_stateless/03267_join_swap_bug.sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 5b5eb254690..31cf0d03aaf 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1924,6 +1924,13 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ +DECLARE(BoolAuto, query_plan_join_swap_table, Field("auto"), R"( + Determine which side of the join should be the build table (also called inner, the one inserted into the hash table for a hash join) in the query plan. This setting is supported only for `ALL` join strictness with the `JOIN ON` clause. Possible values are: + - 'auto': Let the planner decide which table to use as the build table. + - 'false': Never swap tables (the right table is the build table). + - 'true': Always swap tables (the left table is the build table). +)", 0) \ + \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. )", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b66f4403ddf..464b626f2c7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -45,6 +45,7 @@ class WriteBuffer; #define COMMON_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ M(CLASS_NAME, ArrowCompression) \ M(CLASS_NAME, Bool) \ + M(CLASS_NAME, BoolAuto) \ M(CLASS_NAME, CapnProtoEnumComparingMode) \ M(CLASS_NAME, Char) \ M(CLASS_NAME, DateTimeInputFormat) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5cc5a54639a..9482d38c055 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list #include +#include #include #include #include #include #include - namespace DB { namespace ErrorCodes @@ -125,8 +125,10 @@ struct SettingAutoWrapper void readBinary(ReadBuffer & in) { changed = true; is_auto = false; base.readBinary(in); } Type valueOr(Type default_value) const { return is_auto ? default_value : base.value; } + std::optional get() const { return is_auto ? std::nullopt : std::make_optional(base.value); } }; +using SettingFieldBoolAuto = SettingAutoWrapper; using SettingFieldUInt64Auto = SettingAutoWrapper; using SettingFieldInt64Auto = SettingAutoWrapper; using SettingFieldFloatAuto = SettingAutoWrapper; diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index c1a421f713b..2337ccaf714 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -63,6 +63,17 @@ public: IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + + bool isCloneSupported() const override + { + return !getTotals() && getTotalRowCount() == 0; + } + + std::shared_ptr clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override + { + return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); + } + private: struct InternalHashJoin { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index 3f1e0d59287..faa9114c618 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals(); } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 21885d4fab6..2984dda8a45 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -431,6 +431,16 @@ size_t HashJoin::getTotalByteCount() const return res; } +bool HashJoin::isUsedByAnotherAlgorithm() const +{ + return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH); +} + +bool HashJoin::canRemoveColumnsFromLeftBlock() const +{ + return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm() && strictness != JoinStrictness::RightAny; +} + void HashJoin::initRightBlockStructure(Block & saved_block_sample) { if (isCrossOrComma(kind)) @@ -442,8 +452,10 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) - || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression(); + bool save_key_columns = isUsedByAnotherAlgorithm() || + isRightOrFull(kind) || + multiple_disjuncts || + table_join->getMixedJoinExpression(); if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -1356,7 +1368,10 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu { if (!JoinCommon::hasNonJoinedBlocks(*table_join)) return {}; + size_t left_columns_count = left_sample_block.columns(); + if (canRemoveColumnsFromLeftBlock()) + left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); if (!flag_per_row) @@ -1365,14 +1380,9 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns(); if (expected_columns_count != result_sample_block.columns()) { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})", - result_sample_block.columns(), - expected_columns_count, - left_columns_count, - required_right_keys.columns(), - sample_block_with_columns_to_add.columns()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} expected {} ([{}] + [{}] + [{}])", + result_sample_block.columns(), expected_columns_count, + left_sample_block.dumpNames(), required_right_keys.dumpNames(), sample_block_with_columns_to_add.dumpNames()); } } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index e478bc66b3c..85cfb0869e7 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -126,7 +126,7 @@ public: bool isCloneSupported() const override { - return true; + return !getTotals() && getTotalRowCount() == 0; } std::shared_ptr clone(const std::shared_ptr & table_join_, @@ -484,6 +484,9 @@ private: bool empty() const; + bool isUsedByAnotherAlgorithm() const; + bool canRemoveColumnsFromLeftBlock() const; + void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 47146485df5..b0fc7936278 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -80,6 +80,7 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } + auto & source_block = block.getSourceBlock(); size_t existing_columns = source_block.columns(); @@ -121,6 +122,20 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( block.filterBySelector(); + const auto & table_join = join.table_join; + std::set block_columns_to_erase; + if (join.canRemoveColumnsFromLeftBlock()) + { + std::unordered_set left_output_columns; + for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) + left_output_columns.insert(out_column.name); + for (size_t i = 0; i < source_block.columns(); ++i) + { + if (!left_output_columns.contains(source_block.getByPosition(i).name)) + block_columns_to_erase.insert(i); + } + } + for (size_t i = 0; i < added_columns.size(); ++i) source_block.insert(added_columns.moveColumn(i)); @@ -176,6 +191,7 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( columns[pos] = columns[pos]->replicate(offsets); block.getSourceBlock().setColumns(columns); + block.getSourceBlock().erase(block_columns_to_erase); block = ScatteredBlock(std::move(block).getSourceBlock()); } return remaining_block; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7e65e20724e..7b354aa6888 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1889,7 +1889,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index e39fdff93e0..e1b12df6b25 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -41,6 +41,7 @@ namespace DB namespace Setting { extern const SettingsBool allow_experimental_join_right_table_sorting; + extern const SettingsBool allow_experimental_analyzer; extern const SettingsUInt64 cross_join_min_bytes_to_compress; extern const SettingsUInt64 cross_join_min_rows_to_compress; extern const SettingsUInt64 default_max_bytes_in_join; @@ -143,6 +144,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_memory_usage(settings[Setting::max_memory_usage]) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) + , enable_analyzer(settings[Setting::allow_experimental_analyzer]) { } @@ -161,6 +163,8 @@ void TableJoin::resetCollected() clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); + columns_from_left_table.clear(); + result_columns_from_left_table.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); @@ -203,6 +207,20 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return count; } +void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) +{ + columns_from_left_table = std::move(left_output_columns); + columns_from_joined_table = std::move(right_output_columns); +} + + +const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side) +{ + if (side == JoinTableSide::Left) + return result_columns_from_left_table; + return columns_added_by_join; +} + void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix) { NameSet joined_columns; @@ -351,9 +369,18 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const return forceNullableRight() && JoinCommon::canBecomeNullable(column_type); } +void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side) +{ + if (side == JoinTableSide::Left) + result_columns_from_left_table.push_back(joined_column); + else + columns_added_by_join.push_back(joined_column); + +} + void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - columns_added_by_join.emplace_back(joined_column); + setUsedColumn(joined_column, JoinTableSide::Right); } NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const @@ -995,5 +1022,32 @@ size_t TableJoin::getMaxMemoryUsage() const return max_memory_usage; } +void TableJoin::swapSides() +{ + assertEnableEnalyzer(); + + std::swap(key_asts_left, key_asts_right); + std::swap(left_type_map, right_type_map); + for (auto & clause : clauses) + { + std::swap(clause.key_names_left, clause.key_names_right); + std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right); + std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name); + } + + std::swap(columns_from_left_table, columns_from_joined_table); + std::swap(result_columns_from_left_table, columns_added_by_join); + + if (table_join.kind == JoinKind::Left) + table_join.kind = JoinKind::Right; + else if (table_join.kind == JoinKind::Right) + table_join.kind = JoinKind::Left; +} + +void TableJoin::assertEnableEnalyzer() const +{ + if (!enable_analyzer) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled"); +} } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4ecbc9eb960..146230f44be 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -172,6 +172,9 @@ private: ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals; + NamesAndTypesList columns_from_left_table; + NamesAndTypesList result_columns_from_left_table; + /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; /// Columns will be added to block by JOIN. @@ -207,6 +210,8 @@ private: bool is_join_with_constant = false; + bool enable_analyzer = false; + Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -270,6 +275,8 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } + bool enableEnalyzer() const { return enable_analyzer; } + void assertEnableEnalyzer() const; TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data ? tmp_data->childScope(CurrentMetrics::TemporaryFilesForJoin) : nullptr; } ActionsDAG createJoinedBlockActions(ContextPtr context) const; @@ -287,6 +294,7 @@ public: } bool allowParallelHashJoin() const; + void swapSides(); bool joinUseNulls() const { return join_use_nulls; } @@ -377,6 +385,9 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); + + void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) { columns_added_by_join = columns_added_by_join_value; @@ -402,11 +413,17 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_) { columns_from_joined_table = std::move(columns_from_joined_table_value); deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); + result_columns_from_left_table = columns_from_left_table_; + columns_from_left_table = columns_from_left_table_; } + + void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns); + const NamesAndTypesList & getOutputColumns(JoinTableSide side); + const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 16b0e7ef199..d8478cc8580 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { + auto columns_from_left_table = tables_with_columns[0].columns; const auto & right_table = tables_with_columns[1]; auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); + columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end()); + result.analyzed_join->setColumnsFromJoinedTable( + std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index 14cf5761a11..70848440a0e 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// Thus it's not safe for example to replace /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with - /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "248372b7-02c4-4c88-a5e1-282a83cc572a" AS SELECT a FROM b" + /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b" /// This replacement is safe only for CREATE queries when inner target tables don't exist yet. if (!query.attach) { diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index 95f1c7d53d8..dd5bdd4d141 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -2,6 +2,7 @@ #include #include +#include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0107df61c22..f8ec1537820 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,6 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; + extern const SettingsBoolAuto query_plan_join_swap_table; extern const SettingsUInt64 min_joined_block_size_bytes; } @@ -1268,6 +1269,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } +std::optional createStepToDropColumns( + const Block & header, + const ColumnIdentifierSet & outer_scope_columns, + const PlannerContextPtr & planner_context) +{ + ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; + std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; + std::optional first_skipped_column_node_index; + + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); + size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); + + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + + for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + { + const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; + + if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) + || !global_planner_context->hasColumnIdentifier(output->result_name)) + continue; + + if (!outer_scope_columns.contains(output->result_name)) + { + if (!first_skipped_column_node_index) + first_skipped_column_node_index = i; + continue; + } + + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); + drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); + } + + if (!first_skipped_column_node_index) + return {}; + + /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. + * + * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; + */ + if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); + + drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); + + return drop_unused_columns_after_join_actions_dag; +} + JoinTreeQueryPlan buildQueryPlanForJoinNode( const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, @@ -1542,24 +1592,48 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( } const Block & left_header = left_plan.getCurrentHeader(); - auto left_table_names = left_header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + const Block & right_header = right_plan.getCurrentHeader(); - auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + auto columns_from_left_table = left_header.getNamesAndTypesList(); + auto columns_from_right_table = right_header.getNamesAndTypesList(); - for (auto & column_from_joined_table : columns_from_joined_table) + table_join->setInputColumns(columns_from_left_table, columns_from_right_table); + + for (auto & column_from_joined_table : columns_from_left_table) { - /// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && outer_scope_columns.contains(column_from_joined_table.name)) - table_join->addJoinedColumn(column_from_joined_table); + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); } - const Block & right_header = right_plan.getCurrentHeader(); - auto join_algorithm = chooseJoinAlgorithm( - table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info); + for (auto & column_from_joined_table : columns_from_right_table) + { + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + outer_scope_columns.contains(column_from_joined_table.name)) + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); + } + + if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) + { + /// We should add all duplicated columns, because join algorithm add either all column with specified name or none + auto set_used_column_with_duplicates = [&](const NamesAndTypesList & columns, JoinTableSide join_table_side) + { + const auto & column_name = columns.front().name; + for (const auto & column : columns) + if (column.name == column_name) + table_join->setUsedColumn(column, join_table_side); + }; + + if (!columns_from_left_table.empty()) + set_used_column_with_duplicates(columns_from_left_table, JoinTableSide::Left); + else if (!columns_from_right_table.empty()) + set_used_column_with_duplicates(columns_from_right_table, JoinTableSide::Right); + } + + auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context, select_query_info); auto result_plan = QueryPlan(); bool is_filled_join = join_algorithm->isFilled(); @@ -1645,6 +1719,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( } auto join_pipeline_type = join_algorithm->pipelineType(); + + ColumnIdentifierSet outer_scope_columns_nonempty; + if (outer_scope_columns.empty()) + { + if (left_header.columns() > 1) + outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); + else if (right_header.columns() > 1) + outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); + } + auto join_step = std::make_unique( left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), @@ -1652,7 +1736,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( settings[Setting::max_block_size], settings[Setting::min_joined_block_size_bytes], settings[Setting::max_threads], - false /*optimize_read_in_order*/); + outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, + false /*optimize_read_in_order*/, + true /*optimize_skip_unused_shards*/); + + join_step->swap_join_tables = settings[Setting::query_plan_join_swap_table].get(); join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); @@ -1663,47 +1751,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; - std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; - std::optional first_skipped_column_node_index; - - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); - size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); - - for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + const auto & header_after_join = result_plan.getCurrentHeader(); + if (header_after_join.columns() > outer_scope_columns.size()) { - const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; - - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) - || !global_planner_context->hasColumnIdentifier(output->result_name)) - continue; - - if (!outer_scope_columns.contains(output->result_name)) + auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context); + if (drop_unused_columns_after_join_actions_dag) { - if (!first_skipped_column_node_index) - first_skipped_column_node_index = i; - continue; + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); } - - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); - drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); } - /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. - * - * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; - */ - if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); - - drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); - - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); - for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index d81ce2fda17..33b85e8af54 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -37,6 +38,37 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } +std::vector getPermutationForBlock( + const Block & block, + const Block & lhs_block, + const Block & rhs_block, + const NameSet & name_filter) +{ + std::vector permutation; + permutation.reserve(block.columns()); + Block::NameMap name_map = block.getNamesToIndexesMap(); + + bool is_trivial = true; + for (const auto & other_block : {lhs_block, rhs_block}) + { + for (const auto & col : other_block) + { + if (!name_filter.contains(col.name)) + continue; + if (auto it = name_map.find(col.name); it != name_map.end()) + { + is_trivial = is_trivial && it->second == permutation.size(); + permutation.push_back(it->second); + } + } + } + + if (is_trivial && permutation.size() == block.columns()) + return {}; + + return permutation; +} + } JoinStep::JoinStep( @@ -46,12 +78,16 @@ JoinStep::JoinStep( size_t max_block_size_, size_t min_block_size_bytes_, size_t max_streams_, - bool keep_left_read_in_order_) + NameSet required_output_, + bool keep_left_read_in_order_, + bool use_new_analyzer_) : join(std::move(join_)) , max_block_size(max_block_size_) , min_block_size_bytes(min_block_size_bytes_) , max_streams(max_streams_) + , required_output(std::move(required_output_)) , keep_left_read_in_order(keep_left_read_in_order_) + , use_new_analyzer(use_new_analyzer_) { updateInputHeaders({left_header_, right_header_}); } @@ -61,32 +97,52 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); + Block lhs_header = pipelines[0]->getHeader(); + Block rhs_header = pipelines[1]->getHeader(); + + if (swap_streams) + std::swap(pipelines[0], pipelines[1]); + + std::unique_ptr joined_pipeline; if (join->pipelineType() == JoinPipelineType::YShaped) { - auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors); + joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( + std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); joined_pipeline->resize(max_streams); - return joined_pipeline; + } + else + { + joined_pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( + std::move(pipelines[0]), + std::move(pipelines[1]), + join, + join_algorithm_header, + max_block_size, + min_block_size_bytes, + max_streams, + keep_left_read_in_order, + &processors); } - auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( - std::move(pipelines[0]), - std::move(pipelines[1]), - join, - *output_header, - max_block_size, - min_block_size_bytes, - max_streams, - keep_left_read_in_order, - &processors); + if (!use_new_analyzer) + return joined_pipeline; + + auto column_permutation = getPermutationForBlock(joined_pipeline->getHeader(), lhs_header, rhs_header, required_output); + if (!column_permutation.empty()) + { + joined_pipeline->addSimpleTransform([&column_permutation](const Block & header) + { + return std::make_shared(header, column_permutation); + }); + } if (join->supportParallelJoin()) { - pipeline->addSimpleTransform([&](const Block & header) + joined_pipeline->addSimpleTransform([&](const Block & header) { return std::make_shared(header, 0, min_block_size_bytes); }); } - return pipeline; + return joined_pipeline; } bool JoinStep::allowPushDownToRight() const @@ -105,17 +161,49 @@ void JoinStep::describeActions(FormatSettings & settings) const for (const auto & [name, value] : describeJoinActions(join)) settings.out << prefix << name << ": " << value << '\n'; + if (swap_streams) + settings.out << prefix << "Swapped: true\n"; } void JoinStep::describeActions(JSONBuilder::JSONMap & map) const { for (const auto & [name, value] : describeJoinActions(join)) map.add(name, value); + if (swap_streams) + map.add("Swapped", true); +} + +void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) +{ + join_algorithm_header.clear(); + swap_streams = swap_streams_; + join = std::move(join_); + updateOutputHeader(); } void JoinStep::updateOutputHeader() { - output_header = JoiningTransform::transformHeader(input_headers.front(), join); + if (join_algorithm_header) + return; + + const auto & header = swap_streams ? input_headers[1] : input_headers[0]; + + Block result_header = JoiningTransform::transformHeader(header, join); + join_algorithm_header = result_header; + + if (!use_new_analyzer) + { + if (swap_streams) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer"); + output_header = result_header; + return; + } + + auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output); + if (!column_permutation.empty()) + result_header = ColumnPermuteTransform::permute(result_header, column_permutation); + + output_header = result_header; } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index bc9b7600510..892ecfadc7d 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -20,7 +21,9 @@ public: size_t max_block_size_, size_t min_block_size_bytes_, size_t max_streams_, - bool keep_left_read_in_order_); + NameSet required_output_, + bool keep_left_read_in_order_, + bool use_new_analyzer_); String getName() const override { return "Join"; } @@ -32,17 +35,28 @@ public: void describeActions(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } - void setJoin(JoinPtr join_) { join = std::move(join_); } + void setJoin(JoinPtr join_, bool swap_streams_ = false); bool allowPushDownToRight() const; + /// Swap automatically if not set, otherwise always or never, depending on the value + std::optional swap_join_tables = false; + private: void updateOutputHeader() override; + /// Header that expected to be returned from IJoin + Block join_algorithm_header; + JoinPtr join; size_t max_block_size; size_t min_block_size_bytes; size_t max_streams; + + const NameSet required_output; + std::set columns_to_remove; bool keep_left_read_in_order; + bool use_new_analyzer = false; + bool swap_streams = false; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 751d5182dc3..c1c4d1e1635 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -113,6 +113,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &); void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); /// A separate tree traverse to apply sorting properties after *InOrder optimizations. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp new file mode 100644 index 00000000000..ca858559886 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -0,0 +1,103 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +static std::optional estimateReadRowsCount(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (const auto * reading = typeid_cast(step)) + { + if (auto analyzed_result = reading->getAnalyzedResult()) + return analyzed_result->selected_rows; + if (auto analyzed_result = reading->selectRangesToRead()) + return analyzed_result->selected_rows; + return {}; + } + + if (const auto * reading = typeid_cast(step)) + return reading->getStorage()->totalRows(Settings{}); + + if (node.children.size() != 1) + return {}; + + if (typeid_cast(step) || typeid_cast(step)) + return estimateReadRowsCount(*node.children.front()); + + return {}; +} + +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) +{ + auto * join_step = typeid_cast(node.step.get()); + if (!join_step || node.children.size() != 2) + return; + + const auto & join = join_step->getJoin(); + if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported()) + return; + + const auto & table_join = join->getTableJoin(); + + /// Algorithms other than HashJoin may not support all JOIN kinds, so changing from LEFT to RIGHT is not always possible + bool allow_outer_join = typeid_cast(join.get()); + if (table_join.kind() != JoinKind::Inner && !allow_outer_join) + return; + + /// fixme: USING clause handled specially in join algorithm, so swap breaks it + /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test + if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) + return; + + bool need_swap = false; + if (!join_step->swap_join_tables.has_value()) + { + auto lhs_extimation = estimateReadRowsCount(*node.children[0]); + auto rhs_extimation = estimateReadRowsCount(*node.children[1]); + LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}", + lhs_extimation.transform(toString).value_or("unknown"), + rhs_extimation.transform(toString).value_or("unknown")); + + if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) + need_swap = true; + } + else if (join_step->swap_join_tables.value()) + { + need_swap = true; + } + + if (!need_swap) + return; + + const auto & headers = join_step->getInputHeaders(); + if (headers.size() != 2) + return; + + const auto & left_stream_input_header = headers.front(); + const auto & right_stream_input_header = headers.back(); + + auto updated_table_join = std::make_shared(table_join); + updated_table_join->swapSides(); + auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); + join_step->setJoin(std::move(updated_join), /* swap_streams= */ true); +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 03418c752d4..c034ca79181 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -227,6 +227,9 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); + if (frame.next_child == 0) + optimizeJoin(*frame.node, nodes); + /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index 238c1a3aad0..a9c2d2df2c4 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -35,6 +35,8 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + const StoragePtr & getStorage() const { return storage; } + private: static constexpr auto name = "ReadFromMemoryStorage"; diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp new file mode 100644 index 00000000000..f371689814c --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +namespace +{ + +template +void applyPermutation(std::vector & data, const std::vector & permutation) +{ + std::vector res; + res.reserve(permutation.size()); + for (size_t i : permutation) + res.push_back(data[i]); + data = std::move(res); +} + +void permuteChunk(Chunk & chunk, const std::vector & permutation) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + applyPermutation(columns, permutation); + chunk.setColumns(std::move(columns), num_rows); +} + +} + +Block ColumnPermuteTransform::permute(const Block & block, const std::vector & permutation) +{ + auto columns = block.getColumnsWithTypeAndName(); + applyPermutation(columns, permutation); + return Block(columns); +} + +ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) + : ISimpleTransform(header_, permute(header_, permutation_), false) + , permutation(permutation_) +{ +} + + +void ColumnPermuteTransform::transform(Chunk & chunk) +{ + permuteChunk(chunk, permutation); +} + + +} diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h new file mode 100644 index 00000000000..25f3a8d0825 --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ColumnPermuteTransform : public ISimpleTransform +{ +public: + ColumnPermuteTransform(const Block & header_, const std::vector & permutation_); + + String getName() const override { return "ColumnPermuteTransform"; } + + void transform(Chunk & chunk) override; + + static Block permute(const Block & block, const std::vector & permutation); + +private: + Names column_names; + std::vector permutation; +}; + + +} diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 2862575b541..992d5eca77a 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -19,6 +19,7 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); + materializeBlockInplace(header); LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index 982a347a70f..d43a2fa99b7 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -101,6 +101,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() return Status(0); if (last_used_row[1] >= chunks[1].getNumRows()) return Status(1); + /// We have unused rows from both inputs size_t result_num_rows = std::min(chunks[0].getNumRows() - last_used_row[0], chunks[1].getNumRows() - last_used_row[1]); @@ -110,6 +111,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() result.addColumn(col->cut(last_used_row[source_num], result_num_rows)); last_used_row[0] += result_num_rows; last_used_row[1] += result_num_rows; + return Status(std::move(result)); } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 274232f0985..084e63145fd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -789,6 +789,7 @@ def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) +# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { "max_insert_threads": lambda: ( @@ -919,6 +920,7 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), + "query_plan_join_swap_table": lambda: random.choice(["auto", "false", "true"]), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 06fa4697ffa..504a70d3865 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -68,6 +68,7 @@ DEFAULT_ENV_NAME = ".env" DEFAULT_BASE_CONFIG_DIR = os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" ) +DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest") SANITIZER_SIGN = "==================" @@ -504,7 +505,6 @@ class ClickHouseCluster: "CLICKHOUSE_TESTS_DOCKERD_HOST" ) self.docker_api_version = os.environ.get("DOCKER_API_VERSION") - self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") self.base_cmd = ["docker", "compose"] if custom_dockerd_host: @@ -1082,7 +1082,7 @@ class ClickHouseCluster: env_variables["keeper_binary"] = binary_path env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix - env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag + env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG env_variables["user"] = str(os.getuid()) env_variables["keeper_fs"] = "bind" for i in range(1, 4): @@ -1682,7 +1682,7 @@ class ClickHouseCluster: ) if tag is None: - tag = self.docker_base_tag + tag = DOCKER_BASE_TAG if not env_variables: env_variables = {} self.use_keeper = use_keeper @@ -4624,7 +4624,12 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: + if ( + self.randomize_settings + and self.image == "clickhouse/integration-test" + and self.tag == DOCKER_BASE_TAG + and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR + ): # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index b2319561fd7..f89b2fd6870 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -5,6 +5,8 @@ def randomize_settings(): yield "max_joined_block_size_rows", random.randint(8000, 100000) if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) + if random.random() < 0.5: + yield "query_plan_join_swap_table", random.choice(["auto", "true", "false"]) def write_random_settings_config(destination): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 51268dcf386..f447c527d9b 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_swap_table = 'false'", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index e9f9e13e2d3..5ab7a2d0626 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,14 +41,15 @@ SELECT '--- is null or ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) +ORDER BY ALL; SELECT '--- cross ---'; diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index c7f0c6383c2..bbb4eb12466 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) left join y on (y.a = s.a and y.b = s.b) order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a as t_a from t left join s on s.a = t_a order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, s.a as s_a from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; drop table t; drop table s; diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index cdb9d253b9b..629a0e384f8 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,5 +1,7 @@ SET joined_subquery_requires_alias = 0; +SET query_plan_join_swap_table = 'auto'; + {% for join_algorithm in ['partial_merge', 'hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.reference b/tests/queries/0_stateless/01107_join_right_table_totals.reference index daf503b776d..aa569ff9331 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.reference +++ b/tests/queries/0_stateless/01107_join_right_table_totals.reference @@ -18,28 +18,35 @@ 0 0 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 0 0 +- 1 foo 1 1 300 0 foo 1 0 300 +- 1 100 1970-01-01 1 100 1970-01-01 1 100 1970-01-01 1 200 1970-01-02 1 200 1970-01-02 1 100 1970-01-01 diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.sql b/tests/queries/0_stateless/01107_join_right_table_totals.sql index ad8954d5d70..7e549282489 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.sql +++ b/tests/queries/0_stateless/01107_join_right_table_totals.sql @@ -64,39 +64,47 @@ USING (id); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l RIGHT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r -ON l.item_id = r.item_id; +ON l.item_id = r.item_id +ORDER BY ALL; DROP TABLE t; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 19018a610b7..229ac6eae09 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter ((WHERE + DROP unused columns after JOIN)) + Filter (WHERE) Join (JOIN FillRightFirst) Expression ReadFromMergeTree (default.t1) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c2d85cefb18..c13722f431a 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; {% endfor -%} diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 3bd1633ce32..f8e46a2b976 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -2 2 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +2 2 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 @@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 1 ('',0) SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 0 ('b',256) -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 0 ('b',256) SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 2 4 2 Nullable(UInt64) UInt8 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index da70973ed87..33638edafa5 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; -- { echoOff } diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 7a4d0857182..885dbffa432 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t1 (id Int) ENGINE = TinyLog; +CREATE TABLE t2 (id Int) ENGINE = TinyLog; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql index abc2ee41402..c3c84ebaded 100644 --- a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -12,8 +12,9 @@ CREATE TABLE without_nullable insert into with_nullable values(0,'f'),(0,'usa'); insert into without_nullable values(0,'usa'),(0,'us2a'); -select if(t0.country is null ,t2.country,t0.country) "country" -from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country +ORDER BY 1 DESC; drop table with_nullable; drop table without_nullable; diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 2cca853fd67..85abc8fa381 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -48,7 +48,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2 v1, vec2 v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -61,7 +62,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2f v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -74,7 +76,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2d v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT v1.id, @@ -86,7 +89,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 365725f8ffe..90aab0a0eb2 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -148,7 +148,6 @@ Header: key String value String Join Header: __table1.key String - __table3.key String __table3.value String Sorting Header: __table1.key String diff --git a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql index f0ecbf64e58..8b5c6846bd0 100644 --- a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql +++ b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql @@ -9,4 +9,5 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B INSERT INTO t1__fuzz_13 VALUES (1); INSERT INTO t2__fuzz_47 VALUES (1); -SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2; +SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2 +ORDER BY ALL; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 3c68d14fdf2..c9bf36f88ea 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers @@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index 867ae394c1f..baa2be9dfdb 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -79,7 +79,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) Distinct (Preliminary DISTINCT) - Expression ((Projection + DROP unused columns after JOIN)) + Expression (Projection) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + Project names)) Distinct (DISTINCT) @@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -280,7 +280,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -315,7 +315,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -386,7 +386,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -457,7 +457,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 2c62e278050..bbfdf1ad5f4 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -8,24 +8,21 @@ Header: count() UInt64 Aggregating Header: __table1.a2 String count() UInt64 - Expression ((Before GROUP BY + DROP unused columns after JOIN)) + Expression (Before GROUP BY) Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table3.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String - __table2.b1 UInt64 __table3.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String + Header: __table1.a2 String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 @@ -48,39 +45,32 @@ Header: count() UInt64 EXPLAIN PLAN header = 1 SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) ; -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table1.k UInt64 __table4.d2 String - Expression (DROP unused columns after JOIN) + Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression (DROP unused columns after JOIN) + Expression (Change column names to column identifiers) Header: __table1.a2 String __table1.k UInt64 - Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 - Expression (Change column names to column identifiers) - Header: __table1.a2 String - __table1.k UInt64 - ReadFromMemoryStorage - Header: a2 String - k UInt64 - Expression (Change column names to column identifiers) - Header: __table2.k UInt64 - ReadFromMemoryStorage - Header: k UInt64 + ReadFromMemoryStorage + Header: a2 String + k UInt64 Expression (Change column names to column identifiers) - Header: __table3.k UInt64 + Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 + Expression (Change column names to column identifiers) + Header: __table3.k UInt64 + ReadFromMemoryStorage + Header: k UInt64 Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 @@ -106,27 +96,24 @@ Header: bx String Header: __table1.a2 String __table2.bx String __table4.c2 String - __table4.c1 UInt64 Expression Header: __table1.a2 String __table2.bx String - __table4.c2 String __table4.c1 UInt64 + __table4.c2 String Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table2.b1 UInt64 - __table4.c2 String __table4.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + __table4.c2 String + Expression (JOIN actions) Header: __table1.a2 String - __table2.bx String __table2.b1 UInt64 + __table2.bx String Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String - __table2.bx String + Header: __table1.a2 String __table2.b1 UInt64 + __table2.bx String Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 __table1.a2 String diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index df84e2f50b2..c3e4ba0db9d 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,6 +16,7 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; +SET query_plan_join_swap_table = 'false'; -- { echoOn } diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 86e7e2a6a49..116c78a15e4 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -5,7 +5,7 @@ 1 1 -1 +0 \N 100000000000000000000 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 06f4a9cfc99..bdbc019d4f8 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -1,22 +1,22 @@ -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.value_1 String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.value_1 String __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -50,29 +50,25 @@ Positions: 4 0 2 1 Parts: 1 Granules: 1 -- -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT :: 2 -> __table1.value_2 UInt64 : 2 + INPUT : 2 -> __table2.id UInt64 : 2 INPUT : 3 -> __table2.value_1 String : 3 - INPUT :: 4 -> __table2.value_2 UInt64 : 4 - INPUT : 5 -> __table2.id UInt64 : 5 - ALIAS __table1.id :: 0 -> id UInt64 : 6 + ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 -Positions: 6 0 3 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table1.value_2 UInt64 - __table2.value_1 String - __table2.value_2 UInt64 __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ASOF Algorithm: HashJoin diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index 1cdd3684a0b..11f40fb8887 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,6 +19,8 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); +SET query_plan_join_swap_table = 'false'; + EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.reference b/tests/queries/0_stateless/02962_join_using_bug_57894.reference index 454655081df..fc6fe462205 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.reference +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.reference @@ -31,6 +31,7 @@ 8 9 \N +--- analyzer --- 0 1 2 diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index 96190241da5..e29347beb5e 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -21,6 +21,8 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; +SELECT '--- analyzer ---'; + SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index d0a3e7b02ae..1bea145c50a 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -2,7 +2,9 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -10,18 +12,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -69,7 +71,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false'; +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -77,18 +81,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -136,7 +140,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -144,18 +150,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -214,7 +220,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -222,18 +230,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -281,7 +289,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -289,31 +299,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -355,7 +365,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -363,31 +375,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -429,7 +441,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -437,18 +451,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -496,7 +510,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -504,31 +520,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -570,7 +586,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -578,31 +596,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -644,7 +662,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_swap_table = 'false' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -652,18 +672,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String AND column: equals(__table1.id, 5_UInt8) Actions: INPUT : 0 -> __table1.id UInt64 : 0 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1 @@ -672,18 +692,18 @@ Positions: 4 2 0 1 Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT :: 1 -> __table1.id UInt64 : 0 INPUT :: 2 -> __table1.value String : 1 - INPUT :: 3 -> __table2.value String : 2 - INPUT : 4 -> __table2.id UInt64 : 3 + INPUT : 3 -> __table2.id UInt64 : 2 + INPUT :: 4 -> __table2.value String : 3 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4 INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5 - FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 + FUNCTION equals(__table2.id : 2, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 FUNCTION and(equals(__table1.id, 5_UInt8) :: 5, equals(__table2.id, 6_UInt8) :: 6) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 4 Positions: 4 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index e1a13d1ce71..a7b98b95082 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -22,7 +22,9 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -33,7 +35,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false'; +; SELECT '--'; @@ -44,7 +48,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6; @@ -53,7 +59,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -64,7 +72,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -75,7 +85,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -86,7 +98,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -97,7 +111,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -108,7 +124,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -119,7 +137,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index cf070eebc38..64d4d0d0eb1 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -52,7 +52,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_swap_table = 'false' +; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] 2 3 arc 2 -> 3 false [(2,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index 7dad74893b9..65e4439c1fd 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -55,7 +55,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_swap_table = 'false' +; -- ordering by the path column has same effect as SEARCH DEPTH FIRST WITH RECURSIVE search_graph AS ( diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 6ae4e4d4d3c..a3f04fb8870 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,6 +3,7 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES +SET query_plan_join_swap_table = 'auto'; -- 'true' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index d35bdeff98b..5fde4f80c5d 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -5,18 +5,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -75,18 +75,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -145,18 +145,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index b3d1827d98f..0a53bf03ad5 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -22,7 +22,10 @@ SETTINGS index_granularity = 16 INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0; + +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -30,7 +33,9 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; @@ -38,7 +43,9 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 +SETTINGS query_plan_join_swap_table = 'false' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference index 7058d36aaf9..1c82e76cc65 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference @@ -65,8 +65,7 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice'; Expression ((Project names + (Projection + ))) Header: name String Join (JOIN FillRightFirst) - Header: __table1.name String - __table2.name String + Header: __table2.name String Filter (( + Change column names to column identifiers)) Header: __table1.name String ReadFromMergeTree (default.users) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index f6e5dbdef03..20a8be6d499 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,6 +11,7 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; +SET query_plan_join_swap_table = 'false'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, diff --git a/tests/queries/0_stateless/03267_join_swap_bug.reference b/tests/queries/0_stateless/03267_join_swap_bug.reference new file mode 100644 index 00000000000..68b8a88e2c1 --- /dev/null +++ b/tests/queries/0_stateless/03267_join_swap_bug.reference @@ -0,0 +1,4 @@ +1 +1 +0 +1 diff --git a/tests/queries/0_stateless/03267_join_swap_bug.sql b/tests/queries/0_stateless/03267_join_swap_bug.sql new file mode 100644 index 00000000000..1ccfa4924be --- /dev/null +++ b/tests/queries/0_stateless/03267_join_swap_bug.sql @@ -0,0 +1,33 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO TABLE t0 (c0) VALUES (1); + +SELECT 1 FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; +SELECT count() FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; + +SET allow_experimental_analyzer = 1; + +SELECT * +FROM +( + SELECT * + FROM system.one +) AS a +INNER JOIN +( + SELECT * + FROM system.one +) AS b USING (dummy) +INNER JOIN +( + SELECT * + FROM system.one +) AS c USING (dummy) +SETTINGS join_algorithm = 'full_sorting_merge'; + + +SELECT count(1) +FROM ( SELECT 1 AS x, x ) AS t1 +RIGHT JOIN (SELECT materialize(2) AS x) AS t2 +ON t1.x = t2.x +; From 6faa47fad5d3df5d4009bdc8da148c3484b9ed43 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 26 Nov 2024 13:02:38 +0000 Subject: [PATCH 332/433] Fix assert, add 03273_join_columns_comprehensive --- .../HashJoin/HashJoinMethodsImpl.h | 2 +- ...03273_join_columns_comprehensive.reference | 0 .../03273_join_columns_comprehensive.sql.j2 | 64 +++++++++++++++++++ 3 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03273_join_columns_comprehensive.reference create mode 100644 tests/queries/0_stateless/03273_join_columns_comprehensive.sql.j2 diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index b0fc7936278..1e52278f020 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -191,9 +191,9 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( columns[pos] = columns[pos]->replicate(offsets); block.getSourceBlock().setColumns(columns); - block.getSourceBlock().erase(block_columns_to_erase); block = ScatteredBlock(std::move(block).getSourceBlock()); } + block.getSourceBlock().erase(block_columns_to_erase); return remaining_block; } diff --git a/tests/queries/0_stateless/03273_join_columns_comprehensive.reference b/tests/queries/0_stateless/03273_join_columns_comprehensive.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03273_join_columns_comprehensive.sql.j2 b/tests/queries/0_stateless/03273_join_columns_comprehensive.sql.j2 new file mode 100644 index 00000000000..0d88e304c6f --- /dev/null +++ b/tests/queries/0_stateless/03273_join_columns_comprehensive.sql.j2 @@ -0,0 +1,64 @@ +-- Tags: long, no-asan, no-tsan, no-ubsan, no-msan + +-- This test verifies assertions and logical errors when selecting different subsets of columns using various join types and algorithms + +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; + +CREATE TABLE t0 (c0 UInt32, c1 String, c2 String) ENGINE = MergeTree ORDER BY c0; +INSERT INTO t0 VALUES (1, 'a', 'b'), (2, 'c', 'd'), (3, 'e', 'f'); + +CREATE TABLE t1 (c0 UInt32, c1 String, c2 String) ENGINE = MergeTree ORDER BY c0; +INSERT INTO t1 VALUES (2, 'c', 'd'), (3, 'e', 'f'), (4, 'g', 'h'); + +{% set columns_list =[ + '*', + 't0.c0', + 't1.c0', + 't0.c2', + 't1.c2', + 't1.c2, t1.c0', + 't1.c2, t0.c0', + 'count()', + '1', +] -%} + +{%- for kind in [ + 'INNER', + 'LEFT', + 'RIGHT', + 'FULL', + 'ANY INNER', + 'ANY LEFT', + 'ANY RIGHT', + 'ANTI LEFT', + 'ANTI RIGHT', + 'SEMI LEFT', + 'SEMI RIGHT', +] -%} + +{%- for columns in columns_list -%} +{%- for join_algorithm in ['default', 'hash', 'full_sorting_merge', 'parallel_hash', 'grace_hash'] -%} +{%- for condition in ['ON t1.c0 = t0.c0', 'USING (c0)'] -%} + +{%- set is_not_supported = ( + ('ANTI' in kind and join_algorithm == 'full_sorting_merge') or + ('SEMI' in kind and join_algorithm == 'full_sorting_merge') +) -%} + +SELECT {{ columns }} FROM t0 {{ kind }} JOIN t1 {{ condition }} +SETTINGS join_algorithm = '{{ join_algorithm }}' +FORMAT Null; {{ '-- { serverError NOT_IMPLEMENTED }' if is_not_supported else '' }} + +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} + +{% for kind in ['ASOF', 'ASOF LEFT'] -%} +{%- for columns in columns_list -%} + +SELECT {{ columns }} FROM t0 {{ kind }} JOIN t1 ON t1.c1 = t0.c1 AND t1.c0 <= t0.c0 FORMAT Null; + +{% endfor -%} +{% endfor -%} From 0ebee19f2e3494df1c53b1d260fce31472772dcb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 26 Nov 2024 09:21:58 -0400 Subject: [PATCH 333/433] Update docs/en/sql-reference/window-functions/leadInFrame.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/window-functions/leadInFrame.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index 16c7aefd81a..c2bc2f525c5 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -11,7 +11,7 @@ Returns a value evaluated at the row that is offset rows after the current row w :::warning `leadInFrame` behavior differs from the standard SQL `lead` window function. Clickhouse window function `leadInFrame` respects the window frame. -To get behavior identical to the `lead`, use `rows between unbounded preceding and unbounded following`. +To get behavior identical to the `lead`, use `ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. ::: **Syntax** From 5b1bdef54f047e3395031f8bc1bc4acba0c15eaf Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 26 Nov 2024 09:24:42 -0400 Subject: [PATCH 334/433] Update lagInFrame.md --- docs/en/sql-reference/window-functions/lagInFrame.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index c4b7b377761..fb311ab6fcb 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -11,7 +11,7 @@ Returns a value evaluated at the row that is at a specified physical offset row :::warning `lagInFrame` behavior differs from the standard SQL `lag` window function. Clickhouse window function `lagInFrame` respects the window frame. -To get behavior identical to the `lag`, use `rows between unbounded preceding and unbounded following`. +To get behavior identical to the `lag`, use `ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING`. ::: **Syntax** From 8bacc8f0fdd289058aaed7da5947f203636117d5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 26 Nov 2024 14:03:58 +0000 Subject: [PATCH 335/433] randomize settings for priamry key cache --- tests/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 274232f0985..aad3e619e81 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -984,6 +984,8 @@ class MergeTreeSettingsRandomizer: "prewarm_mark_cache": lambda: random.randint(0, 1), "use_const_adaptive_granularity": lambda: random.randint(0, 1), "enable_index_granularity_compression": lambda: random.randint(0, 1), + "use_primary_key_cache": lambda: random.randint(0, 1), + "prewarm_primary_key_cache": lambda: random.randint(0, 1), } @staticmethod From 1bbc585f4069cbee3433162a9ed7c4d53519fada Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 26 Nov 2024 15:44:10 +0000 Subject: [PATCH 336/433] fix build --- src/Storages/StorageMergeTreeIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index c082db68549..8f727114752 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -68,7 +68,7 @@ protected: const auto & part_name_column = StorageMergeTreeIndex::part_name_column; const auto & mark_number_column = StorageMergeTreeIndex::mark_number_column; const auto & rows_in_granule_column = StorageMergeTreeIndex::rows_in_granule_column; - IMergeTreeDataPart::Index index_ptr; + IMergeTreeDataPart::IndexPtr index_ptr; Columns result_columns(num_columns); for (size_t pos = 0; pos < num_columns; ++pos) From 934ae95e396035b7229d25cc02a727e0c2398902 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 26 Nov 2024 17:34:27 +0000 Subject: [PATCH 337/433] Fix bad conflict resolution --- docs/en/operations/server-configuration-parameters/settings.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 044a650744b..fe38e00b637 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3277,6 +3277,8 @@ Type: UInt64 Default value: 100 +Zero means unlimited + ## allow_feature_tier Controls if the user can change settings related to the different feature tiers. From 627e33379fcc7da0f50ac13702aaf8515f8b5dd4 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 26 Nov 2024 16:28:40 +0000 Subject: [PATCH 338/433] add metric MergeTreeIndexGranularityInternalArraysTotalSizeMetric --- src/Common/CurrentMetrics.cpp | 2 ++ .../MergeTree/MergeTreeIndexGranularityAdaptive.cpp | 8 ++++++++ 2 files changed, 10 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index d073e477593..fa3c383cc64 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -354,6 +354,8 @@ M(SharedCatalogDropZooKeeperThreadsScheduled, "Number of queued or active jobs in the threadpool for drop of object in ZooKeeper in Shared Catalog.") \ \ M(SharedDatabaseCatalogTablesInLocalDropDetachQueue, "Number of tables in the queue for local drop or detach in Shared Catalog.") \ + \ + M(MergeTreeIndexGranularityInternalArraysTotalSize, "The total size of all internal arrays in Merge Tree index granularity objects in bytes.") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityAdaptive.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityAdaptive.cpp index d51afb5be69..2815d6705fc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityAdaptive.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityAdaptive.cpp @@ -1,7 +1,13 @@ +#include #include #include +namespace CurrentMetrics +{ + extern const Metric MergeTreeIndexGranularityInternalArraysTotalSize; +} + namespace DB { @@ -60,6 +66,7 @@ void MergeTreeIndexGranularityAdaptive::appendMark(size_t rows_count) { marks_rows_partial_sums.push_back(marks_rows_partial_sums.back() + rows_count); } + CurrentMetrics::add(CurrentMetrics::MergeTreeIndexGranularityInternalArraysTotalSize, sizeof(decltype(marks_rows_partial_sums)::value_type)); } void MergeTreeIndexGranularityAdaptive::adjustLastMark(size_t rows_count) @@ -75,6 +82,7 @@ void MergeTreeIndexGranularityAdaptive::adjustLastMark(size_t rows_count) else { marks_rows_partial_sums.pop_back(); + CurrentMetrics::sub(CurrentMetrics::MergeTreeIndexGranularityInternalArraysTotalSize, sizeof(decltype(marks_rows_partial_sums)::value_type)); appendMark(rows_count); } } From 22f34a2fe9393543f345de03990150fce12108b7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Nov 2024 18:03:24 +0000 Subject: [PATCH 339/433] Update version_date.tsv and changelogs after v24.11.1.2557-stable --- SECURITY.md | 1 + docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.11.1.2557-stable.md | 376 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 381 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.11.1.2557-stable.md diff --git a/SECURITY.md b/SECURITY.md index 1b0648dc489..5722b896b52 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -14,6 +14,7 @@ The following versions of ClickHouse server are currently supported with securit | Version | Supported | |:-|:-| +| 24.11 | ✔️ | | 24.10 | ✔️ | | 24.9 | ✔️ | | 24.8 | ✔️ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index f3f25c1a247..b933b426e01 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -38,7 +38,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.10.3.21" +ARG VERSION="24.11.1.2557" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 3bf23767150..36ceb78e6ef 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -35,7 +35,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.10.3.21" +ARG VERSION="24.11.1.2557" 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 31fdcb8a490..b2732613142 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,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.10.3.21" +ARG VERSION="24.11.1.2557" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.11.1.2557-stable.md b/docs/changelogs/v24.11.1.2557-stable.md new file mode 100644 index 00000000000..4d18897263c --- /dev/null +++ b/docs/changelogs/v24.11.1.2557-stable.md @@ -0,0 +1,376 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.11.1.2557-stable (1574e794bf9) FIXME as compared to v24.11.1.1-new (c82cf25b3e5) + +#### Backward Incompatible Change +* Remove system tables `generate_series` and `generateSeries`. They were added by mistake here: [#59390](https://github.com/ClickHouse/ClickHouse/issues/59390). [#71091](https://github.com/ClickHouse/ClickHouse/pull/71091) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600). ### Documentation entry for user-facing changes. [#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). +* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). +* The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the `SOURCES` hierarchy. Add grants to any non-default database users that create tables with these engine types. [#71250](https://github.com/ClickHouse/ClickHouse/pull/71250) ([Christoph Wurm](https://github.com/cwurm)). +* Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. [#71300](https://github.com/ClickHouse/ClickHouse/pull/71300) ([Christoph Wurm](https://github.com/cwurm)). +* Rename filesystem cache setting `skip_download_if_exceeds_query_cache` to `filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit`. [#71578](https://github.com/ClickHouse/ClickHouse/pull/71578) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for `Enum` as well as `UInt128` and `UInt256` arguments in `deltaSumTimestamp`. Remove support for `Int8`, `UInt8`, `Int16`, and `UInt16` of the second ("timestamp") argument of `deltaSumTimestamp`. [#71790](https://github.com/ClickHouse/ClickHouse/pull/71790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ~~Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled by `query_plan_join_inner_table_selection`, which can be set to `auto`. In this mode, ClickHouse will try to choose the table with the smallest number of rows.~~ Resubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. [#68682](https://github.com/ClickHouse/ClickHouse/pull/68682) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). +* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). +* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). +* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). +* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). +* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). +* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). +* Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). +* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). +* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). +* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). +* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). +* Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). + +#### Performance Improvement +* Add 2 new settings `short_circuit_function_evaluation_for_nulls` and `short_circuit_function_evaluation_for_nulls_threshold` that allow to execute functions over `Nullable` columns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. [#60129](https://github.com/ClickHouse/ClickHouse/pull/60129) ([李扬](https://github.com/taiyang-li)). +* Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). +* Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Improvement +* Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). +* Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). +* Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). +* Fix use-after-dtor logic in HashTable destroyElements. [#65279](https://github.com/ClickHouse/ClickHouse/pull/65279) ([cangyin](https://github.com/cangyin)). +* Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). +* Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). +* 1. Refactor `DDLQueryStatusSource`: * Rename `DDLQueryStatusSource` to `DistributedQueryStatusSource`, and make it a base class * Create two subclasses `DDLOnClusterQueryStatusSource` and `ReplicatedDatabaseQueryStatusSource` derived from `DDLQueryStatusSource` to query the status of DDL tasks from `DDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in `DDLOnClusterQueryStatusSource`. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). +* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). +* Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). +* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). +* Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). +* Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). +* Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). +* Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). +* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). +* Boolean support for parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). +* Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). +* `system.session_log` is quite okay. This closes [#51760](https://github.com/ClickHouse/ClickHouse/issues/51760). [#71150](https://github.com/ClickHouse/ClickHouse/pull/71150) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). +* Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). +* Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add per host dashboards `Overview (host)` and `Cloud overview (host)` to advanced dashboard. [#71422](https://github.com/ClickHouse/ClickHouse/pull/71422) ([alesapin](https://github.com/alesapin)). +* The methods `removeObject` and `removeObjects` are not idempotent. When retries happen due to network errors, the result could be `object not found` because it has been deleted at previous attempts. [#71529](https://github.com/ClickHouse/ClickHouse/pull/71529) ([Sema Checherinda](https://github.com/CheSema)). +* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). +* Allow using clickhouse with a file argument as --queries-file. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). +* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). +* `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. ### Documentation entry for user-facing changes. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). +* Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). +* Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). +* Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). +* Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Turn-off filesystem cache setting `boundary_alignment` for non-disk read. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). +* Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). +* Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). +* Do not increment the `ILLEGAL_TYPE_OF_ARGUMENT` counter in the `system.errors` table when the `bitmapTransform` function is used, and argument types are valid. [#71971](https://github.com/ClickHouse/ClickHouse/pull/71971) ([Dmitry Novik](https://github.com/novikd)). +* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). +* Backported in [#72471](https://github.com/ClickHouse/ClickHouse/issues/72471): Move JSON/Dynamic/Variant types from experimental features to beta. [#72294](https://github.com/ClickHouse/ClickHouse/pull/72294) ([Pavel Kruglov](https://github.com/Avogar)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). +* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). +* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). +* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). +* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). +* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). +* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). +* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). +* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). +* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). +* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). +* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). +* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). +* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). +* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). +* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). +* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). +* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#72336](https://github.com/ClickHouse/ClickHouse/issues/72336): Fix NoSuchKey error during transaction rollback when creating a directory fails for the palin_rewritable disk. [#71439](https://github.com/ClickHouse/ClickHouse/pull/71439) ([Julia Kartseva](https://github.com/jkartseva)). +* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). +* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). +* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). +* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). +* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). +* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). +* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). +* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). +* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). +* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). +* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). +* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). +* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). +* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Backported in [#72275](https://github.com/ClickHouse/ClickHouse/issues/72275): Fix serialization of Dynamic values in Pretty JSON formats. [#71923](https://github.com/ClickHouse/ClickHouse/pull/71923) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). +* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). +* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). +* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Backported in [#72300](https://github.com/ClickHouse/ClickHouse/issues/72300): Fix bugs when using UDF in join on expression with the old analyzer. [#72179](https://github.com/ClickHouse/ClickHouse/pull/72179) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#72378](https://github.com/ClickHouse/ClickHouse/issues/72378): Fixed a crash in `SimpleSquashingChunksTransform` that occurred in rare cases when processing sparse columns. [#72226](https://github.com/ClickHouse/ClickHouse/pull/72226) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Backported in [#72329](https://github.com/ClickHouse/ClickHouse/issues/72329): Fixed data race in `GraceHashJoin` as the result of which some rows might be missing in the join output. [#72233](https://github.com/ClickHouse/ClickHouse/pull/72233) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#72372](https://github.com/ClickHouse/ClickHouse/issues/72372): Fixed `ALTER DELETE` queries with materialized `_block_number` column (if setting `enable_block_number_column` is enabled). [#72261](https://github.com/ClickHouse/ClickHouse/pull/72261) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#72318](https://github.com/ClickHouse/ClickHouse/issues/72318): Fixed data race when `ColumnDynamic::dumpStructure()` is called concurrently e.g. in `ConcurrentHashJoin` constructor. [#72278](https://github.com/ClickHouse/ClickHouse/pull/72278) ([Nikita Taranov](https://github.com/nickitat)). +* Backported in [#72420](https://github.com/ClickHouse/ClickHouse/issues/72420): Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN_PATH`. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#72397](https://github.com/ClickHouse/ClickHouse/pull/72397) ([Konstantin Bogdanov](https://github.com/thevar1able)). + +#### Build/Testing/Packaging Improvement +* Add the script to update sources of [docker official library](https://github.com/ClickHouse/docker-library). [#57203](https://github.com/ClickHouse/ClickHouse/pull/57203) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The build system will prevent libraries with unexpected licenses. [#70988](https://github.com/ClickHouse/ClickHouse/pull/70988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Upgrade docker base image for clickhouse-server and keeper to `ubuntu:22.04`. **Breaking change**: the minimal supported docker version is `20.10.10`. [#71505](https://github.com/ClickHouse/ClickHouse/pull/71505) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve clickhouse-server Dockerfile.ubuntu. Deprecate `CLICKHOUSE_UID/CLICKHOUSE_GID` envs. Remove `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` processing to complien requirements. Consistent `clickhouse/clickhouse-server/clickhouse-keeper` execution to not have it plain in one place and `/usr/bin/clickhouse*` in another. [#71573](https://github.com/ClickHouse/ClickHouse/pull/71573) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NO CL CATEGORY + +* Backported in [#72289](https://github.com/ClickHouse/ClickHouse/issues/72289):. [#72258](https://github.com/ClickHouse/ClickHouse/pull/72258) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#72343](https://github.com/ClickHouse/ClickHouse/issues/72343):. [#72319](https://github.com/ClickHouse/ClickHouse/pull/72319) ([Raúl Marín](https://github.com/Algunenano)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Miscellaneous"'. [#71083](https://github.com/ClickHouse/ClickHouse/pull/71083) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. [#71084](https://github.com/ClickHouse/ClickHouse/pull/71084) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. [#71251](https://github.com/ClickHouse/ClickHouse/pull/71251) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. [#71266](https://github.com/ClickHouse/ClickHouse/pull/71266) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Selection of hash join inner table"'. [#71527](https://github.com/ClickHouse/ClickHouse/pull/71527) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. [#71619](https://github.com/ClickHouse/ClickHouse/pull/71619) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. [#71914](https://github.com/ClickHouse/ClickHouse/pull/71914) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. [#71945](https://github.com/ClickHouse/ClickHouse/pull/71945) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. [#71974](https://github.com/ClickHouse/ClickHouse/pull/71974) ([Max Kainov](https://github.com/maxknv)). +* NO CL ENTRY: 'CI: Functional Tests with praktika'. [#71976](https://github.com/ClickHouse/ClickHouse/pull/71976) ([Max Kainov](https://github.com/maxknv)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Refactor TempDataOnDisk. [#66606](https://github.com/ClickHouse/ClickHouse/pull/66606) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Bump krb5 from v1.21.2 to v1.21.3. [#69360](https://github.com/ClickHouse/ClickHouse/pull/69360) ([Robert Schulze](https://github.com/rschu1ze)). +* USearch: Enable SimSIMD backend + enable dynamic dispatch. [#69387](https://github.com/ClickHouse/ClickHouse/pull/69387) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: enable libfuzzer. [#70112](https://github.com/ClickHouse/ClickHouse/pull/70112) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Use `clang-19`. [#70414](https://github.com/ClickHouse/ClickHouse/pull/70414) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Randomize Keeper feature flags in integration tests. [#70523](https://github.com/ClickHouse/ClickHouse/pull/70523) ([Antonio Andelic](https://github.com/antonio2368)). +* All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. [#70691](https://github.com/ClickHouse/ClickHouse/pull/70691) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). +* Add a settings `filesystem_cache_enable_background_download_for_metadata_files` for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. [#70806](https://github.com/ClickHouse/ClickHouse/pull/70806) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make ParquetMetadata say whether bloom filter is present. [#70947](https://github.com/ClickHouse/ClickHouse/pull/70947) ([Michael Kolupaev](https://github.com/al13n321)). +* test for reproducing that ReplacingMergeTree depends on the order of part attachment. [#71010](https://github.com/ClickHouse/ClickHouse/pull/71010) ([Konstantin Morozov](https://github.com/k-morozov)). +* CI: Build Job with praktika. [#71015](https://github.com/ClickHouse/ClickHouse/pull/71015) ([Max Kainov](https://github.com/maxknv)). +* Fix bad test `01524_do_not_merge_across_partitions_select_final.sql`. [#71035](https://github.com/ClickHouse/ClickHouse/pull/71035) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable enable_job_stack_trace by default. [#71039](https://github.com/ClickHouse/ClickHouse/pull/71039) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix two logical errors when reading from stdin in clickhouse local. [#71046](https://github.com/ClickHouse/ClickHouse/pull/71046) ([Michael Kolupaev](https://github.com/al13n321)). +* Sync changes to `ProtocolServerAdapter`. [#71058](https://github.com/ClickHouse/ClickHouse/pull/71058) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a typo. [#71067](https://github.com/ClickHouse/ClickHouse/pull/71067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71070](https://github.com/ClickHouse/ClickHouse/pull/71070) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove bad test `test_system_replicated_fetches`. [#71071](https://github.com/ClickHouse/ClickHouse/pull/71071) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version after release. [#71076](https://github.com/ClickHouse/ClickHouse/pull/71076) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* CI: Functional tests for ARM + ASAN binary. [#71079](https://github.com/ClickHouse/ClickHouse/pull/71079) ([Max Kainov](https://github.com/maxknv)). +* CI: Functional Tests with praktika. [#71081](https://github.com/ClickHouse/ClickHouse/pull/71081) ([Max Kainov](https://github.com/maxknv)). +* Fixup of TrivialMergeSelector. [#71082](https://github.com/ClickHouse/ClickHouse/pull/71082) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Sync integration test with private. [#71096](https://github.com/ClickHouse/ClickHouse/pull/71096) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unique symbols in the `system.coverage_log`. [#71099](https://github.com/ClickHouse/ClickHouse/pull/71099) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better log messages. [#71102](https://github.com/ClickHouse/ClickHouse/pull/71102) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix error in Replicated database. [#71103](https://github.com/ClickHouse/ClickHouse/pull/71103) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update error message for JSONAsObject format. [#71123](https://github.com/ClickHouse/ClickHouse/pull/71123) ([Pavel Kruglov](https://github.com/Avogar)). +* Initial changelog for 24.10. [#71127](https://github.com/ClickHouse/ClickHouse/pull/71127) ([Raúl Marín](https://github.com/Algunenano)). +* Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). +* Try fix rabbitmq. [#71143](https://github.com/ClickHouse/ClickHouse/pull/71143) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Randomize setting `enable_vertical_final`. [#71144](https://github.com/ClickHouse/ClickHouse/pull/71144) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bad test `02561_sorting_constants_and_distinct_crash`. [#71147](https://github.com/ClickHouse/ClickHouse/pull/71147) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test to verify [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308) works. [#71149](https://github.com/ClickHouse/ClickHouse/pull/71149) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix 02932_refreshable_materialized_views_1 flakiness. [#71160](https://github.com/ClickHouse/ClickHouse/pull/71160) ([Michael Kolupaev](https://github.com/al13n321)). +* Use `_minmax_count_projection` instead of `Optimized trivial count` for `ReadFromPreparedSource` node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses [#70939](https://github.com/ClickHouse/ClickHouse/issues/70939). [#71166](https://github.com/ClickHouse/ClickHouse/pull/71166) ([Amos Bird](https://github.com/amosbird)). +* Close [#8687](https://github.com/ClickHouse/ClickHouse/issues/8687). [#71169](https://github.com/ClickHouse/ClickHouse/pull/71169) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). +* Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). +* Print compression method in `clickhouse-compressor --stat`. Useful for inspecting random data files. [#71192](https://github.com/ClickHouse/ClickHouse/pull/71192) ([Amos Bird](https://github.com/amosbird)). +* Updating the events into the recent category and adding the new york event. [#71194](https://github.com/ClickHouse/ClickHouse/pull/71194) ([Zoe Steinkamp](https://github.com/zoesteinkamp)). +* Improve error and log messages around memory usage. [#71195](https://github.com/ClickHouse/ClickHouse/pull/71195) ([Raúl Marín](https://github.com/Algunenano)). +* Minor test adjustments. [#71199](https://github.com/ClickHouse/ClickHouse/pull/71199) ([Raúl Marín](https://github.com/Algunenano)). +* Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). +* Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#71223](https://github.com/ClickHouse/ClickHouse/pull/71223) ([Tanya Bragin](https://github.com/tbragin)). +* Fix `WITH TOTALS` in subquery with parallel replicas. [#71224](https://github.com/ClickHouse/ClickHouse/pull/71224) ([Nikita Taranov](https://github.com/nickitat)). +* Ignore `No such key` exceptions in some cases. [#71236](https://github.com/ClickHouse/ClickHouse/pull/71236) ([Antonio Andelic](https://github.com/antonio2368)). +* Make cloud sync title shorter. [#71255](https://github.com/ClickHouse/ClickHouse/pull/71255) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Meetups update. [#71271](https://github.com/ClickHouse/ClickHouse/pull/71271) ([Tanya Bragin](https://github.com/tbragin)). +* Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). +* Expose one more simple merge selector setting. [#71313](https://github.com/ClickHouse/ClickHouse/pull/71313) ([alesapin](https://github.com/alesapin)). +* Better style for some sever-level settings. [#71319](https://github.com/ClickHouse/ClickHouse/pull/71319) ([alesapin](https://github.com/alesapin)). +* Sync some changes. [#71321](https://github.com/ClickHouse/ClickHouse/pull/71321) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add library to requirements for style-check and fix warning. [#71322](https://github.com/ClickHouse/ClickHouse/pull/71322) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix test `test_workload_entity_keeper_storage`: add more retries. [#71325](https://github.com/ClickHouse/ClickHouse/pull/71325) ([Sergei Trifonov](https://github.com/serxa)). +* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. [#71335](https://github.com/ClickHouse/ClickHouse/pull/71335) ([Chang chen](https://github.com/baibaichen)). +* Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. [#71336](https://github.com/ClickHouse/ClickHouse/pull/71336) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Remove deprecated release script. [#71341](https://github.com/ClickHouse/ClickHouse/pull/71341) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.1.2812-stable. [#71343](https://github.com/ClickHouse/ClickHouse/pull/71343) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Bump USearch to 2.16.0 and add more tests. [#71344](https://github.com/ClickHouse/ClickHouse/pull/71344) ([Robert Schulze](https://github.com/rschu1ze)). +* check-doc-aspell: Print full path to script in CI report. [#71345](https://github.com/ClickHouse/ClickHouse/pull/71345) ([Vladimir Cherkasov](https://github.com/vdimir)). +* CI: Fix fedora version in create release workflow. [#71347](https://github.com/ClickHouse/ClickHouse/pull/71347) ([Max Kainov](https://github.com/maxknv)). +* fs cache: add assertions. [#71348](https://github.com/ClickHouse/ClickHouse/pull/71348) ([Kseniia Sumarokova](https://github.com/kssenii)). +* More info in TOO_SLOW exception. [#71365](https://github.com/ClickHouse/ClickHouse/pull/71365) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix unused variables around WorkloadEntityStorageBase. [#71367](https://github.com/ClickHouse/ClickHouse/pull/71367) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow to prewarm mark cache by system command without enabled setting. [#71368](https://github.com/ClickHouse/ClickHouse/pull/71368) ([Anton Popov](https://github.com/CurtizJ)). +* Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. [#71380](https://github.com/ClickHouse/ClickHouse/pull/71380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Rename `compare8` to `compare16` for consistency. [#71416](https://github.com/ClickHouse/ClickHouse/pull/71416) ([Nikita Taranov](https://github.com/nickitat)). +* Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. [#71418](https://github.com/ClickHouse/ClickHouse/pull/71418) ([Azat Khuzhin](https://github.com/azat)). +* [Experiment] Analyzer: Check what happens after if-condition removal. [#71425](https://github.com/ClickHouse/ClickHouse/pull/71425) ([Dmitry Novik](https://github.com/novikd)). +* Update version_date.tsv and changelog after v24.8.6.70-lts. [#71428](https://github.com/ClickHouse/ClickHouse/pull/71428) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix upgrade check (24.11). [#71438](https://github.com/ClickHouse/ClickHouse/pull/71438) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assert during insert into vector similarity index in presence of other skipping indexes. [#71457](https://github.com/ClickHouse/ClickHouse/pull/71457) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid `seconds left [-3]` in cache await. [#71468](https://github.com/ClickHouse/ClickHouse/pull/71468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added tests for corner cases for 24.10. [#71469](https://github.com/ClickHouse/ClickHouse/pull/71469) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Expose base setting for merge selector. [#71497](https://github.com/ClickHouse/ClickHouse/pull/71497) ([alesapin](https://github.com/alesapin)). +* Fixed incorrect settings order `max_parser_depth` and `max_parser_backtracks`. [#71498](https://github.com/ClickHouse/ClickHouse/pull/71498) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support the endpoint of oss accelerator. [#71502](https://github.com/ClickHouse/ClickHouse/pull/71502) ([Kai Zhu](https://github.com/nauu)). +* Fix flaky test_drop_complex_columns. [#71504](https://github.com/ClickHouse/ClickHouse/pull/71504) ([Ilya Golshtein](https://github.com/ilejn)). +* Move bitShift function changelog entries to backward incompatible. [#71510](https://github.com/ClickHouse/ClickHouse/pull/71510) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix copy/paste error. [#71513](https://github.com/ClickHouse/ClickHouse/pull/71513) ([Denny Crane](https://github.com/den-crane)). +* Allow specifying cmdline flags in integration test. It's needed by [#71452](https://github.com/ClickHouse/ClickHouse/issues/71452) to validate a bugfix. [#71523](https://github.com/ClickHouse/ClickHouse/pull/71523) ([Amos Bird](https://github.com/amosbird)). +* Add ProfileEvents for merge selector timings. [#71524](https://github.com/ClickHouse/ClickHouse/pull/71524) ([alesapin](https://github.com/alesapin)). +* Minor: Remove "experimental" mention of analyzer. [#71525](https://github.com/ClickHouse/ClickHouse/pull/71525) ([Robert Schulze](https://github.com/rschu1ze)). +* Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. [#71530](https://github.com/ClickHouse/ClickHouse/pull/71530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add test to check that accessing system.functions does not populate query_log used_functions. [#71535](https://github.com/ClickHouse/ClickHouse/pull/71535) ([Raúl Marín](https://github.com/Algunenano)). +* Improve `query_plan_merge_filters` optimization. Fixes [#71408](https://github.com/ClickHouse/ClickHouse/issues/71408). [#71539](https://github.com/ClickHouse/ClickHouse/pull/71539) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix compatibility with refreshable materialized views created by old clickhouse servers. [#71556](https://github.com/ClickHouse/ClickHouse/pull/71556) ([Michael Kolupaev](https://github.com/al13n321)). +* Vector similarity index: Re-introduce support for legacy index creation syntax. [#71572](https://github.com/ClickHouse/ClickHouse/pull/71572) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid port clash in CoordinationTest/0.TestSummingRaft1. [#71584](https://github.com/ClickHouse/ClickHouse/pull/71584) ([Raúl Marín](https://github.com/Algunenano)). +* Fix for `00180_no_seek_avoiding_when_reading_from_cache`. [#71596](https://github.com/ClickHouse/ClickHouse/pull/71596) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix for `test_storage_s3_queue::test_shards_distributed[ordered-2]`. [#71597](https://github.com/ClickHouse/ClickHouse/pull/71597) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable enable_job_stack_trace by default, second attempt. [#71625](https://github.com/ClickHouse/ClickHouse/pull/71625) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update version_date.tsv and changelog after v24.3.13.40-lts. [#71627](https://github.com/ClickHouse/ClickHouse/pull/71627) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fixes the bug regarding max rows/bytes to read. [#71634](https://github.com/ClickHouse/ClickHouse/pull/71634) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* SimSIMD: Improve suppression for msan false positive. [#71635](https://github.com/ClickHouse/ClickHouse/pull/71635) ([Robert Schulze](https://github.com/rschu1ze)). +* Add `min_parts_to_merge_at_once` merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less than `e` (2.71...) data parts at once because it increases both write amplification and parts number. [#71637](https://github.com/ClickHouse/ClickHouse/pull/71637) ([alesapin](https://github.com/alesapin)). +* Fix broken 03247_ghdata_string_to_json_alter. [#71638](https://github.com/ClickHouse/ClickHouse/pull/71638) ([Pavel Kruglov](https://github.com/Avogar)). +* Update test. [#71654](https://github.com/ClickHouse/ClickHouse/pull/71654) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: fix mysql containers using improper log directory. [#71655](https://github.com/ClickHouse/ClickHouse/pull/71655) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update README.md - Update meetups. [#71657](https://github.com/ClickHouse/ClickHouse/pull/71657) ([Tanya Bragin](https://github.com/tbragin)). +* Add index granularity size column to system.parts. [#71658](https://github.com/ClickHouse/ClickHouse/pull/71658) ([alesapin](https://github.com/alesapin)). +* Update PULL_REQUEST_TEMPLATE.md. [#71687](https://github.com/ClickHouse/ClickHouse/pull/71687) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Historically we have a strange cherry-pick branches naming, e.g. `cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e`. The `cherrypick/24.3/62297` looks nicer and more straightforward. [#71698](https://github.com/ClickHouse/ClickHouse/pull/71698) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. [#71715](https://github.com/ClickHouse/ClickHouse/pull/71715) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix flaky test test_prometheus_protocols. [#71772](https://github.com/ClickHouse/ClickHouse/pull/71772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. [#71782](https://github.com/ClickHouse/ClickHouse/pull/71782) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix processors profile is not collected for subqueries in some cases. [#71787](https://github.com/ClickHouse/ClickHouse/pull/71787) ([Nikita Taranov](https://github.com/nickitat)). +* Lint some stuff. [#71795](https://github.com/ClickHouse/ClickHouse/pull/71795) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Backported in [#72280](https://github.com/ClickHouse/ClickHouse/issues/72280): After https://github.com/ClickHouse/ClickHouse/pull/70442 (not included in any release yet) we use new Native serialization for JSON and Dynamic, but it breaks new client - old server communication. This PR fixes it. Also add a setting `merge_tree_use_v1_object_and_dynamic_serialization` that allows to use V1 serialization in newer versions (it can be used during upgrades to new version to be able to rollback). [#71816](https://github.com/ClickHouse/ClickHouse/pull/71816) ([Pavel Kruglov](https://github.com/Avogar)). +* Closes [#71780](https://github.com/ClickHouse/ClickHouse/issues/71780). [#71818](https://github.com/ClickHouse/ClickHouse/pull/71818) ([Kseniia Sumarokova](https://github.com/kssenii)). +* relax memory limit for 00755_avg_value_size_hint_passing.sql. [#71820](https://github.com/ClickHouse/ClickHouse/pull/71820) ([Sema Checherinda](https://github.com/CheSema)). +* The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper `Dockerfile.ubuntu` there. [#71825](https://github.com/ClickHouse/ClickHouse/pull/71825) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests for parseDateTime64InJodaSyntax. [#71829](https://github.com/ClickHouse/ClickHouse/pull/71829) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Enable build profiling in pull requests. [#71847](https://github.com/ClickHouse/ClickHouse/pull/71847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add check and assertion. [#71856](https://github.com/ClickHouse/ClickHouse/pull/71856) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Some healthcheck is better than nothing. [#71865](https://github.com/ClickHouse/ClickHouse/pull/71865) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* More accurate `calculateCacheKey` implementation. [#71868](https://github.com/ClickHouse/ClickHouse/pull/71868) ([Nikita Taranov](https://github.com/nickitat)). +* add test 03248_max_parts_to_move. [#71869](https://github.com/ClickHouse/ClickHouse/pull/71869) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update README.md - Update meetups. [#71872](https://github.com/ClickHouse/ClickHouse/pull/71872) ([Tanya Bragin](https://github.com/tbragin)). +* Prevents listing files from s3 while inserting. [#71889](https://github.com/ClickHouse/ClickHouse/pull/71889) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Keep query_plan_merge_filters disabled by default. [#71890](https://github.com/ClickHouse/ClickHouse/pull/71890) ([Raúl Marín](https://github.com/Algunenano)). +* Remove useless code. [#71900](https://github.com/ClickHouse/ClickHouse/pull/71900) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* TreeRewriter: fix typo: `parititon` => `partition`. [#71907](https://github.com/ClickHouse/ClickHouse/pull/71907) ([yun](https://github.com/yokofly)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. [#71912](https://github.com/ClickHouse/ClickHouse/pull/71912) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix test_storage_mongodb/test.py::test_secure_connection_uri. [#71924](https://github.com/ClickHouse/ClickHouse/pull/71924) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Adapt some test to run in non-CI configurations. [#71928](https://github.com/ClickHouse/ClickHouse/pull/71928) ([Raúl Marín](https://github.com/Algunenano)). +* Fix build after [#71179](https://github.com/ClickHouse/ClickHouse/issues/71179). Clang-19 checks for unused variables inside of if-conditions. [#71929](https://github.com/ClickHouse/ClickHouse/pull/71929) ([Dmitry Novik](https://github.com/novikd)). +* Fix flaky test 03262_column_sizes_with_dynamic_structure. [#71931](https://github.com/ClickHouse/ClickHouse/pull/71931) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. [#71944](https://github.com/ClickHouse/ClickHouse/pull/71944) ([Nikita Taranov](https://github.com/nickitat)). +* Enabling `query_plan_merge_filters` again after [#71890](https://github.com/ClickHouse/ClickHouse/issues/71890). [#71964](https://github.com/ClickHouse/ClickHouse/pull/71964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add comment about bf16 to CMake docs. [#71973](https://github.com/ClickHouse/ClickHouse/pull/71973) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 00098_primary_key_memory_allocated. [#71977](https://github.com/ClickHouse/ClickHouse/pull/71977) ([Alexander Gololobov](https://github.com/davenger)). +* Add a test for [#71908](https://github.com/ClickHouse/ClickHouse/issues/71908). [#71986](https://github.com/ClickHouse/ClickHouse/pull/71986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71987](https://github.com/ClickHouse/ClickHouse/pull/71987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor follow-up to [#71581](https://github.com/ClickHouse/ClickHouse/issues/71581). [#71993](https://github.com/ClickHouse/ClickHouse/pull/71993) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Stress test with praktika. [#71995](https://github.com/ClickHouse/ClickHouse/pull/71995) ([Max Kainov](https://github.com/maxknv)). +* Fix prewarm of mark cache after adding a new column. [#71996](https://github.com/ClickHouse/ClickHouse/pull/71996) ([Anton Popov](https://github.com/CurtizJ)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. [#72018](https://github.com/ClickHouse/ClickHouse/pull/72018) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix comments. [#72023](https://github.com/ClickHouse/ClickHouse/pull/72023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test for 33604. [#72026](https://github.com/ClickHouse/ClickHouse/pull/72026) ([Nikita Taranov](https://github.com/nickitat)). +* CI: Remove unsafe secret_envs input from yml workflows. [#72028](https://github.com/ClickHouse/ClickHouse/pull/72028) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.2.80-stable. [#72029](https://github.com/ClickHouse/ClickHouse/pull/72029) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.8.7.41-lts. [#72037](https://github.com/ClickHouse/ClickHouse/pull/72037) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.9.3.128-stable. [#72041](https://github.com/ClickHouse/ClickHouse/pull/72041) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.3.14.35-lts. [#72042](https://github.com/ClickHouse/ClickHouse/pull/72042) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Save several minutes of build time. [#72046](https://github.com/ClickHouse/ClickHouse/pull/72046) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#72048](https://github.com/ClickHouse/ClickHouse/pull/72048) ([Tanya Bragin](https://github.com/tbragin)). +* Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix test_disk_over_web_server/. [#72075](https://github.com/ClickHouse/ClickHouse/pull/72075) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Minor improvement for system.query_metric_log stateless test. [#72076](https://github.com/ClickHouse/ClickHouse/pull/72076) ([Pablo Marcos](https://github.com/pamarcos)). +* A follow-up for [#72057](https://github.com/ClickHouse/ClickHouse/issues/72057) and https://github.com/ClickHouse/ClickHouse/pull/71505. [#72079](https://github.com/ClickHouse/ClickHouse/pull/72079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add google-cloud-cpp submodule. [#72092](https://github.com/ClickHouse/ClickHouse/pull/72092) ([Pablo Marcos](https://github.com/pamarcos)). +* CI: Enable fuzzer job in Nightly workflow. [#72101](https://github.com/ClickHouse/ClickHouse/pull/72101) ([Max Kainov](https://github.com/maxknv)). +* Get rid of code duplication after adding `CHECK GRANT` in https://github.com/ClickHouse/ClickHouse/pull/68885. [#72103](https://github.com/ClickHouse/ClickHouse/pull/72103) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add jwt-cpp submodule. [#72104](https://github.com/ClickHouse/ClickHouse/pull/72104) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix list-licenses.sh with OSX. [#72107](https://github.com/ClickHouse/ClickHouse/pull/72107) ([Raúl Marín](https://github.com/Algunenano)). +* fix cancelation for PartitionedSink. [#72126](https://github.com/ClickHouse/ClickHouse/pull/72126) ([Sema Checherinda](https://github.com/CheSema)). +* FIx 02374_analyzer_join_using. [#72145](https://github.com/ClickHouse/ClickHouse/pull/72145) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed a test which was flaky-flaky. [#72147](https://github.com/ClickHouse/ClickHouse/pull/72147) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Backported in [#72236](https://github.com/ClickHouse/ClickHouse/issues/72236): Revert "CI: Stress test with praktika". [#72231](https://github.com/ClickHouse/ClickHouse/pull/72231) ([Max Kainov](https://github.com/maxknv)). +* Backported in [#72345](https://github.com/ClickHouse/ClickHouse/issues/72345): Apply colors correctly to terminal output. [#72283](https://github.com/ClickHouse/ClickHouse/pull/72283) ([Antonio Andelic](https://github.com/antonio2368)). +* Backported in [#72361](https://github.com/ClickHouse/ClickHouse/issues/72361): disable a cloud setting. [#72292](https://github.com/ClickHouse/ClickHouse/pull/72292) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#72469](https://github.com/ClickHouse/ClickHouse/issues/72469): Remove flaky test test_move_shared_lock_fail_keeper_unavailable and extend the stable one. [#72357](https://github.com/ClickHouse/ClickHouse/pull/72357) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#72463](https://github.com/ClickHouse/ClickHouse/issues/72463): Fix data race in Squashing with LowCardinality. [#72392](https://github.com/ClickHouse/ClickHouse/pull/72392) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Backported in [#72453](https://github.com/ClickHouse/ClickHouse/issues/72453): make operations_to_execute as shared ptr. [#72400](https://github.com/ClickHouse/ClickHouse/pull/72400) ([Sema Checherinda](https://github.com/CheSema)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 7591f7050cb..d2f31d4780b 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.11.1.2557-stable 2024-11-26 v24.10.3.21-stable 2024-11-22 v24.10.2.80-stable 2024-11-18 v24.10.1.2812-stable 2024-11-01 From 6e9e0c82be945dbfde179a15b95fb935328a42fb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 Nov 2024 19:39:34 +0100 Subject: [PATCH 340/433] Allow to use pulling executor in materialized views --- src/Core/Settings.cpp | 2 ++ .../Transforms/buildPushingToViewsChain.cpp | 36 ++++++++++++++----- .../0_stateless/03275_matview_with_union.sql | 2 ++ 3 files changed, 32 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 5b5eb254690..3be6f85eae7 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5521,6 +5521,8 @@ The default value is `CURRENT_USER`. DECLARE(UInt64, cache_warmer_threads, 4, R"( Only available in ClickHouse Cloud. Number of background threads for speculatively downloading new data parts into file cache, when cache_populated_by_fetch is enabled. Zero to disable. )", 0) \ + DECLARE(Bool, use_async_executor_for_materialized_views, false, R"( +Use async and potentially multithreaded execution of materialized view query, can speedup views processing during INSERT, but also consume more memory.)", 0) \ DECLARE(Int64, ignore_cold_parts_seconds, 0, R"( Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree. )", 0) \ diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 9d914149b4a..95a7de2614f 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -63,6 +64,7 @@ namespace Setting extern const SettingsUInt64 min_insert_block_size_rows_for_materialized_views; extern const SettingsBool parallel_view_processing; extern const SettingsBool use_concurrency_control; + extern const SettingsBool use_async_executor_for_materialized_views; } namespace ErrorCodes @@ -129,6 +131,7 @@ private: }; /// For source chunk, execute view query over it. +template class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: @@ -148,7 +151,7 @@ private: struct State { QueryPipeline pipeline; - PullingPipelineExecutor executor; + Executor executor; explicit State(QueryPipeline pipeline_) : pipeline(std::move(pipeline_)) @@ -428,17 +431,31 @@ std::optional generateViewChain( out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); #endif - auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); - executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); + if (context->getSettingsRef()[Setting::use_async_executor_for_materialized_views]) + { + auto executing_inner_query = std::make_shared>( + storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); + executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); - out.addSource(std::move(executing_inner_query)); + out.addSource(std::move(executing_inner_query)); + } + else + { + + auto executing_inner_query = std::make_shared>( + storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); + executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); + + out.addSource(std::move(executing_inner_query)); + + } #ifdef ABORT_ON_LOGICAL_ERROR out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); #endif } + return out; } @@ -766,7 +783,8 @@ IProcessor::Status CopyingDataToViewsTransform::prepare() } -ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( +template +ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( const Block & header, ViewRuntimeData & view_, std::shared_ptr views_data_, @@ -778,14 +796,16 @@ ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( { } -void ExecutingInnerQueryFromViewTransform::onConsume(Chunk chunk) +template +void ExecutingInnerQueryFromViewTransform::onConsume(Chunk chunk) { auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); state.emplace(process(std::move(block), view, *views_data, std::move(chunk.getChunkInfos()), disable_deduplication_for_children)); } -ExecutingInnerQueryFromViewTransform::GenerateResult ExecutingInnerQueryFromViewTransform::onGenerate() +template +ExecutingInnerQueryFromViewTransform::GenerateResult ExecutingInnerQueryFromViewTransform::onGenerate() { GenerateResult res; if (!state.has_value()) diff --git a/tests/queries/0_stateless/03275_matview_with_union.sql b/tests/queries/0_stateless/03275_matview_with_union.sql index caf4a6ac5fc..0fc64ae54ca 100644 --- a/tests/queries/0_stateless/03275_matview_with_union.sql +++ b/tests/queries/0_stateless/03275_matview_with_union.sql @@ -2,6 +2,8 @@ DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS matview; +SET use_async_executor_for_materialized_views=1; + CREATE TABLE src ( event_time DateTime, key UInt64, From d78c3d4cba9986bbbec1029a74f3468ba8a79b28 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 27 Nov 2024 00:30:12 +0100 Subject: [PATCH 341/433] Fix data race in ProfileEvents::Counters::setParent(). --- src/Common/ProfileEvents.cpp | 2 +- src/Common/ProfileEvents.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 1534aa9bd4e..daee30b201e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -970,7 +970,7 @@ void Counters::resetCounters() void Counters::reset() { - parent = nullptr; + setParent(nullptr); resetCounters(); } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 94551c23ac0..b88f3a9b45c 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -60,7 +60,7 @@ namespace ProfileEvents Counter * counters = nullptr; std::unique_ptr counters_holder; /// Used to propagate increments - Counters * parent = nullptr; + std::atomic parent = {}; bool trace_profile_events = false; public: @@ -114,13 +114,13 @@ namespace ProfileEvents /// Get parent (thread unsafe) Counters * getParent() { - return parent; + return parent.load(std::memory_order_relaxed); } /// Set parent (thread unsafe) void setParent(Counters * parent_) { - parent = parent_; + parent.store(parent_, std::memory_order_relaxed); } void setTraceProfileEvents(bool value) From 1166e93447c9ab70aa97314a1f6cd9ab198a8dd9 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Wed, 27 Nov 2024 00:56:03 +0100 Subject: [PATCH 342/433] SettingsChangesHistory: disable allow_experimental_shared_set_join --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5cc5a54639a..682d5845f73 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -117,7 +117,7 @@ static std::initializer_list Date: Wed, 27 Nov 2024 01:46:06 +0100 Subject: [PATCH 343/433] Fix compilation. --- src/Common/ProfileEvents.cpp | 9 +++++++++ src/Common/ProfileEvents.h | 2 ++ 2 files changed, 11 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index daee30b201e..f5ec3dfd202 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -959,6 +959,15 @@ Counters::Counters(VariableContext level_, Counters * parent_) counters = counters_holder.get(); } +Counters::Counters(Counters && src) noexcept + : counters(std::exchange(src.counters, nullptr)) + , counters_holder(std::move(src.counters_holder)) + , parent(src.parent.exchange(nullptr)) + , trace_profile_events(src.trace_profile_events) + , level(src.level) +{ +} + void Counters::resetCounters() { if (counters) diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index b88f3a9b45c..012fe4ffdfa 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -74,6 +74,8 @@ namespace ProfileEvents explicit Counters(Counter * allocated_counters) noexcept : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} + Counters(Counters && src) noexcept; + Counter & operator[] (Event event) { return counters[event]; From b6725d33d115bb3b5992388b6c67600389ae0843 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Tue, 26 Nov 2024 16:57:49 -0800 Subject: [PATCH 344/433] [Docs] Specify that Replicated is default engine for cloud --- docs/en/engines/database-engines/atomic.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/atomic.md b/docs/en/engines/database-engines/atomic.md index 0224e1aba21..e2d6025f98f 100644 --- a/docs/en/engines/database-engines/atomic.md +++ b/docs/en/engines/database-engines/atomic.md @@ -6,7 +6,7 @@ sidebar_position: 10 # Atomic -It supports non-blocking [DROP TABLE](#drop-detach-table) and [RENAME TABLE](#rename-table) queries and atomic [EXCHANGE TABLES](#exchange-tables) queries. `Atomic` database engine is used by default. +It supports non-blocking [DROP TABLE](#drop-detach-table) and [RENAME TABLE](#rename-table) queries and atomic [EXCHANGE TABLES](#exchange-tables) queries. `Atomic` database engine is used by default. Note that on ClickHouse Cloud, the `Replicated` database engine is used by default. ## Creating a Database {#creating-a-database} From 344000efcfb448a7fa8642d20f216fa11a782d0b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 27 Nov 2024 14:38:32 +0800 Subject: [PATCH 345/433] add functionsconversions --- src/Functions/FunctionsConversion.h | 141 ++++++++++++++++++++++++++++ 1 file changed, 141 insertions(+) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 2a0f349e25a..3455dd9b9cc 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -73,6 +73,10 @@ #include #include +#if USE_EMBEDDED_COMPILER +# include "DataTypes/Native.h" +#endif + namespace DB { @@ -117,6 +121,43 @@ namespace ErrorCodes namespace detail { +#if USE_EMBEDDED_COMPILER +bool castType(const IDataType * type, auto && f) +{ + using Types = TypeList< + DataTypeUInt8, + DataTypeUInt16, + DataTypeUInt32, + DataTypeUInt64, + DataTypeUInt128, + DataTypeUInt256, + DataTypeInt8, + DataTypeInt16, + DataTypeInt32, + DataTypeInt64, + DataTypeInt128, + DataTypeInt256, + DataTypeFloat32, + DataTypeFloat64, + DataTypeDecimal32, + DataTypeDecimal64, + DataTypeDecimal128, + DataTypeDecimal256, + DataTypeDate, + DataTypeDateTime, + DataTypeFixedString, + DataTypeString, + DataTypeInterval>; + return castTypeToEither(Types{}, type, std::forward(f)); +} + +template +bool castBothTypes(const IDataType * left, const IDataType * right, F && f) +{ + return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); }); +} +#endif + /** Type conversion functions. * toType - conversion in "natural way"; */ @@ -2208,6 +2249,52 @@ public: } } +#if USE_EMBEDDED_COMPILER + bool isCompilableImpl(const DataTypes & types, const DataTypePtr & result_type) const override + { + if (types.size() != 1) + return false; + + if (!canBeNativeType(types[0]) || !canBeNativeType(result_type)) + return false; + + return castBothTypes(types[0].get(), result_type.get(), [](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + return true; + + return false; + }); + } + + llvm::Value * + compileImpl(llvm::IRBuilderBase & builder, const ValuesWithType & arguments, const DataTypePtr & result_type) const override + { + llvm::Value * result = nullptr; + castBothTypes( + arguments[0].type.get(), + result_type.get(), + [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + { + result = nativeCast(builder, arguments[0], result_type); + return true; + } + + return false; + }); + + return result; + } +#endif + bool hasInformationAboutMonotonicity() const override { return Monotonic::has(); @@ -3332,6 +3419,60 @@ public: return monotonicity_for_range(type, left, right); } +#if USE_EMBEDDED_COMPILER + bool isCompilable() const override + { + if (getName() != "CAST" || argument_types.size() != 2) + return false; + + const auto & from_type = argument_types[0]; + const auto & to_type = return_type; + auto denull_from_type = removeNullable(from_type); + auto denull_to_type = removeNullable(to_type); + if (!canBeNativeType(denull_from_type) || !canBeNativeType(denull_to_type)) + return false; + + return castBothTypes(denull_from_type.get(), denull_to_type.get(), [](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + return true; + + return false; + }); + } + + llvm::Value * compile(llvm::IRBuilderBase & builder, const ValuesWithType & arguments) const override + { + llvm::Value * result = nullptr; + + const auto & from_type = arguments[0].type; + const auto & to_type = return_type; + auto denull_from_type = removeNullable(from_type); + auto denull_to_type = removeNullable(to_type); + castBothTypes( + denull_from_type.get(), + denull_to_type.get(), + [&](const auto & left, const auto & right) + { + using LeftDataType = std::decay_t; + using RightDataType = std::decay_t; + + if constexpr (IsDataTypeNativeNumber && IsDataTypeNativeNumber) + { + result = nativeCast(builder, arguments[0], return_type); + return true; + } + + return false; + }); + + return result; + } +#endif + private: const char * cast_name; MonotonicityForRange monotonicity_for_range; From b069aa30067a6af47d75f762a12a970ab2b3a9c6 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 27 Nov 2024 14:40:13 +0800 Subject: [PATCH 346/433] change as request --- src/Functions/FunctionsComparison.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 79107731d45..6a94785c20c 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1391,7 +1391,6 @@ public: }); } - // bool isCompilableImpl(const DataTypes &, const DataTypePtr &) const override { return false; } bool isCompilableImpl(const DataTypes & arguments, const DataTypePtr & result_type) const override { if (2 != arguments.size()) From 0a6965537a8313ecdb75a3b87c9bee025768d8a9 Mon Sep 17 00:00:00 2001 From: Zaynulla <36727185+Zaynulla@users.noreply.github.com> Date: Wed, 27 Nov 2024 10:32:01 +0300 Subject: [PATCH 347/433] window func doc teamMax output header fixed --- 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 3f97bad558d..3350334d0a5 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -174,7 +174,7 @@ FROM salaries; ``` ```text -┌─player──────────┬─salary─┬─team──────────────────────┬─teamAvg─┬───diff─┐ +┌─player──────────┬─salary─┬─team──────────────────────┬─teamMax─┬───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 │ From 09aa96370edaa5795144aa4548d343542fb03d6d Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 27 Nov 2024 10:16:45 +0100 Subject: [PATCH 348/433] CI: Remove functional tests with hdfs --- docker/test/stateless/Dockerfile | 4 ---- .../docker_scripts/setup_hdfs_minicluster.sh | 21 ------------------- tests/docker_scripts/stateless_runner.sh | 2 -- 3 files changed, 27 deletions(-) delete mode 100755 tests/docker_scripts/setup_hdfs_minicluster.sh diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 69f81b35a95..7dd258fe907 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -78,10 +78,6 @@ RUN arch=${TARGETARCH:-amd64} \ && curl -L "https://dl.min.io/client/mc/release/linux-${arch}/archive/mc.RELEASE.${MINIO_CLIENT_VERSION}" -o ./mc \ && chmod +x ./mc ./minio -RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ - && tar -xvf hadoop-3.3.1.tar.gz \ - && rm -rf hadoop-3.3.1.tar.gz - ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 diff --git a/tests/docker_scripts/setup_hdfs_minicluster.sh b/tests/docker_scripts/setup_hdfs_minicluster.sh deleted file mode 100755 index 622270ba5d5..00000000000 --- a/tests/docker_scripts/setup_hdfs_minicluster.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/bin/bash -# shellcheck disable=SC2024 - -set -e -x -a -u - -ls -lha - -cd /hadoop-3.3.1 - -export JAVA_HOME=/usr -mkdir -p target/test/data -chown clickhouse ./target/test/data -sudo -E -u clickhouse bin/mapred minicluster -format -nomr -nnport 12222 >> /test_output/hdfs_minicluster.log 2>&1 & - -while ! nc -z localhost 12222; do - sleep 1 -done - -lsof -i :12222 - -sleep 5 diff --git a/tests/docker_scripts/stateless_runner.sh b/tests/docker_scripts/stateless_runner.sh index 3a06da65a4b..2e43a2e3438 100755 --- a/tests/docker_scripts/stateless_runner.sh +++ b/tests/docker_scripts/stateless_runner.sh @@ -57,8 +57,6 @@ source /repo/tests/docker_scripts/utils.lib /repo/tests/docker_scripts/setup_minio.sh stateless -/repo/tests/docker_scripts/setup_hdfs_minicluster.sh - config_logs_export_cluster /etc/clickhouse-server/config.d/system_logs_export.yaml if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; then From bae2c068e99394034d32aa4d820059976008d602 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 27 Nov 2024 10:23:28 +0100 Subject: [PATCH 349/433] fix sed in test --- .../0_stateless/01301_aggregate_state_exception_memory_leak.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index ceb7b60be0f..2a585b4c015 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -16,5 +16,5 @@ for _ in {1..1000}; do if [[ $elapsed -gt 30 ]]; then break fi -done 2>&1 | grep -o -P 'Query memory limit exceeded' | sed -r -e 's/(.*):([a-Z ]*)([mM]emory limit exceeded)(.*)/\2\3/' | uniq +done 2>&1 | grep -o 'Query memory limit exceeded' | head -n1 echo 'Ok' From 0e82fb82fe3bf63eae46fc7796e07b19dfa6d0b5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 27 Nov 2024 17:28:06 +0800 Subject: [PATCH 350/433] fix building --- src/Functions/FunctionsComparison.h | 55 +++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 6a94785c20c..5a4276e0d75 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -635,6 +635,61 @@ struct GenericComparisonImpl } }; + +#if USE_EMBEDDED_COMPILER + +template

setStepDescription(fmt::format("JOIN {}", expressions.join->pipelineType())); std::vector plans; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 2532dddba3c..555aaff2e06 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -41,6 +41,7 @@ namespace DB namespace Setting { extern const SettingsBool allow_experimental_join_right_table_sorting; + extern const SettingsBool allow_experimental_analyzer; extern const SettingsUInt64 cross_join_min_bytes_to_compress; extern const SettingsUInt64 cross_join_min_rows_to_compress; extern const SettingsUInt64 default_max_bytes_in_join; @@ -143,6 +144,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_memory_usage(settings[Setting::max_memory_usage]) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) + , enable_analyzer(settings[Setting::allow_experimental_analyzer]) { } @@ -161,6 +163,8 @@ void TableJoin::resetCollected() clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); + columns_from_left_table.clear(); + result_columns_from_left_table.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); @@ -203,6 +207,20 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return count; } +void TableJoin::setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns) +{ + columns_from_left_table = std::move(left_output_columns); + columns_from_joined_table = std::move(right_output_columns); +} + + +const NamesAndTypesList & TableJoin::getOutputColumns(JoinTableSide side) +{ + if (side == JoinTableSide::Left) + return result_columns_from_left_table; + return columns_added_by_join; +} + void TableJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix) { NameSet joined_columns; @@ -351,9 +369,18 @@ bool TableJoin::rightBecomeNullable(const DataTypePtr & column_type) const return forceNullableRight() && JoinCommon::canBecomeNullable(column_type); } +void TableJoin::setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side) +{ + if (side == JoinTableSide::Left) + result_columns_from_left_table.push_back(joined_column); + else + columns_added_by_join.push_back(joined_column); + +} + void TableJoin::addJoinedColumn(const NameAndTypePair & joined_column) { - columns_added_by_join.emplace_back(joined_column); + setUsedColumn(joined_column, JoinTableSide::Right); } NamesAndTypesList TableJoin::correctedColumnsAddedByJoin() const @@ -995,5 +1022,32 @@ size_t TableJoin::getMaxMemoryUsage() const return max_memory_usage; } +void TableJoin::swapSides() +{ + assertEnableEnalyzer(); + + std::swap(key_asts_left, key_asts_right); + std::swap(left_type_map, right_type_map); + for (auto & clause : clauses) + { + std::swap(clause.key_names_left, clause.key_names_right); + std::swap(clause.on_filter_condition_left, clause.on_filter_condition_right); + std::swap(clause.analyzer_left_filter_condition_column_name, clause.analyzer_right_filter_condition_column_name); + } + + std::swap(columns_from_left_table, columns_from_joined_table); + std::swap(result_columns_from_left_table, columns_added_by_join); + + if (table_join.kind == JoinKind::Left) + table_join.kind = JoinKind::Right; + else if (table_join.kind == JoinKind::Right) + table_join.kind = JoinKind::Left; +} + +void TableJoin::assertEnableEnalyzer() const +{ + if (!enable_analyzer) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TableJoin: analyzer is disabled"); +} } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index e1bae55a4ed..e0e1926fb12 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -167,6 +167,9 @@ private: ASOFJoinInequality asof_inequality = ASOFJoinInequality::GreaterOrEquals; + NamesAndTypesList columns_from_left_table; + NamesAndTypesList result_columns_from_left_table; + /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; /// Columns will be added to block by JOIN. @@ -202,6 +205,8 @@ private: bool is_join_with_constant = false; + bool enable_analyzer = false; + Names requiredJoinedNames() const; /// Create converting actions and change key column names if required @@ -266,6 +271,8 @@ public: VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } + bool enableEnalyzer() const { return enable_analyzer; } + void assertEnableEnalyzer() const; ActionsDAG createJoinedBlockActions(ContextPtr context) const; @@ -282,6 +289,7 @@ public: } bool allowParallelHashJoin() const; + void swapSides(); bool joinUseNulls() const { return join_use_nulls; } @@ -372,6 +380,9 @@ public: bool leftBecomeNullable(const DataTypePtr & column_type) const; bool rightBecomeNullable(const DataTypePtr & column_type) const; void addJoinedColumn(const NameAndTypePair & joined_column); + + void setUsedColumn(const NameAndTypePair & joined_column, JoinTableSide side); + void setColumnsAddedByJoin(const NamesAndTypesList & columns_added_by_join_value) { columns_added_by_join = columns_added_by_join_value; @@ -397,11 +408,17 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix) + void setColumnsFromJoinedTable(NamesAndTypesList columns_from_joined_table_value, const NameSet & left_table_columns, const String & right_table_prefix, const NamesAndTypesList & columns_from_left_table_) { columns_from_joined_table = std::move(columns_from_joined_table_value); deduplicateAndQualifyColumnNames(left_table_columns, right_table_prefix); + result_columns_from_left_table = columns_from_left_table_; + columns_from_left_table = columns_from_left_table_; } + + void setInputColumns(NamesAndTypesList left_output_columns, NamesAndTypesList right_output_columns); + const NamesAndTypesList & getOutputColumns(JoinTableSide side); + const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ea08fd92339..28e11166762 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1353,12 +1353,15 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (tables_with_columns.size() > 1) { + auto columns_from_left_table = tables_with_columns[0].columns; const auto & right_table = tables_with_columns[1]; auto columns_from_joined_table = right_table.columns; /// query can use materialized or aliased columns from right joined table, /// we want to request it for right table columns_from_joined_table.insert(columns_from_joined_table.end(), right_table.hidden_columns.begin(), right_table.hidden_columns.end()); - result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); + columns_from_left_table.insert(columns_from_left_table.end(), tables_with_columns[0].hidden_columns.begin(), tables_with_columns[0].hidden_columns.end()); + result.analyzed_join->setColumnsFromJoinedTable( + std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix(), columns_from_left_table); } translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); diff --git a/src/Parsers/CreateQueryUUIDs.cpp b/src/Parsers/CreateQueryUUIDs.cpp index c788cc7a025..70848440a0e 100644 --- a/src/Parsers/CreateQueryUUIDs.cpp +++ b/src/Parsers/CreateQueryUUIDs.cpp @@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// Thus it's not safe for example to replace /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with - /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b" + /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID '123e4567-e89b-12d3-a456-426614174000' AS SELECT a FROM b" /// This replacement is safe only for CREATE queries when inner target tables don't exist yet. if (!query.attach) { diff --git a/src/Planner/CollectColumnIdentifiers.cpp b/src/Planner/CollectColumnIdentifiers.cpp index 95f1c7d53d8..dd5bdd4d141 100644 --- a/src/Planner/CollectColumnIdentifiers.cpp +++ b/src/Planner/CollectColumnIdentifiers.cpp @@ -2,6 +2,7 @@ #include #include +#include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5c153f6db39..a1ce455f266 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,6 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; + extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection; } namespace ErrorCodes @@ -1241,6 +1242,55 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP plan_to_add_cast.addStep(std::move(cast_join_columns_step)); } +std::optional createStepToDropColumns( + const Block & header, + const ColumnIdentifierSet & outer_scope_columns, + const PlannerContextPtr & planner_context) +{ + ActionsDAG drop_unused_columns_after_join_actions_dag(header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; + std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; + std::optional first_skipped_column_node_index; + + auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); + size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); + + const auto & global_planner_context = planner_context->getGlobalPlannerContext(); + + for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + { + const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; + + if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) + || !global_planner_context->hasColumnIdentifier(output->result_name)) + continue; + + if (!outer_scope_columns.contains(output->result_name)) + { + if (!first_skipped_column_node_index) + first_skipped_column_node_index = i; + continue; + } + + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); + drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); + } + + if (!first_skipped_column_node_index) + return {}; + + /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. + * + * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; + */ + if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) + drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); + + drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); + + return drop_unused_columns_after_join_actions_dag; +} + JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, @@ -1513,21 +1563,37 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } const Block & left_header = left_plan.getCurrentHeader(); - auto left_table_names = left_header.getNames(); - NameSet left_table_names_set(left_table_names.begin(), left_table_names.end()); + const Block & right_header = right_plan.getCurrentHeader(); - auto columns_from_joined_table = right_plan.getCurrentHeader().getNamesAndTypesList(); - table_join->setColumnsFromJoinedTable(columns_from_joined_table, left_table_names_set, ""); + auto columns_from_left_table = left_header.getNamesAndTypesList(); + auto columns_from_right_table = right_header.getNamesAndTypesList(); - for (auto & column_from_joined_table : columns_from_joined_table) + table_join->setInputColumns(columns_from_left_table, columns_from_right_table); + + for (auto & column_from_joined_table : columns_from_left_table) { - /// Add columns from joined table only if they are presented in outer scope, otherwise they can be dropped + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && outer_scope_columns.contains(column_from_joined_table.name)) - table_join->addJoinedColumn(column_from_joined_table); + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + } + + for (auto & column_from_joined_table : columns_from_right_table) + { + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + outer_scope_columns.contains(column_from_joined_table.name)) + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); + } + + if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) + { + if (!columns_from_left_table.empty()) + table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left); + else if (!columns_from_right_table.empty()) + table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right); } - const Block & right_header = right_plan.getCurrentHeader(); auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); auto result_plan = QueryPlan(); @@ -1615,13 +1681,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } auto join_pipeline_type = join_algorithm->pipelineType(); + + ColumnIdentifierSet outer_scope_columns_nonempty; + if (outer_scope_columns.empty()) + { + if (left_header.columns() > 1) + outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); + else if (right_header.columns() > 1) + outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); + } + auto join_step = std::make_unique( left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), std::move(join_algorithm), settings[Setting::max_block_size], settings[Setting::max_threads], - false /*optimize_read_in_order*/); + outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, + false /*optimize_read_in_order*/, + true /*optimize_skip_unused_shards*/); + join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); @@ -1632,47 +1711,18 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); - ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; - std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; - std::optional first_skipped_column_node_index; - - auto & drop_unused_columns_after_join_actions_dag_outputs = drop_unused_columns_after_join_actions_dag.getOutputs(); - size_t drop_unused_columns_after_join_actions_dag_outputs_size = drop_unused_columns_after_join_actions_dag_outputs.size(); - - for (size_t i = 0; i < drop_unused_columns_after_join_actions_dag_outputs_size; ++i) + const auto & header_after_join = result_plan.getCurrentHeader(); + if (header_after_join.columns() > outer_scope_columns.size()) { - const auto & output = drop_unused_columns_after_join_actions_dag_outputs[i]; - - const auto & global_planner_context = planner_context->getGlobalPlannerContext(); - if (drop_unused_columns_after_join_actions_dag_updated_outputs_names.contains(output->result_name) - || !global_planner_context->hasColumnIdentifier(output->result_name)) - continue; - - if (!outer_scope_columns.contains(output->result_name)) + auto drop_unused_columns_after_join_actions_dag = createStepToDropColumns(header_after_join, outer_scope_columns, planner_context); + if (drop_unused_columns_after_join_actions_dag) { - if (!first_skipped_column_node_index) - first_skipped_column_node_index = i; - continue; + auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(*drop_unused_columns_after_join_actions_dag)); + drop_unused_columns_after_join_transform_step->setStepDescription("Drop unused columns after JOIN"); + result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); } - - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(output); - drop_unused_columns_after_join_actions_dag_updated_outputs_names.insert(output->result_name); } - /** It is expected that JOIN TREE query plan will contain at least 1 column, even if there are no columns in outer scope. - * - * Example: SELECT count() FROM test_table_1 AS t1, test_table_2 AS t2; - */ - if (drop_unused_columns_after_join_actions_dag_updated_outputs.empty() && first_skipped_column_node_index) - drop_unused_columns_after_join_actions_dag_updated_outputs.push_back(drop_unused_columns_after_join_actions_dag_outputs[*first_skipped_column_node_index]); - - drop_unused_columns_after_join_actions_dag_outputs = std::move(drop_unused_columns_after_join_actions_dag_updated_outputs); - - auto drop_unused_columns_after_join_transform_step = std::make_unique(result_plan.getCurrentHeader(), std::move(drop_unused_columns_after_join_actions_dag)); - drop_unused_columns_after_join_transform_step->setStepDescription("DROP unused columns after JOIN"); - result_plan.addStep(std::move(drop_unused_columns_after_join_transform_step)); - for (const auto & right_join_tree_query_plan_row_policy : right_join_tree_query_plan.used_row_policies) left_join_tree_query_plan.used_row_policies.insert(right_join_tree_query_plan_row_policy); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 018b52a5c68..7ade437822e 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -36,6 +37,37 @@ std::vector> describeJoinActions(const JoinPtr & join) return description; } +std::vector getPermutationForBlock( + const Block & block, + const Block & lhs_block, + const Block & rhs_block, + const NameSet & name_filter) +{ + std::vector permutation; + permutation.reserve(block.columns()); + Block::NameMap name_map = block.getNamesToIndexesMap(); + + bool is_trivial = true; + for (const auto & other_block : {lhs_block, rhs_block}) + { + for (const auto & col : other_block) + { + if (!name_filter.contains(col.name)) + continue; + if (auto it = name_map.find(col.name); it != name_map.end()) + { + is_trivial = is_trivial && it->second == permutation.size(); + permutation.push_back(it->second); + } + } + } + + if (is_trivial && permutation.size() == block.columns()) + return {}; + + return permutation; +} + } JoinStep::JoinStep( @@ -44,8 +76,15 @@ JoinStep::JoinStep( JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_) - : join(std::move(join_)), max_block_size(max_block_size_), max_streams(max_streams_), keep_left_read_in_order(keep_left_read_in_order_) + NameSet required_output_, + bool keep_left_read_in_order_, + bool use_new_analyzer_) + : join(std::move(join_)) + , max_block_size(max_block_size_) + , max_streams(max_streams_) + , required_output(std::move(required_output_)) + , keep_left_read_in_order(keep_left_read_in_order_) + , use_new_analyzer(use_new_analyzer_) { updateInputHeaders({left_header_, right_header_}); } @@ -55,23 +94,43 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); + Block lhs_header = pipelines[0]->getHeader(); + Block rhs_header = pipelines[1]->getHeader(); + + if (swap_streams) + std::swap(pipelines[0], pipelines[1]); + if (join->pipelineType() == JoinPipelineType::YShaped) { auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( - std::move(pipelines[0]), std::move(pipelines[1]), join, *output_header, max_block_size, &processors); + std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); joined_pipeline->resize(max_streams); return joined_pipeline; } - return QueryPipelineBuilder::joinPipelinesRightLeft( + auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), std::move(pipelines[1]), join, - *output_header, + join_algorithm_header, max_block_size, max_streams, keep_left_read_in_order, &processors); + + if (!use_new_analyzer) + return pipeline; + + auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output); + if (!column_permutation.empty()) + { + pipeline->addSimpleTransform([&column_permutation](const Block & header) + { + return std::make_shared(header, column_permutation); + }); + } + + return pipeline; } bool JoinStep::allowPushDownToRight() const @@ -90,17 +149,49 @@ void JoinStep::describeActions(FormatSettings & settings) const for (const auto & [name, value] : describeJoinActions(join)) settings.out << prefix << name << ": " << value << '\n'; + if (swap_streams) + settings.out << prefix << "Swapped: true\n"; } void JoinStep::describeActions(JSONBuilder::JSONMap & map) const { for (const auto & [name, value] : describeJoinActions(join)) map.add(name, value); + if (swap_streams) + map.add("Swapped", true); +} + +void JoinStep::setJoin(JoinPtr join_, bool swap_streams_) +{ + join_algorithm_header.clear(); + swap_streams = swap_streams_; + join = std::move(join_); + updateOutputHeader(); } void JoinStep::updateOutputHeader() { - output_header = JoiningTransform::transformHeader(input_headers.front(), join); + if (join_algorithm_header) + return; + + const auto & header = swap_streams ? input_headers[1] : input_headers[0]; + + Block result_header = JoiningTransform::transformHeader(header, join); + join_algorithm_header = result_header; + + if (!use_new_analyzer) + { + if (swap_streams) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot swap streams without new analyzer"); + output_header = result_header; + return; + } + + auto column_permutation = getPermutationForBlock(result_header, input_headers[0], input_headers[1], required_output); + if (!column_permutation.empty()) + result_header = ColumnPermuteTransform::permute(result_header, column_permutation); + + output_header = result_header; } static ITransformingStep::Traits getStorageJoinTraits() diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 2793784d633..1eca42c62cf 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -19,7 +20,9 @@ public: JoinPtr join_, size_t max_block_size_, size_t max_streams_, - bool keep_left_read_in_order_); + NameSet required_output_, + bool keep_left_read_in_order_, + bool use_new_analyzer_); String getName() const override { return "Join"; } @@ -31,16 +34,26 @@ public: void describeActions(FormatSettings & settings) const override; const JoinPtr & getJoin() const { return join; } - void setJoin(JoinPtr join_) { join = std::move(join_); } + void setJoin(JoinPtr join_, bool swap_streams_ = false); bool allowPushDownToRight() const; + JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right; + private: void updateOutputHeader() override; + /// Header that expected to be returned from IJoin + Block join_algorithm_header; + JoinPtr join; size_t max_block_size; size_t max_streams; + + const NameSet required_output; + std::set columns_to_remove; bool keep_left_read_in_order; + bool use_new_analyzer = false; + bool swap_streams = false; }; /// Special step for the case when Join is already filled. diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 751d5182dc3..c1c4d1e1635 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -113,6 +113,7 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &); void optimizeDistinctInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); /// A separate tree traverse to apply sorting properties after *InOrder optimizations. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp new file mode 100644 index 00000000000..c0b31864eac --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +static std::optional estimateReadRowsCount(QueryPlan::Node & node) +{ + IQueryPlanStep * step = node.step.get(); + if (const auto * reading = typeid_cast(step)) + { + if (auto analyzed_result = reading->getAnalyzedResult()) + return analyzed_result->selected_rows; + if (auto analyzed_result = reading->selectRangesToRead()) + return analyzed_result->selected_rows; + return {}; + } + + if (const auto * reading = typeid_cast(step)) + return reading->getStorage()->totalRows(Settings{}); + + if (node.children.size() != 1) + return {}; + + if (typeid_cast(step) || typeid_cast(step)) + return estimateReadRowsCount(*node.children.front()); + + return {}; +} + +void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) +{ + auto * join_step = typeid_cast(node.step.get()); + if (!join_step || node.children.size() != 2) + return; + + const auto & join = join_step->getJoin(); + if (join->pipelineType() != JoinPipelineType::FillRightFirst || !join->isCloneSupported()) + return; + + const auto & table_join = join->getTableJoin(); + + /// Algorithms other than HashJoin may not support OUTER JOINs + if (table_join.kind() != JoinKind::Inner && !typeid_cast(join.get())) + return; + + /// fixme: USING clause handled specially in join algorithm, so swap breaks it + /// fixme: Swapping for SEMI and ANTI joins should be alright, need to try to enable it and test + if (table_join.hasUsing() || table_join.strictness() != JoinStrictness::All) + return; + + bool need_swap = false; + if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto) + { + auto lhs_extimation = estimateReadRowsCount(*node.children[0]); + auto rhs_extimation = estimateReadRowsCount(*node.children[1]); + LOG_TRACE(getLogger("optimizeJoin"), "Left table estimation: {}, right table estimation: {}", + lhs_extimation.transform(toString).value_or("unknown"), + rhs_extimation.transform(toString).value_or("unknown")); + + if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) + need_swap = true; + } + else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left) + { + need_swap = true; + } + + if (!need_swap) + return; + + const auto & headers = join_step->getInputHeaders(); + if (headers.size() != 2) + return; + + const auto & left_stream_input_header = headers.front(); + const auto & right_stream_input_header = headers.back(); + + auto updated_table_join = std::make_shared(table_join); + updated_table_join->swapSides(); + auto updated_join = join->clone(updated_table_join, right_stream_input_header, left_stream_input_header); + join_step->setJoin(std::move(updated_join), /* swap_streams= */ true); +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 03418c752d4..c034ca79181 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -227,6 +227,9 @@ void addStepsToBuildSets(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::No /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); + if (frame.next_child == 0) + optimizeJoin(*frame.node, nodes); + /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index 238c1a3aad0..a9c2d2df2c4 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -35,6 +35,8 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + const StoragePtr & getStorage() const { return storage; } + private: static constexpr auto name = "ReadFromMemoryStorage"; diff --git a/src/Processors/Transforms/ColumnPermuteTransform.cpp b/src/Processors/Transforms/ColumnPermuteTransform.cpp new file mode 100644 index 00000000000..f371689814c --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +namespace +{ + +template +void applyPermutation(std::vector & data, const std::vector & permutation) +{ + std::vector res; + res.reserve(permutation.size()); + for (size_t i : permutation) + res.push_back(data[i]); + data = std::move(res); +} + +void permuteChunk(Chunk & chunk, const std::vector & permutation) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + applyPermutation(columns, permutation); + chunk.setColumns(std::move(columns), num_rows); +} + +} + +Block ColumnPermuteTransform::permute(const Block & block, const std::vector & permutation) +{ + auto columns = block.getColumnsWithTypeAndName(); + applyPermutation(columns, permutation); + return Block(columns); +} + +ColumnPermuteTransform::ColumnPermuteTransform(const Block & header_, const std::vector & permutation_) + : ISimpleTransform(header_, permute(header_, permutation_), false) + , permutation(permutation_) +{ +} + + +void ColumnPermuteTransform::transform(Chunk & chunk) +{ + permuteChunk(chunk, permutation); +} + + +} diff --git a/src/Processors/Transforms/ColumnPermuteTransform.h b/src/Processors/Transforms/ColumnPermuteTransform.h new file mode 100644 index 00000000000..25f3a8d0825 --- /dev/null +++ b/src/Processors/Transforms/ColumnPermuteTransform.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class ColumnPermuteTransform : public ISimpleTransform +{ +public: + ColumnPermuteTransform(const Block & header_, const std::vector & permutation_); + + String getName() const override { return "ColumnPermuteTransform"; } + + void transform(Chunk & chunk) override; + + static Block permute(const Block & block, const std::vector & permutation); + +private: + Names column_names; + std::vector permutation; +}; + + +} diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f2fb6327129..187f4bf6728 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -19,6 +19,7 @@ Block JoiningTransform::transformHeader(Block header, const JoinPtr & join) join->initialize(header); ExtraBlockPtr tmp; join->joinBlock(header, tmp); + materializeBlockInplace(header); LOG_TEST(getLogger("JoiningTransform"), "After join block: '{}'", header.dumpStructure()); return header; } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 9c035b7cc35..f4c3b368632 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -789,6 +789,7 @@ def get_localzone(): return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) +# Refer to `tests/integration/helpers/random_settings.py` for integration test random settings class SettingsRandomizer: settings = { "max_insert_threads": lambda: ( @@ -919,6 +920,9 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), + "query_plan_join_inner_table_selection": lambda: random.choice( + ["left", "auto", "right"] + ), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b24593602ec..e2237363131 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -67,6 +67,7 @@ DEFAULT_ENV_NAME = ".env" DEFAULT_BASE_CONFIG_DIR = os.environ.get( "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" ) +DOCKER_BASE_TAG = os.environ.get("DOCKER_BASE_TAG", "latest") SANITIZER_SIGN = "==================" @@ -503,7 +504,6 @@ class ClickHouseCluster: "CLICKHOUSE_TESTS_DOCKERD_HOST" ) self.docker_api_version = os.environ.get("DOCKER_API_VERSION") - self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") self.base_cmd = ["docker", "compose"] if custom_dockerd_host: @@ -1079,7 +1079,7 @@ class ClickHouseCluster: env_variables["keeper_binary"] = binary_path env_variables["keeper_cmd_prefix"] = keeper_cmd_prefix - env_variables["image"] = "clickhouse/integration-test:" + self.docker_base_tag + env_variables["image"] = "clickhouse/integration-test:" + DOCKER_BASE_TAG env_variables["user"] = str(os.getuid()) env_variables["keeper_fs"] = "bind" for i in range(1, 4): @@ -1676,7 +1676,7 @@ class ClickHouseCluster: ) if tag is None: - tag = self.docker_base_tag + tag = DOCKER_BASE_TAG if not env_variables: env_variables = {} self.use_keeper = use_keeper @@ -4548,7 +4548,12 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: + if ( + self.randomize_settings + and self.image == "clickhouse/integration-test" + and self.tag == DOCKER_BASE_TAG + and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR + ): # If custom main config is used, do not apply random settings to it write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index b2319561fd7..32cde54d0e7 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -5,6 +5,8 @@ def randomize_settings(): yield "max_joined_block_size_rows", random.randint(8000, 100000) if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) + if random.random() < 0.5: + yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"]) def write_random_settings_config(destination): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 51268dcf386..69057573173 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_inner_table_selection = 'right'", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/tests/queries/0_stateless/00826_cross_to_inner_join.sql index e9f9e13e2d3..5ab7a2d0626 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -15,9 +15,9 @@ INSERT INTO t2_00826 values (1,1), (1,2); INSERT INTO t2_00826 (a) values (2), (3); SELECT '--- cross ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- cross nullable ---'; -SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- cross nullable vs not nullable ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.a = t2_00826.b ORDER BY t1_00826.a; SELECT '--- cross self ---'; @@ -41,14 +41,15 @@ SELECT '--- is null or ---'; SELECT * FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b > t2_00826.a) ORDER BY t1_00826.a; SELECT '--- do not rewrite alias ---'; -SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0; +SELECT a as b FROM t1_00826 cross join t2_00826 where t1_00826.b = t2_00826.a AND b > 0 ORDER BY ALL; SELECT '--- comma ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a; +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a ORDER BY ALL; SELECT '--- comma nullable ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b; +SELECT * FROM t1_00826, t2_00826 where t1_00826.b = t2_00826.b ORDER BY ALL; SELECT '--- comma and or ---'; -SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2); +SELECT * FROM t1_00826, t2_00826 where t1_00826.a = t2_00826.a AND (t2_00826.b IS NULL OR t2_00826.b < 2) +ORDER BY ALL; SELECT '--- cross ---'; diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index c7f0c6383c2..bbb4eb12466 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -20,42 +20,42 @@ select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) left join y on (y.a = s.a and y.b = s.b) order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a as t_a from t left join s on s.a = t_a order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, s.a as s_a from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a left join y on y.b = s.b order by t.a -format PrettyCompactNoEscapes; +format PrettyCompactMonoBlock; drop table t; drop table s; diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index cdb9d253b9b..cdbb0542ffb 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,5 +1,7 @@ SET joined_subquery_requires_alias = 0; +SET query_plan_join_inner_table_selection = 'auto'; + {% for join_algorithm in ['partial_merge', 'hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.reference b/tests/queries/0_stateless/01107_join_right_table_totals.reference index daf503b776d..aa569ff9331 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.reference +++ b/tests/queries/0_stateless/01107_join_right_table_totals.reference @@ -18,28 +18,35 @@ 0 0 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 1 1 0 0 +- 1 1 0 0 +- 1 foo 1 1 300 0 foo 1 0 300 +- 1 100 1970-01-01 1 100 1970-01-01 1 100 1970-01-01 1 200 1970-01-02 1 200 1970-01-02 1 100 1970-01-01 diff --git a/tests/queries/0_stateless/01107_join_right_table_totals.sql b/tests/queries/0_stateless/01107_join_right_table_totals.sql index ad8954d5d70..7e549282489 100644 --- a/tests/queries/0_stateless/01107_join_right_table_totals.sql +++ b/tests/queries/0_stateless/01107_join_right_table_totals.sql @@ -64,39 +64,47 @@ USING (id); INSERT INTO t VALUES (1, 100, '1970-01-01'), (1, 200, '1970-01-02'); +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l RIGHT JOIN (SELECT item_id FROM t ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t) l RIGHT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT item_id, 'foo' AS key, 1 AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id) l LEFT JOIN (SELECT item_id, sum(price_sold) AS val FROM t GROUP BY item_id WITH TOTALS ORDER BY item_id ) r ON l.item_id = r.item_id; +SELECT '-'; SELECT * FROM (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date) l LEFT JOIN (SELECT * FROM t GROUP BY item_id, price_sold, date WITH TOTALS ORDER BY item_id, price_sold, date ) r -ON l.item_id = r.item_id; +ON l.item_id = r.item_id +ORDER BY ALL; DROP TABLE t; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 19018a610b7..229ac6eae09 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -26,7 +26,7 @@ Expression ((Projection + Before ORDER BY)) Parts: 1/1 Granules: 1/1 Expression ((Project names + Projection)) - Filter ((WHERE + DROP unused columns after JOIN)) + Filter (WHERE) Join (JOIN FillRightFirst) Expression ReadFromMergeTree (default.t1) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c2d85cefb18..c13722f431a 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -75,7 +75,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -89,10 +89,10 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; {% endfor -%} diff --git a/tests/queries/0_stateless/02000_join_on_const.reference b/tests/queries/0_stateless/02000_join_on_const.reference index 3bd1633ce32..f8e46a2b976 100644 --- a/tests/queries/0_stateless/02000_join_on_const.reference +++ b/tests/queries/0_stateless/02000_join_on_const.reference @@ -33,23 +33,23 @@ 2 2 2 2 -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 2 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -2 2 +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +2 2 +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 2 0 3 -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; 1 0 2 0 -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; 0 2 0 3 -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; 1 0 2 0 0 2 @@ -59,11 +59,11 @@ SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 1 ('',0) SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; 0 ('b',256) -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 0 ('b',256) SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; 1 ('',0) 2 4 2 Nullable(UInt64) UInt8 diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index da70973ed87..33638edafa5 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -73,20 +73,20 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 0; -- SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 SETTINGS enable_analyzer = 1; -- { echoOn } -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 1 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 LEFT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 1 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 RIGHT JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2 SETTINGS enable_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 ON t1.id = t2.id AND 1 = 2 ORDER BY 2, 1 SETTINGS enable_analyzer = 1; SELECT * FROM (SELECT 1 as a) as t1 INNER JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 LEFT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; SELECT * FROM (SELECT 1 as a) as t1 RIGHT JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 FULL JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; SELECT * FROM (SELECT 1 as a) as t1 SEMI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; -SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL; +SELECT * FROM (SELECT 1 as a) as t1 ANTI JOIN ( SELECT ('b', 256) as b ) AS t2 ON NULL ORDER BY 2; -- { echoOff } diff --git a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 index 1726bcb7062..83548e087bd 100644 --- a/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 +++ b/tests/queries/0_stateless/02001_join_on_const_bs_long.sql.j2 @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (id Int) ENGINE = MergeTree ORDER BY id; +CREATE TABLE t1 (id Int) ENGINE = TinyLog; +CREATE TABLE t2 (id Int) ENGINE = TinyLog; INSERT INTO t1 VALUES (1), (2); INSERT INTO t2 SELECT number + 5 AS x FROM (SELECT * FROM system.numbers LIMIT 1111); diff --git a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql index abc2ee41402..c3c84ebaded 100644 --- a/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql +++ b/tests/queries/0_stateless/02245_join_with_nullable_lowcardinality_crash.sql @@ -12,8 +12,9 @@ CREATE TABLE without_nullable insert into with_nullable values(0,'f'),(0,'usa'); insert into without_nullable values(0,'usa'),(0,'us2a'); -select if(t0.country is null ,t2.country,t0.country) "country" -from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country; +select if(t0.country is null ,t2.country,t0.country) "country" +from without_nullable t0 right outer join with_nullable t2 on t0.country=t2.country +ORDER BY 1 DESC; drop table with_nullable; drop table without_nullable; diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 2cca853fd67..85abc8fa381 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -48,7 +48,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2 v1, vec2 v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2f VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -61,7 +62,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2f v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; INSERT INTO vec2d VALUES (1, [100, 200, 0]), (2, [888, 777, 666]), (3, range(1, 35, 1)), (4, range(3, 37, 1)), (5, range(1, 135, 1)), (6, range(3, 137, 1)); SELECT @@ -74,7 +76,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2d v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT v1.id, @@ -86,7 +89,8 @@ SELECT L2SquaredDistance(v1.v, v2.v), cosineDistance(v1.v, v2.v) FROM vec2f v1, vec2d v2 -WHERE length(v1.v) == length(v2.v); +WHERE length(v1.v) == length(v2.v) +ORDER BY ALL; SELECT L1Distance([0, 0], [1]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT L2Distance([1, 2], (3,4)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 365725f8ffe..90aab0a0eb2 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -148,7 +148,6 @@ Header: key String value String Join Header: __table1.key String - __table3.key String __table3.value String Sorting Header: __table1.key String diff --git a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql index f0ecbf64e58..8b5c6846bd0 100644 --- a/tests/queries/0_stateless/02461_join_lc_issue_42380.sql +++ b/tests/queries/0_stateless/02461_join_lc_issue_42380.sql @@ -9,4 +9,5 @@ CREATE TABLE t2__fuzz_47 (id LowCardinality(Int16)) ENGINE = MergeTree() ORDER B INSERT INTO t1__fuzz_13 VALUES (1); INSERT INTO t2__fuzz_47 VALUES (1); -SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2; +SELECT * FROM t1__fuzz_13 FULL OUTER JOIN t2__fuzz_47 ON 1 = 2 +ORDER BY ALL; diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index 3c68d14fdf2..c9bf36f88ea 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -117,7 +117,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers @@ -161,7 +161,7 @@ ORDER BY t1.number, t2.number -- explain Expression (Project names) Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) + Expression ((Before ORDER BY + Projection)) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index 867ae394c1f..baa2be9dfdb 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -79,7 +79,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) Distinct (Preliminary DISTINCT) - Expression ((Projection + DROP unused columns after JOIN)) + Expression (Projection) Join (JOIN FillRightFirst) Expression ((Change column names to column identifiers + Project names)) Distinct (DISTINCT) @@ -244,7 +244,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -280,7 +280,7 @@ Expression (Project names) Sorting (Sorting for ORDER BY) Expression ((Before ORDER BY + Projection)) Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -315,7 +315,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -348,7 +348,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Rollup Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -386,7 +386,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -419,7 +419,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) Cube Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -457,7 +457,7 @@ Expression (Project names) Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers @@ -491,7 +491,7 @@ Expression ((Project names + (Projection + (Change column names to column identi Expression ((Before ORDER BY + Projection)) TotalsHaving Aggregating - Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) + Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + Projection)))) Join (JOIN FillRightFirst) Expression (Change column names to column identifiers) ReadFromSystemNumbers diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 2c62e278050..bbfdf1ad5f4 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -8,24 +8,21 @@ Header: count() UInt64 Aggregating Header: __table1.a2 String count() UInt64 - Expression ((Before GROUP BY + DROP unused columns after JOIN)) + Expression (Before GROUP BY) Header: __table1.a2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table3.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table3.c1 UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String - __table2.b1 UInt64 __table3.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + Expression (JOIN actions) Header: __table1.a2 String __table2.b1 UInt64 Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String + Header: __table1.a2 String __table2.b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 @@ -48,39 +45,32 @@ Header: count() UInt64 EXPLAIN PLAN header = 1 SELECT a.a2, d.d2 FROM a JOIN b USING (k) JOIN c USING (k) JOIN d USING (k) ; -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: a2 String d2 String Join (JOIN FillRightFirst) Header: __table1.a2 String - __table1.k UInt64 __table4.d2 String - Expression (DROP unused columns after JOIN) + Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 Join (JOIN FillRightFirst) Header: __table1.a2 String __table1.k UInt64 - Expression (DROP unused columns after JOIN) + Expression (Change column names to column identifiers) Header: __table1.a2 String __table1.k UInt64 - Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 - Expression (Change column names to column identifiers) - Header: __table1.a2 String - __table1.k UInt64 - ReadFromMemoryStorage - Header: a2 String - k UInt64 - Expression (Change column names to column identifiers) - Header: __table2.k UInt64 - ReadFromMemoryStorage - Header: k UInt64 + ReadFromMemoryStorage + Header: a2 String + k UInt64 Expression (Change column names to column identifiers) - Header: __table3.k UInt64 + Header: __table2.k UInt64 ReadFromMemoryStorage Header: k UInt64 + Expression (Change column names to column identifiers) + Header: __table3.k UInt64 + ReadFromMemoryStorage + Header: k UInt64 Expression (Change column names to column identifiers) Header: __table4.d2 String __table4.k UInt64 @@ -106,27 +96,24 @@ Header: bx String Header: __table1.a2 String __table2.bx String __table4.c2 String - __table4.c1 UInt64 Expression Header: __table1.a2 String __table2.bx String - __table4.c2 String __table4.c1 UInt64 + __table4.c2 String Join (JOIN FillRightFirst) Header: __table1.a2 String __table2.bx String - __table2.b1 UInt64 - __table4.c2 String __table4.c1 UInt64 - Expression ((JOIN actions + DROP unused columns after JOIN)) + __table4.c2 String + Expression (JOIN actions) Header: __table1.a2 String - __table2.bx String __table2.b1 UInt64 + __table2.bx String Join (JOIN FillRightFirst) - Header: __table1.a1 UInt64 - __table1.a2 String - __table2.bx String + Header: __table1.a2 String __table2.b1 UInt64 + __table2.bx String Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.a1 UInt64 __table1.a2 String diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index df84e2f50b2..b10bf38e495 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,6 +16,7 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; +SET query_plan_join_inner_table_selection = 'right'; -- { echoOn } diff --git a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference index 86e7e2a6a49..116c78a15e4 100644 --- a/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference +++ b/tests/queries/0_stateless/02516_join_with_totals_and_subquery_bug.reference @@ -5,7 +5,7 @@ 1 1 -1 +0 \N 100000000000000000000 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 06f4a9cfc99..bdbc019d4f8 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -1,22 +1,22 @@ -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT : 2 -> __table2.value_1 String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value_1 String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 2 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table2.value_1 String __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -50,29 +50,25 @@ Positions: 4 0 2 1 Parts: 1 Granules: 1 -- -Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Expression ((Project names + Projection)) Header: id UInt64 value_1 String rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value_1 String : 1 - INPUT :: 2 -> __table1.value_2 UInt64 : 2 + INPUT : 2 -> __table2.id UInt64 : 2 INPUT : 3 -> __table2.value_1 String : 3 - INPUT :: 4 -> __table2.value_2 UInt64 : 4 - INPUT : 5 -> __table2.id UInt64 : 5 - ALIAS __table1.id :: 0 -> id UInt64 : 6 + ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 - ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 -Positions: 6 0 3 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value_1 String - __table1.value_2 UInt64 - __table2.value_1 String - __table2.value_2 UInt64 __table2.id UInt64 + __table2.value_1 String Type: INNER Strictness: ASOF Algorithm: HashJoin diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index 1cdd3684a0b..b803ddbd911 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,6 +19,8 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); +SET query_plan_join_inner_table_selection = 'right'; + EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.reference b/tests/queries/0_stateless/02962_join_using_bug_57894.reference index 454655081df..fc6fe462205 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.reference +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.reference @@ -31,6 +31,7 @@ 8 9 \N +--- analyzer --- 0 1 2 diff --git a/tests/queries/0_stateless/02962_join_using_bug_57894.sql b/tests/queries/0_stateless/02962_join_using_bug_57894.sql index 96190241da5..e29347beb5e 100644 --- a/tests/queries/0_stateless/02962_join_using_bug_57894.sql +++ b/tests/queries/0_stateless/02962_join_using_bug_57894.sql @@ -21,6 +21,8 @@ SETTINGS join_algorithm = 'partial_merge'; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL SETTINGS join_algorithm = 'full_sorting_merge'; +SELECT '--- analyzer ---'; + SET enable_analyzer = 1; SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 80f4e309505..b7718d926c6 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -2,7 +2,9 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -10,18 +12,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -69,7 +71,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right'; +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -77,18 +81,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -136,7 +140,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -144,18 +150,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -206,7 +212,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -214,18 +222,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -273,7 +281,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -281,31 +291,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: LEFT Strictness: ALL Algorithm: HashJoin @@ -347,7 +357,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -355,31 +367,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -421,7 +433,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + (Projection + ))) Header: id UInt64 rhs.id UInt64 @@ -429,18 +443,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: RIGHT Strictness: ALL Algorithm: HashJoin @@ -488,7 +502,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -496,31 +512,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table1.id, 5_UInt8) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT :: 3 -> __table2.id UInt64 : 3 + INPUT :: 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -562,7 +578,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -570,31 +588,31 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: equals(__table2.id, 5_UInt8) (removed) Actions: INPUT :: 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 - FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 + FUNCTION equals(__table2.id : 2, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin @@ -636,7 +654,9 @@ SELECT '--'; -- EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; Expression ((Project names + Projection)) Header: id UInt64 rhs.id UInt64 @@ -644,34 +664,34 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 2 0 1 - Filter ((WHERE + DROP unused columns after JOIN)) + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 1 0 2 + Filter (WHERE) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT :: 1 -> __table1.value String : 1 - INPUT :: 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT :: 3 -> __table2.value String : 3 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5 FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6 - FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 + FUNCTION equals(__table2.id : 2, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4 FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5 Positions: 5 0 1 2 3 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: FULL Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index e1a13d1ce71..d6dcc34c796 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -22,7 +22,9 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -33,7 +35,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right'; +; SELECT '--'; @@ -44,7 +48,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6; @@ -53,7 +59,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -64,7 +72,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -75,7 +85,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -86,7 +98,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -97,7 +111,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5; +WHERE lhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -108,7 +124,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE rhs.id = 5; +WHERE rhs.id = 5 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -119,7 +137,9 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id -WHERE lhs.id = 5 AND rhs.id = 6; +WHERE lhs.id = 5 AND rhs.id = 6 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index cf070eebc38..7df38e855f6 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -52,7 +52,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_inner_table_selection = 'right' +; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] 2 3 arc 2 -> 3 false [(2,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index 7dad74893b9..d33ca7b078e 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -55,7 +55,9 @@ WITH RECURSIVE search_graph AS ( FROM graph g, search_graph sg WHERE g.f = sg.t AND NOT is_cycle ) -SELECT * FROM search_graph; +SELECT * FROM search_graph +SETTINGS query_plan_join_inner_table_selection = 'right' +; -- ordering by the path column has same effect as SEARCH DEPTH FIRST WITH RECURSIVE search_graph AS ( diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 6ae4e4d4d3c..69c4fb42a6b 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,6 +3,7 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES +SET query_plan_join_inner_table_selection = 'auto'; -- 'left' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference index d35bdeff98b..5fde4f80c5d 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.reference @@ -5,18 +5,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -75,18 +75,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin @@ -145,18 +145,18 @@ Header: id UInt64 rhs.value String Actions: INPUT : 0 -> __table1.id UInt64 : 0 INPUT : 1 -> __table1.value String : 1 - INPUT : 2 -> __table2.value String : 2 - INPUT : 3 -> __table2.id UInt64 : 3 + INPUT : 2 -> __table2.id UInt64 : 2 + INPUT : 3 -> __table2.value String : 3 ALIAS __table1.id :: 0 -> id UInt64 : 4 ALIAS __table1.value :: 1 -> value String : 0 - ALIAS __table2.value :: 2 -> rhs.value String : 1 - ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2 -Positions: 4 0 2 1 + ALIAS __table2.id :: 2 -> rhs.id UInt64 : 1 + ALIAS __table2.value :: 3 -> rhs.value String : 2 +Positions: 4 0 1 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 __table1.value String - __table2.value String __table2.id UInt64 + __table2.value String Type: INNER Strictness: ALL Algorithm: HashJoin diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index b3d1827d98f..ddefc322b4f 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -22,7 +22,10 @@ SETTINGS index_granularity = 16 INSERT INTO test_table_1 VALUES (1, 'Value_1'), (2, 'Value_2'); INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0; + +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -30,7 +33,9 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; @@ -38,7 +43,9 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; -EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0; +EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 +SETTINGS query_plan_join_inner_table_selection = 'right' +; SELECT '--'; diff --git a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference index 7058d36aaf9..1c82e76cc65 100644 --- a/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference +++ b/tests/queries/0_stateless/03152_join_filter_push_down_equivalent_columns.reference @@ -65,8 +65,7 @@ SELECT name FROM users RIGHT JOIN users2 USING name WHERE users2.name ='Alice'; Expression ((Project names + (Projection + ))) Header: name String Join (JOIN FillRightFirst) - Header: __table1.name String - __table2.name String + Header: __table2.name String Filter (( + Change column names to column identifiers)) Header: __table1.name String ReadFromMergeTree (default.users) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index f6e5dbdef03..eeb3ae85e84 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,6 +11,7 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; +SET query_plan_join_inner_table_selection = 'right'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, From 39b19c5dbe5f1f78e1e0e4be41b0d656bf31d0b7 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Nov 2024 19:00:13 +0100 Subject: [PATCH 090/433] Init --- src/Functions/FunctionsConversion.cpp | 2 +- ...wcardinality_string_monotonicity.reference | 2 ++ ...266_lowcardinality_string_monotonicity.sql | 35 +++++++++++++++++++ 3 files changed, 38 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference create mode 100644 tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index ee04916e7b4..a08530790ab 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -5421,7 +5421,7 @@ FunctionBasePtr createFunctionBaseCast( DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeFloat32, DataTypeFloat64, DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, - DataTypeString>(return_type.get(), [&](auto & type) + DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) { monotonicity = FunctionTo>::Type::Monotonic::get; return true; diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference new file mode 100644 index 00000000000..3d4b151810c --- /dev/null +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference @@ -0,0 +1,2 @@ +s +s diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql new file mode 100644 index 00000000000..dded41ab5fc --- /dev/null +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql @@ -0,0 +1,35 @@ +CREATE TABLE test_lc_pk (s String) engine = MergeTree ORDER BY s; + +INSERT INTO test_lc_pk SELECT toString(number) FROM numbers(1e6); + +WITH explain_table AS + ( + SELECT + explain, + rowNumberInAllBlocks() AS rn + FROM viewExplain('EXPLAIN', 'indexes = 1', ( + SELECT count() + FROM test_lc_pk + WHERE CAST(s, 'LowCardinality(String)') = '42' + )) + ) +SELECT trimLeft(e2.explain) AS keys_value +FROM explain_table AS e1 +INNER JOIN explain_table AS e2 ON e2.rn = (e1.rn + 1) +WHERE e1.explain ILIKE '%Keys%'; -- We basically try to verify that we have our column as the key in explain indexes (we don't read all data) + +WITH explain_table AS + ( + SELECT + explain, + rowNumberInAllBlocks() AS rn + FROM viewExplain('EXPLAIN', 'indexes = 1', ( + SELECT count() + FROM test_lc_pk + WHERE CAST(s, 'String') = '42' + )) + ) +SELECT trimLeft(e2.explain) AS keys_value +FROM explain_table AS e1 +INNER JOIN explain_table AS e2 ON e2.rn = (e1.rn + 1) +WHERE e1.explain ILIKE '%Keys%'; From 2480b46cd667caa749cbfd4fff47100c410552b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 19:03:25 +0100 Subject: [PATCH 091/433] Update 03266_lowcardinality_string_monotonicity.sql --- .../0_stateless/03266_lowcardinality_string_monotonicity.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql index dded41ab5fc..8c5425423f0 100644 --- a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql @@ -1,3 +1,4 @@ +DROP TABLE IF EXISTS test_lc_pk; CREATE TABLE test_lc_pk (s String) engine = MergeTree ORDER BY s; INSERT INTO test_lc_pk SELECT toString(number) FROM numbers(1e6); @@ -33,3 +34,5 @@ SELECT trimLeft(e2.explain) AS keys_value FROM explain_table AS e1 INNER JOIN explain_table AS e2 ON e2.rn = (e1.rn + 1) WHERE e1.explain ILIKE '%Keys%'; + +DROP TABLE test_lc_pk; From 3b1b9e356b1c82c37d20fdc681fa1380996b37fe Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Nov 2024 19:26:28 +0100 Subject: [PATCH 092/433] Update 03266_lowcardinality_string_monotonicity.sql --- ...266_lowcardinality_string_monotonicity.sql | 52 ++++++++----------- 1 file changed, 22 insertions(+), 30 deletions(-) diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql index 8c5425423f0..4ec03f1c64d 100644 --- a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.sql @@ -3,36 +3,28 @@ CREATE TABLE test_lc_pk (s String) engine = MergeTree ORDER BY s; INSERT INTO test_lc_pk SELECT toString(number) FROM numbers(1e6); -WITH explain_table AS - ( - SELECT - explain, - rowNumberInAllBlocks() AS rn - FROM viewExplain('EXPLAIN', 'indexes = 1', ( - SELECT count() - FROM test_lc_pk - WHERE CAST(s, 'LowCardinality(String)') = '42' - )) - ) -SELECT trimLeft(e2.explain) AS keys_value -FROM explain_table AS e1 -INNER JOIN explain_table AS e2 ON e2.rn = (e1.rn + 1) -WHERE e1.explain ILIKE '%Keys%'; -- We basically try to verify that we have our column as the key in explain indexes (we don't read all data) +SELECT trimLeft(explain) +FROM +( + SELECT * + FROM viewExplain('EXPLAIN', 'indexes = 1', ( + SELECT count() + FROM test_lc_pk + WHERE CAST(s, 'LowCardinality(String)') = '42' + )) +) +WHERE explain LIKE '%Condition%'; -- We basically try to verify that we have our column as the key in explain indexes (we don't read all data) -WITH explain_table AS - ( - SELECT - explain, - rowNumberInAllBlocks() AS rn - FROM viewExplain('EXPLAIN', 'indexes = 1', ( - SELECT count() - FROM test_lc_pk - WHERE CAST(s, 'String') = '42' - )) - ) -SELECT trimLeft(e2.explain) AS keys_value -FROM explain_table AS e1 -INNER JOIN explain_table AS e2 ON e2.rn = (e1.rn + 1) -WHERE e1.explain ILIKE '%Keys%'; +SELECT trimLeft(explain) +FROM +( + SELECT * + FROM viewExplain('EXPLAIN', 'indexes = 1', ( + SELECT count() + FROM test_lc_pk + WHERE CAST(s, 'String') = '42' + )) +) +WHERE explain LIKE '%Condition%'; DROP TABLE test_lc_pk; From d609684efff7c4321ba60c25536f6deffdf16e8a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Nov 2024 19:27:03 +0100 Subject: [PATCH 093/433] Update 03266_lowcardinality_string_monotonicity.reference --- .../03266_lowcardinality_string_monotonicity.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference index 3d4b151810c..dffe97c1d16 100644 --- a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference @@ -1,2 +1,2 @@ -s -s +Condition: (CAST(s, 'LowCardinality(String)') in ['42', '42']) +Condition: (CAST(s, 'String') in ['42', '42']) From 86593faf1936dc77796f65da9830e4c907a600dd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Nov 2024 19:30:56 +0100 Subject: [PATCH 094/433] Empty commit From ca842fbc1faf1d3446da4d9c8bf6ee0df00d1d3c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Nov 2024 19:46:35 +0100 Subject: [PATCH 095/433] Update 03266_lowcardinality_string_monotonicity.reference --- .../03266_lowcardinality_string_monotonicity.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference index dffe97c1d16..5970daf13c8 100644 --- a/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference +++ b/tests/queries/0_stateless/03266_lowcardinality_string_monotonicity.reference @@ -1,2 +1,2 @@ -Condition: (CAST(s, 'LowCardinality(String)') in ['42', '42']) -Condition: (CAST(s, 'String') in ['42', '42']) +Condition: (CAST(s, \'LowCardinality(String)\') in [\'42\', \'42\']) +Condition: (CAST(s, \'String\') in [\'42\', \'42\']) From 69e4f93a2adb1b8a88e6239406f6a4f122c08010 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Nov 2024 13:14:53 +0000 Subject: [PATCH 096/433] Fix JSON/Dynamic Native serialization with old server and new client --- src/Core/ProtocolDefines.h | 4 +++- src/DataTypes/Serializations/ISerialization.h | 3 +++ src/DataTypes/Serializations/SerializationDynamic.cpp | 7 +++++-- src/DataTypes/Serializations/SerializationObject.cpp | 10 +++++++++- src/Formats/NativeWriter.cpp | 5 +++-- 5 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index b68eff0aa5a..f8b0544a26b 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -90,6 +90,8 @@ static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54470; static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL = 54471; +static constexpr auto DBMS_MIN_REVISION_WITH_V2_DYNAMIC_AND_JSON_SERIALIZATION = 54472; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -97,6 +99,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCO /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54471; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54472; } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 400bdbf32d3..90ae6cde0ce 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -286,6 +286,9 @@ public: SUFFIX, /// Write statistics in suffix. }; ObjectAndDynamicStatisticsMode object_and_dynamic_write_statistics = ObjectAndDynamicStatisticsMode::NONE; + + /// Use old V1 serialization of JSON and Dynamic types. Needed for compatibility. + bool use_v1_object_and_dynamic_serialization = false; }; struct DeserializeBinaryBulkSettings diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 91c8797d43f..fe8e79f1ee9 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -108,14 +108,17 @@ void SerializationDynamic::serializeBinaryBulkStatePrefix( throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Dynamic column structure during serialization of binary bulk state prefix"); /// Write structure serialization version. - UInt64 structure_version = DynamicSerializationVersion::Value::V2; + UInt64 structure_version = settings.use_v1_object_and_dynamic_serialization ? DynamicSerializationVersion::Value::V1 : DynamicSerializationVersion::Value::V2; writeBinaryLittleEndian(structure_version, *stream); auto dynamic_state = std::make_shared(structure_version); - dynamic_state->variant_type = variant_info.variant_type; dynamic_state->variant_names = variant_info.variant_names; const auto & variant_column = column_dynamic.getVariantColumn(); + /// In V1 version write max_dynamic_types parameter. + if (structure_version == DynamicSerializationVersion::Value::V1) + writeVarUInt(column_dynamic.getMaxDynamicTypes(), *stream); + /// Write information about dynamic types. dynamic_state->num_dynamic_types = dynamic_state->variant_names.size() - 1; /// -1 for SharedVariant writeVarUInt(dynamic_state->num_dynamic_types, *stream); diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 1b95fddee9f..924a6a30610 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -187,7 +187,11 @@ void SerializationObject::serializeBinaryBulkStatePrefix( throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for Object column structure during serialization of binary bulk state prefix"); /// Write serialization version. - UInt64 serialization_version = settings.write_json_as_string ? ObjectSerializationVersion::Value::STRING : ObjectSerializationVersion::Value::V2; + UInt64 serialization_version = ObjectSerializationVersion::Value::V2; + if (settings.write_json_as_string) + serialization_version = ObjectSerializationVersion::Value::STRING; + else if (settings.use_v1_object_and_dynamic_serialization) + serialization_version = ObjectSerializationVersion::Value::V1; writeBinaryLittleEndian(serialization_version, *stream); auto object_state = std::make_shared(serialization_version); @@ -197,6 +201,10 @@ void SerializationObject::serializeBinaryBulkStatePrefix( return; } + /// In V1 version write max_dynamic_paths parameter. + if (serialization_version == ObjectSerializationVersion::Value::V1) + writeVarUInt(column_object.getMaxDynamicPaths(), *stream); + /// Write all dynamic paths in sorted order. object_state->sorted_dynamic_paths.reserve(dynamic_paths.size()); for (const auto & [path, _] : dynamic_paths) diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index a2e0b0a5571..8db63136008 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -56,7 +56,7 @@ void NativeWriter::flush() } -static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, const std::optional & format_settings, UInt64 offset, UInt64 limit) +static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, const std::optional & format_settings, UInt64 offset, UInt64 limit, UInt64 client_revision) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) @@ -70,6 +70,7 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co settings.low_cardinality_max_dictionary_size = 0; settings.data_types_binary_encoding = format_settings && format_settings->native.encode_types_in_binary_format; settings.write_json_as_string = format_settings && format_settings->native.write_json_as_string; + settings.use_v1_object_and_dynamic_serialization = client_revision < DBMS_MIN_REVISION_WITH_V2_DYNAMIC_AND_JSON_SERIALIZATION; ISerialization::SerializeBinaryBulkStatePtr state; serialization.serializeBinaryBulkStatePrefix(*full_column, settings, state); @@ -181,7 +182,7 @@ size_t NativeWriter::write(const Block & block) /// Data if (rows) /// Zero items of data is always represented as zero number of bytes. - writeData(*serialization, column.column, ostr, format_settings, 0, 0); + writeData(*serialization, column.column, ostr, format_settings, 0, 0, client_revision); if (index) { From 013fde41e46ab03a8e32a96b2865375781a434cd Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Nov 2024 15:18:00 +0000 Subject: [PATCH 097/433] Add setting to fallback to V1 serialization for Dynamic and Object --- src/Core/Settings.cpp | 3 +++ src/Core/SettingsChangesHistory.cpp | 1 + src/DataTypes/Serializations/SerializationDynamic.cpp | 11 +++++++---- src/DataTypes/Serializations/SerializationObject.cpp | 11 +++++++---- .../MergeTree/MergeTreeDataPartWriterCompact.cpp | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 1 + src/Storages/MergeTree/MergeTreeIOSettings.cpp | 2 ++ src/Storages/MergeTree/MergeTreeIOSettings.h | 1 + 8 files changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4c8761e503e..89adade7094 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1222,6 +1222,9 @@ Possible values: non-negative numbers. Note that if the value is too small or to If true then data can be parsed directly to columns with custom serialization (e.g. Sparse) according to hints for serialization got from the table. )", 0) \ \ + DECLARE(Bool, merge_tree_use_v1_object_and_dynamic_serialization, false, R"( +When enabled, V1 serialization version of JSON and Dynamic types will be used in MergeTree instead of V2. +)", 0) \ DECLARE(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), R"( If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0ff9d0a6833..ef51df7b26b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,6 +77,7 @@ static std::initializer_listvariant_names = variant_info.variant_names; const auto & variant_column = column_dynamic.getVariantColumn(); - /// In V1 version write max_dynamic_types parameter. - if (structure_version == DynamicSerializationVersion::Value::V1) - writeVarUInt(column_dynamic.getMaxDynamicTypes(), *stream); - /// Write information about dynamic types. dynamic_state->num_dynamic_types = dynamic_state->variant_names.size() - 1; /// -1 for SharedVariant + + /// In V1 version we had max_dynamic_types parameter written, but now we need only actual number of variants. + /// For compatibility we need to write V1 version sometimes, but we should write number of variants instead of + /// max_dynamic_types (because now max_dynamic_types can be different in different serialized columns). + if (structure_version == DynamicSerializationVersion::Value::V1) + writeVarUInt(dynamic_state->num_dynamic_types, *stream); + writeVarUInt(dynamic_state->num_dynamic_types, *stream); if (settings.data_types_binary_encoding) { diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 924a6a30610..6d09c0a0396 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -201,15 +201,18 @@ void SerializationObject::serializeBinaryBulkStatePrefix( return; } - /// In V1 version write max_dynamic_paths parameter. - if (serialization_version == ObjectSerializationVersion::Value::V1) - writeVarUInt(column_object.getMaxDynamicPaths(), *stream); - /// Write all dynamic paths in sorted order. object_state->sorted_dynamic_paths.reserve(dynamic_paths.size()); for (const auto & [path, _] : dynamic_paths) object_state->sorted_dynamic_paths.push_back(path); std::sort(object_state->sorted_dynamic_paths.begin(), object_state->sorted_dynamic_paths.end()); + + /// In V1 version we had max_dynamic_paths parameter written, but now we need only actual number of dynamic paths. + /// For compatibility we need to write V1 version sometimes, but we should write number of dynamic paths instead of + /// max_dynamic_paths (because now max_dynamic_paths can be different in different serialized columns). + if (serialization_version == ObjectSerializationVersion::Value::V1) + writeVarUInt(object_state->sorted_dynamic_paths.size(), *stream); + writeVarUInt(object_state->sorted_dynamic_paths.size(), *stream); for (const auto & path : object_state->sorted_dynamic_paths) writeStringBinary(path, *stream); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index c8d11ced683..94de76b6d52 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -154,6 +154,7 @@ void writeColumnSingleGranule( serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; serialize_settings.use_compact_variant_discriminators_serialization = settings.use_compact_variant_discriminators_serialization; + serialize_settings.use_v1_object_and_dynamic_serialization = settings.use_v1_object_and_dynamic_serialization; serialize_settings.object_and_dynamic_write_statistics = ISerialization::SerializeBinaryBulkSettings::ObjectAndDynamicStatisticsMode::PREFIX; serialization->serializeBinaryBulkStatePrefix(*column.column, serialize_settings, state); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 7c9724b1b75..c8416847cb5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -462,6 +462,7 @@ void MergeTreeDataPartWriterWide::writeColumn( { ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.use_compact_variant_discriminators_serialization = settings.use_compact_variant_discriminators_serialization; + serialize_settings.use_v1_object_and_dynamic_serialization = settings.use_v1_object_and_dynamic_serialization; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.cpp b/src/Storages/MergeTree/MergeTreeIOSettings.cpp index bacfbbd5720..dd6d0fea602 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeIOSettings.cpp @@ -12,6 +12,7 @@ namespace Setting extern const SettingsBool low_cardinality_use_single_dictionary_for_part; extern const SettingsUInt64 min_compress_block_size; extern const SettingsUInt64 max_compress_block_size; + extern const SettingsBool merge_tree_use_v1_object_and_dynamic_serialization; } namespace MergeTreeSetting @@ -53,6 +54,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , low_cardinality_max_dictionary_size(global_settings[Setting::low_cardinality_max_dictionary_size]) , low_cardinality_use_single_dictionary_for_part(global_settings[Setting::low_cardinality_use_single_dictionary_for_part] != 0) , use_compact_variant_discriminators_serialization((*storage_settings)[MergeTreeSetting::use_compact_variant_discriminators_serialization]) + , use_v1_object_and_dynamic_serialization(global_settings[Setting::merge_tree_use_v1_object_and_dynamic_serialization]) , use_adaptive_write_buffer_for_dynamic_subcolumns((*storage_settings)[MergeTreeSetting::use_adaptive_write_buffer_for_dynamic_subcolumns]) , adaptive_write_buffer_initial_size((*storage_settings)[MergeTreeSetting::adaptive_write_buffer_initial_size]) { diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 4d1d2533729..474fe8aae41 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -83,6 +83,7 @@ struct MergeTreeWriterSettings size_t low_cardinality_max_dictionary_size; bool low_cardinality_use_single_dictionary_for_part; bool use_compact_variant_discriminators_serialization; + bool use_v1_object_and_dynamic_serialization; bool use_adaptive_write_buffer_for_dynamic_subcolumns; size_t adaptive_write_buffer_initial_size; }; From 19290298825f42800ed5a16aa5f4c2670164ca99 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Nov 2024 15:35:38 +0000 Subject: [PATCH 098/433] fix --- src/Core/Joins.h | 11 ------ src/Core/Settings.cpp | 10 ++++-- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 4 --- src/Core/SettingsEnums.h | 2 -- src/Core/SettingsFields.h | 4 ++- src/Interpreters/HashJoin/HashJoin.cpp | 4 +-- src/Planner/PlannerJoinTree.cpp | 19 ++++++++--- src/Processors/QueryPlan/JoinStep.cpp | 34 ++++++++++--------- src/Processors/QueryPlan/JoinStep.h | 3 +- .../QueryPlan/Optimizations/optimizeJoin.cpp | 10 +++--- .../Transforms/PasteJoinTransform.cpp | 2 ++ tests/clickhouse-test | 4 +-- tests/integration/helpers/random_settings.py | 2 +- .../test_peak_memory_usage/test.py | 2 +- .../01015_empty_in_inner_right_join.sql.j2 | 2 +- .../02514_analyzer_drop_join_on.sql | 2 +- .../0_stateless/02835_join_step_explain.sql | 2 +- ...filter_push_down_equivalent_sets.reference | 20 +++++------ ..._join_filter_push_down_equivalent_sets.sql | 20 +++++------ .../03038_recursive_cte_postgres_4.reference | 2 +- .../03038_recursive_cte_postgres_4.sql | 2 +- .../0_stateless/03094_one_thousand_joins.sql | 2 +- ...03130_convert_outer_join_to_inner_join.sql | 6 ++-- .../03236_squashing_high_memory.sql | 2 +- .../0_stateless/03267_join_swap_bug.reference | 4 +++ .../0_stateless/03267_join_swap_bug.sql | 33 ++++++++++++++++++ 28 files changed, 129 insertions(+), 83 deletions(-) create mode 100644 tests/queries/0_stateless/03267_join_swap_bug.reference create mode 100644 tests/queries/0_stateless/03267_join_swap_bug.sql diff --git a/src/Core/Joins.h b/src/Core/Joins.h index dd6d86fc902..0964bf86e6b 100644 --- a/src/Core/Joins.h +++ b/src/Core/Joins.h @@ -119,15 +119,4 @@ enum class JoinTableSide : uint8_t const char * toString(JoinTableSide join_table_side); -/// Setting to choose which table to use as the inner table in hash join -enum class JoinInnerTableSelectionMode : uint8_t -{ - /// Use left table - Left, - /// Use right table - Right, - /// Use the table with the smallest number of rows - Auto, -}; - } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 7e8d0aabce0..9d2f448fb81 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1912,8 +1912,14 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ - DECLARE(JoinInnerTableSelectionMode, query_plan_join_inner_table_selection, JoinInnerTableSelectionMode::Auto, R"( -Select the side of the join to be the inner table in the query plan. Supported only for `ALL` join strictness with `JOIN ON` clause. Possible values: 'auto', 'left', 'right'. +DECLARE(BoolAuto, query_plan_join_swap_table, {}, R"( + Determine which side of the join should be the build table (also called inner, the one inserted into the hash table for a hash join) in the query plan. This setting is supported only for `ALL` join strictness with the `JOIN ON` clause. Possible values are: + - 'auto': Let the planner decide which table to use as the build table. + - 'false': Never swap tables (the right table is the build table). + - 'true': Always swap tables (the left table is the build table). +)", 0) \ + \ + DECLARE(Bool, join_use_nulls_for_system_columns, false, R"( )", 0) \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1cc58deb94a..a73122b0356 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -44,6 +44,7 @@ class WriteBuffer; #define COMMON_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ M(CLASS_NAME, ArrowCompression) \ M(CLASS_NAME, Bool) \ + M(CLASS_NAME, BoolAuto) \ M(CLASS_NAME, CapnProtoEnumComparingMode) \ M(CLASS_NAME, Char) \ M(CLASS_NAME, DateTimeInputFormat) \ @@ -66,7 +67,6 @@ class WriteBuffer; M(CLASS_NAME, IntervalOutputFormat) \ M(CLASS_NAME, JoinAlgorithm) \ M(CLASS_NAME, JoinStrictness) \ - M(CLASS_NAME, JoinInnerTableSelectionMode) \ M(CLASS_NAME, LightweightMutationProjectionMode) \ M(CLASS_NAME, LoadBalancing) \ M(CLASS_NAME, LocalFSReadMethod) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ed87fde8b7e..c4dda82a40e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -73,7 +73,7 @@ static std::initializer_list #include +#include #include #include #include #include #include - namespace DB { namespace ErrorCodes @@ -125,8 +125,10 @@ struct SettingAutoWrapper void readBinary(ReadBuffer & in) { changed = true; is_auto = false; base.readBinary(in); } Type valueOr(Type default_value) const { return is_auto ? default_value : base.value; } + std::optional get() const { return is_auto ? std::nullopt : std::make_optional(base.value); } }; +using SettingFieldBoolAuto = SettingAutoWrapper; using SettingFieldUInt64Auto = SettingAutoWrapper; using SettingFieldInt64Auto = SettingAutoWrapper; using SettingFieldFloatAuto = SettingAutoWrapper; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index dad8a487745..c02b8a4914e 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1249,9 +1249,9 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns(); if (expected_columns_count != result_sample_block.columns()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} expected {} ([{}] + [{}] + [{}])", result_sample_block.columns(), expected_columns_count, - left_columns_count, required_right_keys.columns(), sample_block_with_columns_to_add.columns()); + left_sample_block.dumpNames(), required_right_keys.dumpNames(), sample_block_with_columns_to_add.dumpNames()); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a1ce455f266..83a8775216b 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -104,7 +104,7 @@ namespace Setting extern const SettingsBool optimize_move_to_prewhere; extern const SettingsBool optimize_move_to_prewhere_if_final; extern const SettingsBool use_concurrency_control; - extern const SettingsJoinInnerTableSelectionMode query_plan_join_inner_table_selection; + extern const SettingsBoolAuto query_plan_join_swap_table; } namespace ErrorCodes @@ -1586,12 +1586,22 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); } + if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) { + /// We should add all duplicated columns, because join algorithm add either all colum with specified name or none + auto set_used_column_with_duplicates = [&](const NamesAndTypesList & columns, JoinTableSide join_table_side) + { + const auto & column_name = columns.front().name; + for (const auto & column : columns) + if (column.name == column_name) + table_join->setUsedColumn(column, join_table_side); + }; + if (!columns_from_left_table.empty()) - table_join->setUsedColumn(columns_from_left_table.front(), JoinTableSide::Left); + set_used_column_with_duplicates(columns_from_left_table, JoinTableSide::Left); else if (!columns_from_right_table.empty()) - table_join->setUsedColumn(columns_from_right_table.front(), JoinTableSide::Right); + set_used_column_with_duplicates(columns_from_right_table, JoinTableSide::Right); } auto join_algorithm = chooseJoinAlgorithm(table_join, join_node.getRightTableExpression(), left_header, right_header, planner_context); @@ -1700,7 +1710,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, false /*optimize_read_in_order*/, true /*optimize_skip_unused_shards*/); - join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; + + join_step->swap_join_tables = settings[Setting::query_plan_join_swap_table].get(); join_step->setStepDescription(fmt::format("JOIN {}", join_pipeline_type)); diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 7ade437822e..0cf42b5565c 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -100,37 +100,39 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (swap_streams) std::swap(pipelines[0], pipelines[1]); + std::unique_ptr joined_pipeline; if (join->pipelineType() == JoinPipelineType::YShaped) { - auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( + joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( std::move(pipelines[0]), std::move(pipelines[1]), join, join_algorithm_header, max_block_size, &processors); joined_pipeline->resize(max_streams); - return joined_pipeline; + } + else + { + joined_pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( + std::move(pipelines[0]), + std::move(pipelines[1]), + join, + join_algorithm_header, + max_block_size, + max_streams, + keep_left_read_in_order, + &processors); } - auto pipeline = QueryPipelineBuilder::joinPipelinesRightLeft( - std::move(pipelines[0]), - std::move(pipelines[1]), - join, - join_algorithm_header, - max_block_size, - max_streams, - keep_left_read_in_order, - &processors); - if (!use_new_analyzer) - return pipeline; + return joined_pipeline; - auto column_permutation = getPermutationForBlock(pipeline->getHeader(), lhs_header, rhs_header, required_output); + auto column_permutation = getPermutationForBlock(joined_pipeline->getHeader(), lhs_header, rhs_header, required_output); if (!column_permutation.empty()) { - pipeline->addSimpleTransform([&column_permutation](const Block & header) + joined_pipeline->addSimpleTransform([&column_permutation](const Block & header) { return std::make_shared(header, column_permutation); }); } - return pipeline; + return joined_pipeline; } bool JoinStep::allowPushDownToRight() const diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 1eca42c62cf..4b9b8e825f8 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -37,7 +37,8 @@ public: void setJoin(JoinPtr join_, bool swap_streams_ = false); bool allowPushDownToRight() const; - JoinInnerTableSelectionMode inner_table_selection_mode = JoinInnerTableSelectionMode::Right; + /// Swap automatically if not set, othewise always or never, depending on the value + std::optional swap_join_tables = {}; private: void updateOutputHeader() override; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index c0b31864eac..90e1f633899 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -57,8 +57,10 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) const auto & table_join = join->getTableJoin(); - /// Algorithms other than HashJoin may not support OUTER JOINs - if (table_join.kind() != JoinKind::Inner && !typeid_cast(join.get())) + if (table_join.kind() != JoinKind::Inner + && table_join.kind() != JoinKind::Left + && table_join.kind() != JoinKind::Right + && table_join.kind() != JoinKind::Full) return; /// fixme: USING clause handled specially in join algorithm, so swap breaks it @@ -67,7 +69,7 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) return; bool need_swap = false; - if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Auto) + if (!join_step->swap_join_tables.has_value()) { auto lhs_extimation = estimateReadRowsCount(*node.children[0]); auto rhs_extimation = estimateReadRowsCount(*node.children[1]); @@ -78,7 +80,7 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) if (lhs_extimation && rhs_extimation && *lhs_extimation < *rhs_extimation) need_swap = true; } - else if (join_step->inner_table_selection_mode == JoinInnerTableSelectionMode::Left) + else if (join_step->swap_join_tables.value()) { need_swap = true; } diff --git a/src/Processors/Transforms/PasteJoinTransform.cpp b/src/Processors/Transforms/PasteJoinTransform.cpp index 982a347a70f..d43a2fa99b7 100644 --- a/src/Processors/Transforms/PasteJoinTransform.cpp +++ b/src/Processors/Transforms/PasteJoinTransform.cpp @@ -101,6 +101,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() return Status(0); if (last_used_row[1] >= chunks[1].getNumRows()) return Status(1); + /// We have unused rows from both inputs size_t result_num_rows = std::min(chunks[0].getNumRows() - last_used_row[0], chunks[1].getNumRows() - last_used_row[1]); @@ -110,6 +111,7 @@ IMergingAlgorithm::Status PasteJoinAlgorithm::merge() result.addColumn(col->cut(last_used_row[source_num], result_num_rows)); last_used_row[0] += result_num_rows; last_used_row[1] += result_num_rows; + return Status(std::move(result)); } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f4c3b368632..6ece1eb7f4b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -920,8 +920,8 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), - "query_plan_join_inner_table_selection": lambda: random.choice( - ["left", "auto", "right"] + "query_plan_join_swap_table": lambda: random.choice( + ["auto", "false", "true"] ), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py index 32cde54d0e7..f89b2fd6870 100644 --- a/tests/integration/helpers/random_settings.py +++ b/tests/integration/helpers/random_settings.py @@ -6,7 +6,7 @@ def randomize_settings(): if random.random() < 0.5: yield "max_block_size", random.randint(8000, 100000) if random.random() < 0.5: - yield "query_plan_join_inner_table_selection", random.choice(["auto", "left"]) + yield "query_plan_join_swap_table", random.choice(["auto", "true", "false"]) def write_random_settings_config(destination): diff --git a/tests/integration/test_peak_memory_usage/test.py b/tests/integration/test_peak_memory_usage/test.py index 69057573173..f447c527d9b 100644 --- a/tests/integration/test_peak_memory_usage/test.py +++ b/tests/integration/test_peak_memory_usage/test.py @@ -91,7 +91,7 @@ def test_clickhouse_client_max_peak_memory_usage_distributed(started_cluster): with client(name="client1>", log=client_output, command=command_text) as client1: client1.expect(prompt) client1.send( - "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_inner_table_selection = 'right'", + "SELECT COUNT(*) FROM distributed_fixed_numbers JOIN fixed_numbers_2 ON distributed_fixed_numbers.number=fixed_numbers_2.number SETTINGS query_plan_join_swap_table = 'false'", ) client1.expect("Peak memory usage", timeout=60) client1.expect(prompt) diff --git a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 index cdbb0542ffb..629a0e384f8 100644 --- a/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 +++ b/tests/queries/0_stateless/01015_empty_in_inner_right_join.sql.j2 @@ -1,6 +1,6 @@ SET joined_subquery_requires_alias = 0; -SET query_plan_join_inner_table_selection = 'auto'; +SET query_plan_join_swap_table = 'auto'; {% for join_algorithm in ['partial_merge', 'hash'] -%} diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql index b10bf38e495..c3e4ba0db9d 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.sql @@ -16,7 +16,7 @@ CREATE TABLE d (k UInt64, d1 UInt64, d2 String) ENGINE = Memory; INSERT INTO d VALUES (1, 1, 'a'), (2, 2, 'b'), (3, 3, 'c'); SET enable_analyzer = 1; -SET query_plan_join_inner_table_selection = 'right'; +SET query_plan_join_swap_table = 'false'; -- { echoOn } diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql index b803ddbd911..11f40fb8887 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.sql +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -19,7 +19,7 @@ CREATE TABLE test_table_2 INSERT INTO test_table_1 VALUES (0, 'Value', 0); INSERT INTO test_table_2 VALUES (0, 'Value', 0); -SET query_plan_join_inner_table_selection = 'right'; +SET query_plan_join_swap_table = 'false'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index b7718d926c6..9caaf0ab77c 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -3,7 +3,7 @@ EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + (Projection + ))) Header: id UInt64 @@ -72,7 +72,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right'; +SETTINGS query_plan_join_swap_table = 'false'; ; Expression ((Project names + (Projection + ))) Header: id UInt64 @@ -141,7 +141,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + (Projection + ))) Header: id UInt64 @@ -213,7 +213,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + (Projection + ))) Header: id UInt64 @@ -282,7 +282,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + Projection)) Header: id UInt64 @@ -358,7 +358,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + Projection)) Header: id UInt64 @@ -434,7 +434,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + (Projection + ))) Header: id UInt64 @@ -503,7 +503,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + Projection)) Header: id UInt64 @@ -579,7 +579,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + Projection)) Header: id UInt64 @@ -655,7 +655,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; Expression ((Project names + Projection)) Header: id UInt64 diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index d6dcc34c796..a7b98b95082 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -23,7 +23,7 @@ INSERT INTO test_table_2 SELECT number, number FROM numbers(10); EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -36,7 +36,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right'; +SETTINGS query_plan_join_swap_table = 'false'; ; SELECT '--'; @@ -49,7 +49,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id @@ -60,7 +60,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -73,7 +73,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -86,7 +86,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -99,7 +99,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -112,7 +112,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -125,7 +125,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -138,7 +138,7 @@ SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 6 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index 7df38e855f6..64d4d0d0eb1 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -53,7 +53,7 @@ WITH RECURSIVE search_graph AS ( WHERE g.f = sg.t AND NOT is_cycle ) SELECT * FROM search_graph -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; 1 2 arc 1 -> 2 false [(1,2)] 1 3 arc 1 -> 3 false [(1,3)] diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index d33ca7b078e..65e4439c1fd 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -56,7 +56,7 @@ WITH RECURSIVE search_graph AS ( WHERE g.f = sg.t AND NOT is_cycle ) SELECT * FROM search_graph -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; -- ordering by the path column has same effect as SEARCH DEPTH FIRST diff --git a/tests/queries/0_stateless/03094_one_thousand_joins.sql b/tests/queries/0_stateless/03094_one_thousand_joins.sql index 69c4fb42a6b..a3f04fb8870 100644 --- a/tests/queries/0_stateless/03094_one_thousand_joins.sql +++ b/tests/queries/0_stateless/03094_one_thousand_joins.sql @@ -3,7 +3,7 @@ SET join_algorithm = 'default'; -- for 'full_sorting_merge' the query is 10x slower SET enable_analyzer = 1; -- old analyzer returns TOO_DEEP_SUBQUERIES -SET query_plan_join_inner_table_selection = 'auto'; -- 'left' is slower +SET query_plan_join_swap_table = 'auto'; -- 'true' is slower -- Bug 33446, marked as 'long' because it still runs around 10 sec SELECT * FROM (SELECT 1 AS x) t1 JOIN (SELECT 1 AS x) t2 ON t1.x = t2.x JOIN (SELECT 1 AS x) t3 ON t1.x = t3.x JOIN (SELECT 1 AS x) t4 ON t1.x = t4.x JOIN (SELECT 1 AS x) t5 ON t1.x = t5.x JOIN (SELECT 1 AS x) t6 ON t1.x = t6.x JOIN (SELECT 1 AS x) t7 ON t1.x = t7.x JOIN (SELECT 1 AS x) t8 ON t1.x = t8.x JOIN (SELECT 1 AS x) t9 ON t1.x = t9.x JOIN (SELECT 1 AS x) t10 ON t1.x = t10.x JOIN (SELECT 1 AS x) t11 ON t1.x = t11.x JOIN (SELECT 1 AS x) t12 ON t1.x = t12.x JOIN (SELECT 1 AS x) t13 ON t1.x = t13.x JOIN (SELECT 1 AS x) t14 ON t1.x = t14.x JOIN (SELECT 1 AS x) t15 ON t1.x = t15.x JOIN (SELECT 1 AS x) t16 ON t1.x = t16.x JOIN (SELECT 1 AS x) t17 ON t1.x = t17.x JOIN (SELECT 1 AS x) t18 ON t1.x = t18.x JOIN (SELECT 1 AS x) t19 ON t1.x = t19.x JOIN (SELECT 1 AS x) t20 ON t1.x = t20.x JOIN (SELECT 1 AS x) t21 ON t1.x = t21.x JOIN (SELECT 1 AS x) t22 ON t1.x = t22.x JOIN (SELECT 1 AS x) t23 ON t1.x = t23.x JOIN (SELECT 1 AS x) t24 ON t1.x = t24.x JOIN (SELECT 1 AS x) t25 ON t1.x = t25.x JOIN (SELECT 1 AS x) t26 ON t1.x = t26.x JOIN (SELECT 1 AS x) t27 ON t1.x = t27.x JOIN (SELECT 1 AS x) t28 ON t1.x = t28.x JOIN (SELECT 1 AS x) t29 ON t1.x = t29.x JOIN (SELECT 1 AS x) t30 ON t1.x = t30.x JOIN (SELECT 1 AS x) t31 ON t1.x = t31.x JOIN (SELECT 1 AS x) t32 ON t1.x = t32.x JOIN (SELECT 1 AS x) t33 ON t1.x = t33.x JOIN (SELECT 1 AS x) t34 ON t1.x = t34.x JOIN (SELECT 1 AS x) t35 ON t1.x = t35.x JOIN (SELECT 1 AS x) t36 ON t1.x = t36.x JOIN (SELECT 1 AS x) t37 ON t1.x = t37.x JOIN (SELECT 1 AS x) t38 ON t1.x = t38.x JOIN (SELECT 1 AS x) t39 ON t1.x = t39.x JOIN (SELECT 1 AS x) t40 ON t1.x = t40.x JOIN (SELECT 1 AS x) t41 ON t1.x = t41.x JOIN (SELECT 1 AS x) t42 ON t1.x = t42.x JOIN (SELECT 1 AS x) t43 ON t1.x = t43.x JOIN (SELECT 1 AS x) t44 ON t1.x = t44.x JOIN (SELECT 1 AS x) t45 ON t1.x = t45.x JOIN (SELECT 1 AS x) t46 ON t1.x = t46.x JOIN (SELECT 1 AS x) t47 ON t1.x = t47.x JOIN (SELECT 1 AS x) t48 ON t1.x = t48.x JOIN (SELECT 1 AS x) t49 ON t1.x = t49.x JOIN (SELECT 1 AS x) t50 ON t1.x = t50.x JOIN (SELECT 1 AS x) t51 ON t1.x = t51.x JOIN (SELECT 1 AS x) t52 ON t1.x = t52.x JOIN (SELECT 1 AS x) t53 ON t1.x = t53.x JOIN (SELECT 1 AS x) t54 ON t1.x = t54.x JOIN (SELECT 1 AS x) t55 ON t1.x = t55.x JOIN (SELECT 1 AS x) t56 ON t1.x = t56.x JOIN (SELECT 1 AS x) t57 ON t1.x = t57.x JOIN (SELECT 1 AS x) t58 ON t1.x = t58.x JOIN (SELECT 1 AS x) t59 ON t1.x = t59.x JOIN (SELECT 1 AS x) t60 ON t1.x = t60.x JOIN (SELECT 1 AS x) t61 ON t1.x = t61.x JOIN (SELECT 1 AS x) t62 ON t1.x = t62.x JOIN (SELECT 1 AS x) t63 ON t1.x = t63.x JOIN (SELECT 1 AS x) t64 ON t1.x = t64.x JOIN (SELECT 1 AS x) t65 ON t1.x = t65.x JOIN (SELECT 1 AS x) t66 ON t1.x = t66.x JOIN (SELECT 1 AS x) t67 ON t1.x = t67.x JOIN (SELECT 1 AS x) t68 ON t1.x = t68.x JOIN (SELECT 1 AS x) t69 ON t1.x = t69.x JOIN (SELECT 1 AS x) t70 ON t1.x = t70.x JOIN (SELECT 1 AS x) t71 ON t1.x = t71.x JOIN (SELECT 1 AS x) t72 ON t1.x = t72.x JOIN (SELECT 1 AS x) t73 ON t1.x = t73.x JOIN (SELECT 1 AS x) t74 ON t1.x = t74.x JOIN (SELECT 1 AS x) t75 ON t1.x = t75.x JOIN (SELECT 1 AS x) t76 ON t1.x = t76.x JOIN (SELECT 1 AS x) t77 ON t1.x = t77.x JOIN (SELECT 1 AS x) t78 ON t1.x = t78.x JOIN (SELECT 1 AS x) t79 ON t1.x = t79.x JOIN (SELECT 1 AS x) t80 ON t1.x = t80.x JOIN (SELECT 1 AS x) t81 ON t1.x = t81.x JOIN (SELECT 1 AS x) t82 ON t1.x = t82.x JOIN (SELECT 1 AS x) t83 ON t1.x = t83.x JOIN (SELECT 1 AS x) t84 ON t1.x = t84.x JOIN (SELECT 1 AS x) t85 ON t1.x = t85.x JOIN (SELECT 1 AS x) t86 ON t1.x = t86.x JOIN (SELECT 1 AS x) t87 ON t1.x = t87.x JOIN (SELECT 1 AS x) t88 ON t1.x = t88.x JOIN (SELECT 1 AS x) t89 ON t1.x = t89.x JOIN (SELECT 1 AS x) t90 ON t1.x = t90.x JOIN (SELECT 1 AS x) t91 ON t1.x = t91.x JOIN (SELECT 1 AS x) t92 ON t1.x = t92.x JOIN (SELECT 1 AS x) t93 ON t1.x = t93.x JOIN (SELECT 1 AS x) t94 ON t1.x = t94.x JOIN (SELECT 1 AS x) t95 ON t1.x = t95.x JOIN (SELECT 1 AS x) t96 ON t1.x = t96.x JOIN (SELECT 1 AS x) t97 ON t1.x = t97.x JOIN (SELECT 1 AS x) t98 ON t1.x = t98.x JOIN (SELECT 1 AS x) t99 ON t1.x = t99.x JOIN (SELECT 1 AS x) t100 ON t1.x = t100.x JOIN (SELECT 1 AS x) t101 ON t1.x = t101.x JOIN (SELECT 1 AS x) t102 ON t1.x = t102.x JOIN (SELECT 1 AS x) t103 ON t1.x = t103.x JOIN (SELECT 1 AS x) t104 ON t1.x = t104.x JOIN (SELECT 1 AS x) t105 ON t1.x = t105.x JOIN (SELECT 1 AS x) t106 ON t1.x = t106.x JOIN (SELECT 1 AS x) t107 ON t1.x = t107.x JOIN (SELECT 1 AS x) t108 ON t1.x = t108.x JOIN (SELECT 1 AS x) t109 ON t1.x = t109.x JOIN (SELECT 1 AS x) t110 ON t1.x = t110.x JOIN (SELECT 1 AS x) t111 ON t1.x = t111.x JOIN (SELECT 1 AS x) t112 ON t1.x = t112.x JOIN (SELECT 1 AS x) t113 ON t1.x = t113.x JOIN (SELECT 1 AS x) t114 ON t1.x = t114.x JOIN (SELECT 1 AS x) t115 ON t1.x = t115.x JOIN (SELECT 1 AS x) t116 ON t1.x = t116.x JOIN (SELECT 1 AS x) t117 ON t1.x = t117.x JOIN (SELECT 1 AS x) t118 ON t1.x = t118.x JOIN (SELECT 1 AS x) t119 ON t1.x = t119.x JOIN (SELECT 1 AS x) t120 ON t1.x = t120.x JOIN (SELECT 1 AS x) t121 ON t1.x = t121.x JOIN (SELECT 1 AS x) t122 ON t1.x = t122.x JOIN (SELECT 1 AS x) t123 ON t1.x = t123.x JOIN (SELECT 1 AS x) t124 ON t1.x = t124.x JOIN (SELECT 1 AS x) t125 ON t1.x = t125.x JOIN (SELECT 1 AS x) t126 ON t1.x = t126.x JOIN (SELECT 1 AS x) t127 ON t1.x = t127.x JOIN (SELECT 1 AS x) t128 ON t1.x = t128.x JOIN (SELECT 1 AS x) t129 ON t1.x = t129.x JOIN (SELECT 1 AS x) t130 ON t1.x = t130.x JOIN (SELECT 1 AS x) t131 ON t1.x = t131.x JOIN (SELECT 1 AS x) t132 ON t1.x = t132.x JOIN (SELECT 1 AS x) t133 ON t1.x = t133.x JOIN (SELECT 1 AS x) t134 ON t1.x = t134.x JOIN (SELECT 1 AS x) t135 ON t1.x = t135.x JOIN (SELECT 1 AS x) t136 ON t1.x = t136.x JOIN (SELECT 1 AS x) t137 ON t1.x = t137.x JOIN (SELECT 1 AS x) t138 ON t1.x = t138.x JOIN (SELECT 1 AS x) t139 ON t1.x = t139.x JOIN (SELECT 1 AS x) t140 ON t1.x = t140.x JOIN (SELECT 1 AS x) t141 ON t1.x = t141.x JOIN (SELECT 1 AS x) t142 ON t1.x = t142.x JOIN (SELECT 1 AS x) t143 ON t1.x = t143.x JOIN (SELECT 1 AS x) t144 ON t1.x = t144.x JOIN (SELECT 1 AS x) t145 ON t1.x = t145.x JOIN (SELECT 1 AS x) t146 ON t1.x = t146.x JOIN (SELECT 1 AS x) t147 ON t1.x = t147.x JOIN (SELECT 1 AS x) t148 ON t1.x = t148.x JOIN (SELECT 1 AS x) t149 ON t1.x = t149.x JOIN (SELECT 1 AS x) t150 ON t1.x = t150.x JOIN (SELECT 1 AS x) t151 ON t1.x = t151.x JOIN (SELECT 1 AS x) t152 ON t1.x = t152.x JOIN (SELECT 1 AS x) t153 ON t1.x = t153.x JOIN (SELECT 1 AS x) t154 ON t1.x = t154.x JOIN (SELECT 1 AS x) t155 ON t1.x = t155.x JOIN (SELECT 1 AS x) t156 ON t1.x = t156.x JOIN (SELECT 1 AS x) t157 ON t1.x = t157.x JOIN (SELECT 1 AS x) t158 ON t1.x = t158.x JOIN (SELECT 1 AS x) t159 ON t1.x = t159.x JOIN (SELECT 1 AS x) t160 ON t1.x = t160.x JOIN (SELECT 1 AS x) t161 ON t1.x = t161.x JOIN (SELECT 1 AS x) t162 ON t1.x = t162.x JOIN (SELECT 1 AS x) t163 ON t1.x = t163.x JOIN (SELECT 1 AS x) t164 ON t1.x = t164.x JOIN (SELECT 1 AS x) t165 ON t1.x = t165.x JOIN (SELECT 1 AS x) t166 ON t1.x = t166.x JOIN (SELECT 1 AS x) t167 ON t1.x = t167.x JOIN (SELECT 1 AS x) t168 ON t1.x = t168.x JOIN (SELECT 1 AS x) t169 ON t1.x = t169.x JOIN (SELECT 1 AS x) t170 ON t1.x = t170.x JOIN (SELECT 1 AS x) t171 ON t1.x = t171.x JOIN (SELECT 1 AS x) t172 ON t1.x = t172.x JOIN (SELECT 1 AS x) t173 ON t1.x = t173.x JOIN (SELECT 1 AS x) t174 ON t1.x = t174.x JOIN (SELECT 1 AS x) t175 ON t1.x = t175.x JOIN (SELECT 1 AS x) t176 ON t1.x = t176.x JOIN (SELECT 1 AS x) t177 ON t1.x = t177.x JOIN (SELECT 1 AS x) t178 ON t1.x = t178.x JOIN (SELECT 1 AS x) t179 ON t1.x = t179.x JOIN (SELECT 1 AS x) t180 ON t1.x = t180.x JOIN (SELECT 1 AS x) t181 ON t1.x = t181.x JOIN (SELECT 1 AS x) t182 ON t1.x = t182.x JOIN (SELECT 1 AS x) t183 ON t1.x = t183.x JOIN (SELECT 1 AS x) t184 ON t1.x = t184.x JOIN (SELECT 1 AS x) t185 ON t1.x = t185.x JOIN (SELECT 1 AS x) t186 ON t1.x = t186.x JOIN (SELECT 1 AS x) t187 ON t1.x = t187.x JOIN (SELECT 1 AS x) t188 ON t1.x = t188.x JOIN (SELECT 1 AS x) t189 ON t1.x = t189.x JOIN (SELECT 1 AS x) t190 ON t1.x = t190.x JOIN (SELECT 1 AS x) t191 ON t1.x = t191.x JOIN (SELECT 1 AS x) t192 ON t1.x = t192.x JOIN (SELECT 1 AS x) t193 ON t1.x = t193.x JOIN (SELECT 1 AS x) t194 ON t1.x = t194.x JOIN (SELECT 1 AS x) t195 ON t1.x = t195.x JOIN (SELECT 1 AS x) t196 ON t1.x = t196.x JOIN (SELECT 1 AS x) t197 ON t1.x = t197.x JOIN (SELECT 1 AS x) t198 ON t1.x = t198.x JOIN (SELECT 1 AS x) t199 ON t1.x = t199.x JOIN (SELECT 1 AS x) t200 ON t1.x = t200.x JOIN (SELECT 1 AS x) t201 ON t1.x = t201.x JOIN (SELECT 1 AS x) t202 ON t1.x = t202.x JOIN (SELECT 1 AS x) t203 ON t1.x = t203.x JOIN (SELECT 1 AS x) t204 ON t1.x = t204.x JOIN (SELECT 1 AS x) t205 ON t1.x = t205.x JOIN (SELECT 1 AS x) t206 ON t1.x = t206.x JOIN (SELECT 1 AS x) t207 ON t1.x = t207.x JOIN (SELECT 1 AS x) t208 ON t1.x = t208.x JOIN (SELECT 1 AS x) t209 ON t1.x = t209.x JOIN (SELECT 1 AS x) t210 ON t1.x = t210.x JOIN (SELECT 1 AS x) t211 ON t1.x = t211.x JOIN (SELECT 1 AS x) t212 ON t1.x = t212.x JOIN (SELECT 1 AS x) t213 ON t1.x = t213.x JOIN (SELECT 1 AS x) t214 ON t1.x = t214.x JOIN (SELECT 1 AS x) t215 ON t1.x = t215.x JOIN (SELECT 1 AS x) t216 ON t1.x = t216.x JOIN (SELECT 1 AS x) t217 ON t1.x = t217.x JOIN (SELECT 1 AS x) t218 ON t1.x = t218.x JOIN (SELECT 1 AS x) t219 ON t1.x = t219.x JOIN (SELECT 1 AS x) t220 ON t1.x = t220.x JOIN (SELECT 1 AS x) t221 ON t1.x = t221.x JOIN (SELECT 1 AS x) t222 ON t1.x = t222.x JOIN (SELECT 1 AS x) t223 ON t1.x = t223.x JOIN (SELECT 1 AS x) t224 ON t1.x = t224.x JOIN (SELECT 1 AS x) t225 ON t1.x = t225.x JOIN (SELECT 1 AS x) t226 ON t1.x = t226.x JOIN (SELECT 1 AS x) t227 ON t1.x = t227.x JOIN (SELECT 1 AS x) t228 ON t1.x = t228.x JOIN (SELECT 1 AS x) t229 ON t1.x = t229.x JOIN (SELECT 1 AS x) t230 ON t1.x = t230.x JOIN (SELECT 1 AS x) t231 ON t1.x = t231.x JOIN (SELECT 1 AS x) t232 ON t1.x = t232.x JOIN (SELECT 1 AS x) t233 ON t1.x = t233.x JOIN (SELECT 1 AS x) t234 ON t1.x = t234.x JOIN (SELECT 1 AS x) t235 ON t1.x = t235.x JOIN (SELECT 1 AS x) t236 ON t1.x = t236.x JOIN (SELECT 1 AS x) t237 ON t1.x = t237.x JOIN (SELECT 1 AS x) t238 ON t1.x = t238.x JOIN (SELECT 1 AS x) t239 ON t1.x = t239.x JOIN (SELECT 1 AS x) t240 ON t1.x = t240.x JOIN (SELECT 1 AS x) t241 ON t1.x = t241.x JOIN (SELECT 1 AS x) t242 ON t1.x = t242.x JOIN (SELECT 1 AS x) t243 ON t1.x = t243.x JOIN (SELECT 1 AS x) t244 ON t1.x = t244.x JOIN (SELECT 1 AS x) t245 ON t1.x = t245.x JOIN (SELECT 1 AS x) t246 ON t1.x = t246.x JOIN (SELECT 1 AS x) t247 ON t1.x = t247.x JOIN (SELECT 1 AS x) t248 ON t1.x = t248.x JOIN (SELECT 1 AS x) t249 ON t1.x = t249.x JOIN (SELECT 1 AS x) t250 ON t1.x = t250.x JOIN (SELECT 1 AS x) t251 ON t1.x = t251.x JOIN (SELECT 1 AS x) t252 ON t1.x = t252.x JOIN (SELECT 1 AS x) t253 ON t1.x = t253.x JOIN (SELECT 1 AS x) t254 ON t1.x = t254.x JOIN (SELECT 1 AS x) t255 ON t1.x = t255.x JOIN (SELECT 1 AS x) t256 ON t1.x = t256.x JOIN (SELECT 1 AS x) t257 ON t1.x = t257.x JOIN (SELECT 1 AS x) t258 ON t1.x = t258.x JOIN (SELECT 1 AS x) t259 ON t1.x = t259.x JOIN (SELECT 1 AS x) t260 ON t1.x = t260.x JOIN (SELECT 1 AS x) t261 ON t1.x = t261.x JOIN (SELECT 1 AS x) t262 ON t1.x = t262.x JOIN (SELECT 1 AS x) t263 ON t1.x = t263.x JOIN (SELECT 1 AS x) t264 ON t1.x = t264.x JOIN (SELECT 1 AS x) t265 ON t1.x = t265.x JOIN (SELECT 1 AS x) t266 ON t1.x = t266.x JOIN (SELECT 1 AS x) t267 ON t1.x = t267.x JOIN (SELECT 1 AS x) t268 ON t1.x = t268.x JOIN (SELECT 1 AS x) t269 ON t1.x = t269.x JOIN (SELECT 1 AS x) t270 ON t1.x = t270.x JOIN (SELECT 1 AS x) t271 ON t1.x = t271.x JOIN (SELECT 1 AS x) t272 ON t1.x = t272.x JOIN (SELECT 1 AS x) t273 ON t1.x = t273.x JOIN (SELECT 1 AS x) t274 ON t1.x = t274.x JOIN (SELECT 1 AS x) t275 ON t1.x = t275.x JOIN (SELECT 1 AS x) t276 ON t1.x = t276.x JOIN (SELECT 1 AS x) t277 ON t1.x = t277.x JOIN (SELECT 1 AS x) t278 ON t1.x = t278.x JOIN (SELECT 1 AS x) t279 ON t1.x = t279.x JOIN (SELECT 1 AS x) t280 ON t1.x = t280.x JOIN (SELECT 1 AS x) t281 ON t1.x = t281.x JOIN (SELECT 1 AS x) t282 ON t1.x = t282.x JOIN (SELECT 1 AS x) t283 ON t1.x = t283.x JOIN (SELECT 1 AS x) t284 ON t1.x = t284.x JOIN (SELECT 1 AS x) t285 ON t1.x = t285.x JOIN (SELECT 1 AS x) t286 ON t1.x = t286.x JOIN (SELECT 1 AS x) t287 ON t1.x = t287.x JOIN (SELECT 1 AS x) t288 ON t1.x = t288.x JOIN (SELECT 1 AS x) t289 ON t1.x = t289.x JOIN (SELECT 1 AS x) t290 ON t1.x = t290.x JOIN (SELECT 1 AS x) t291 ON t1.x = t291.x JOIN (SELECT 1 AS x) t292 ON t1.x = t292.x JOIN (SELECT 1 AS x) t293 ON t1.x = t293.x JOIN (SELECT 1 AS x) t294 ON t1.x = t294.x JOIN (SELECT 1 AS x) t295 ON t1.x = t295.x JOIN (SELECT 1 AS x) t296 ON t1.x = t296.x JOIN (SELECT 1 AS x) t297 ON t1.x = t297.x JOIN (SELECT 1 AS x) t298 ON t1.x = t298.x JOIN (SELECT 1 AS x) t299 ON t1.x = t299.x JOIN (SELECT 1 AS x) t300 ON t1.x = t300.x JOIN (SELECT 1 AS x) t301 ON t1.x = t301.x JOIN (SELECT 1 AS x) t302 ON t1.x = t302.x JOIN (SELECT 1 AS x) t303 ON t1.x = t303.x JOIN (SELECT 1 AS x) t304 ON t1.x = t304.x JOIN (SELECT 1 AS x) t305 ON t1.x = t305.x JOIN (SELECT 1 AS x) t306 ON t1.x = t306.x JOIN (SELECT 1 AS x) t307 ON t1.x = t307.x JOIN (SELECT 1 AS x) t308 ON t1.x = t308.x JOIN (SELECT 1 AS x) t309 ON t1.x = t309.x JOIN (SELECT 1 AS x) t310 ON t1.x = t310.x JOIN (SELECT 1 AS x) t311 ON t1.x = t311.x JOIN (SELECT 1 AS x) t312 ON t1.x = t312.x JOIN (SELECT 1 AS x) t313 ON t1.x = t313.x JOIN (SELECT 1 AS x) t314 ON t1.x = t314.x JOIN (SELECT 1 AS x) t315 ON t1.x = t315.x JOIN (SELECT 1 AS x) t316 ON t1.x = t316.x JOIN (SELECT 1 AS x) t317 ON t1.x = t317.x JOIN (SELECT 1 AS x) t318 ON t1.x = t318.x JOIN (SELECT 1 AS x) t319 ON t1.x = t319.x JOIN (SELECT 1 AS x) t320 ON t1.x = t320.x JOIN (SELECT 1 AS x) t321 ON t1.x = t321.x JOIN (SELECT 1 AS x) t322 ON t1.x = t322.x JOIN (SELECT 1 AS x) t323 ON t1.x = t323.x JOIN (SELECT 1 AS x) t324 ON t1.x = t324.x JOIN (SELECT 1 AS x) t325 ON t1.x = t325.x JOIN (SELECT 1 AS x) t326 ON t1.x = t326.x JOIN (SELECT 1 AS x) t327 ON t1.x = t327.x JOIN (SELECT 1 AS x) t328 ON t1.x = t328.x JOIN (SELECT 1 AS x) t329 ON t1.x = t329.x JOIN (SELECT 1 AS x) t330 ON t1.x = t330.x JOIN (SELECT 1 AS x) t331 ON t1.x = t331.x JOIN (SELECT 1 AS x) t332 ON t1.x = t332.x JOIN (SELECT 1 AS x) t333 ON t1.x = t333.x JOIN (SELECT 1 AS x) t334 ON t1.x = t334.x JOIN (SELECT 1 AS x) t335 ON t1.x = t335.x JOIN (SELECT 1 AS x) t336 ON t1.x = t336.x JOIN (SELECT 1 AS x) t337 ON t1.x = t337.x JOIN (SELECT 1 AS x) t338 ON t1.x = t338.x JOIN (SELECT 1 AS x) t339 ON t1.x = t339.x JOIN (SELECT 1 AS x) t340 ON t1.x = t340.x JOIN (SELECT 1 AS x) t341 ON t1.x = t341.x JOIN (SELECT 1 AS x) t342 ON t1.x = t342.x JOIN (SELECT 1 AS x) t343 ON t1.x = t343.x JOIN (SELECT 1 AS x) t344 ON t1.x = t344.x JOIN (SELECT 1 AS x) t345 ON t1.x = t345.x JOIN (SELECT 1 AS x) t346 ON t1.x = t346.x JOIN (SELECT 1 AS x) t347 ON t1.x = t347.x JOIN (SELECT 1 AS x) t348 ON t1.x = t348.x JOIN (SELECT 1 AS x) t349 ON t1.x = t349.x JOIN (SELECT 1 AS x) t350 ON t1.x = t350.x JOIN (SELECT 1 AS x) t351 ON t1.x = t351.x JOIN (SELECT 1 AS x) t352 ON t1.x = t352.x JOIN (SELECT 1 AS x) t353 ON t1.x = t353.x JOIN (SELECT 1 AS x) t354 ON t1.x = t354.x JOIN (SELECT 1 AS x) t355 ON t1.x = t355.x JOIN (SELECT 1 AS x) t356 ON t1.x = t356.x JOIN (SELECT 1 AS x) t357 ON t1.x = t357.x JOIN (SELECT 1 AS x) t358 ON t1.x = t358.x JOIN (SELECT 1 AS x) t359 ON t1.x = t359.x JOIN (SELECT 1 AS x) t360 ON t1.x = t360.x JOIN (SELECT 1 AS x) t361 ON t1.x = t361.x JOIN (SELECT 1 AS x) t362 ON t1.x = t362.x JOIN (SELECT 1 AS x) t363 ON t1.x = t363.x JOIN (SELECT 1 AS x) t364 ON t1.x = t364.x JOIN (SELECT 1 AS x) t365 ON t1.x = t365.x JOIN (SELECT 1 AS x) t366 ON t1.x = t366.x JOIN (SELECT 1 AS x) t367 ON t1.x = t367.x JOIN (SELECT 1 AS x) t368 ON t1.x = t368.x JOIN (SELECT 1 AS x) t369 ON t1.x = t369.x JOIN (SELECT 1 AS x) t370 ON t1.x = t370.x JOIN (SELECT 1 AS x) t371 ON t1.x = t371.x JOIN (SELECT 1 AS x) t372 ON t1.x = t372.x JOIN (SELECT 1 AS x) t373 ON t1.x = t373.x JOIN (SELECT 1 AS x) t374 ON t1.x = t374.x JOIN (SELECT 1 AS x) t375 ON t1.x = t375.x JOIN (SELECT 1 AS x) t376 ON t1.x = t376.x JOIN (SELECT 1 AS x) t377 ON t1.x = t377.x JOIN (SELECT 1 AS x) t378 ON t1.x = t378.x JOIN (SELECT 1 AS x) t379 ON t1.x = t379.x JOIN (SELECT 1 AS x) t380 ON t1.x = t380.x JOIN (SELECT 1 AS x) t381 ON t1.x = t381.x JOIN (SELECT 1 AS x) t382 ON t1.x = t382.x JOIN (SELECT 1 AS x) t383 ON t1.x = t383.x JOIN (SELECT 1 AS x) t384 ON t1.x = t384.x JOIN (SELECT 1 AS x) t385 ON t1.x = t385.x JOIN (SELECT 1 AS x) t386 ON t1.x = t386.x JOIN (SELECT 1 AS x) t387 ON t1.x = t387.x JOIN (SELECT 1 AS x) t388 ON t1.x = t388.x JOIN (SELECT 1 AS x) t389 ON t1.x = t389.x JOIN (SELECT 1 AS x) t390 ON t1.x = t390.x JOIN (SELECT 1 AS x) t391 ON t1.x = t391.x JOIN (SELECT 1 AS x) t392 ON t1.x = t392.x JOIN (SELECT 1 AS x) t393 ON t1.x = t393.x JOIN (SELECT 1 AS x) t394 ON t1.x = t394.x JOIN (SELECT 1 AS x) t395 ON t1.x = t395.x JOIN (SELECT 1 AS x) t396 ON t1.x = t396.x JOIN (SELECT 1 AS x) t397 ON t1.x = t397.x JOIN (SELECT 1 AS x) t398 ON t1.x = t398.x JOIN (SELECT 1 AS x) t399 ON t1.x = t399.x JOIN (SELECT 1 AS x) t400 ON t1.x = t400.x JOIN (SELECT 1 AS x) t401 ON t1.x = t401.x JOIN (SELECT 1 AS x) t402 ON t1.x = t402.x JOIN (SELECT 1 AS x) t403 ON t1.x = t403.x JOIN (SELECT 1 AS x) t404 ON t1.x = t404.x JOIN (SELECT 1 AS x) t405 ON t1.x = t405.x JOIN (SELECT 1 AS x) t406 ON t1.x = t406.x JOIN (SELECT 1 AS x) t407 ON t1.x = t407.x JOIN (SELECT 1 AS x) t408 ON t1.x = t408.x JOIN (SELECT 1 AS x) t409 ON t1.x = t409.x JOIN (SELECT 1 AS x) t410 ON t1.x = t410.x JOIN (SELECT 1 AS x) t411 ON t1.x = t411.x JOIN (SELECT 1 AS x) t412 ON t1.x = t412.x JOIN (SELECT 1 AS x) t413 ON t1.x = t413.x JOIN (SELECT 1 AS x) t414 ON t1.x = t414.x JOIN (SELECT 1 AS x) t415 ON t1.x = t415.x JOIN (SELECT 1 AS x) t416 ON t1.x = t416.x JOIN (SELECT 1 AS x) t417 ON t1.x = t417.x JOIN (SELECT 1 AS x) t418 ON t1.x = t418.x JOIN (SELECT 1 AS x) t419 ON t1.x = t419.x JOIN (SELECT 1 AS x) t420 ON t1.x = t420.x JOIN (SELECT 1 AS x) t421 ON t1.x = t421.x JOIN (SELECT 1 AS x) t422 ON t1.x = t422.x JOIN (SELECT 1 AS x) t423 ON t1.x = t423.x JOIN (SELECT 1 AS x) t424 ON t1.x = t424.x JOIN (SELECT 1 AS x) t425 ON t1.x = t425.x JOIN (SELECT 1 AS x) t426 ON t1.x = t426.x JOIN (SELECT 1 AS x) t427 ON t1.x = t427.x JOIN (SELECT 1 AS x) t428 ON t1.x = t428.x JOIN (SELECT 1 AS x) t429 ON t1.x = t429.x JOIN (SELECT 1 AS x) t430 ON t1.x = t430.x JOIN (SELECT 1 AS x) t431 ON t1.x = t431.x JOIN (SELECT 1 AS x) t432 ON t1.x = t432.x JOIN (SELECT 1 AS x) t433 ON t1.x = t433.x JOIN (SELECT 1 AS x) t434 ON t1.x = t434.x JOIN (SELECT 1 AS x) t435 ON t1.x = t435.x JOIN (SELECT 1 AS x) t436 ON t1.x = t436.x JOIN (SELECT 1 AS x) t437 ON t1.x = t437.x JOIN (SELECT 1 AS x) t438 ON t1.x = t438.x JOIN (SELECT 1 AS x) t439 ON t1.x = t439.x JOIN (SELECT 1 AS x) t440 ON t1.x = t440.x JOIN (SELECT 1 AS x) t441 ON t1.x = t441.x JOIN (SELECT 1 AS x) t442 ON t1.x = t442.x JOIN (SELECT 1 AS x) t443 ON t1.x = t443.x JOIN (SELECT 1 AS x) t444 ON t1.x = t444.x JOIN (SELECT 1 AS x) t445 ON t1.x = t445.x JOIN (SELECT 1 AS x) t446 ON t1.x = t446.x JOIN (SELECT 1 AS x) t447 ON t1.x = t447.x JOIN (SELECT 1 AS x) t448 ON t1.x = t448.x JOIN (SELECT 1 AS x) t449 ON t1.x = t449.x JOIN (SELECT 1 AS x) t450 ON t1.x = t450.x JOIN (SELECT 1 AS x) t451 ON t1.x = t451.x JOIN (SELECT 1 AS x) t452 ON t1.x = t452.x JOIN (SELECT 1 AS x) t453 ON t1.x = t453.x JOIN (SELECT 1 AS x) t454 ON t1.x = t454.x JOIN (SELECT 1 AS x) t455 ON t1.x = t455.x JOIN (SELECT 1 AS x) t456 ON t1.x = t456.x JOIN (SELECT 1 AS x) t457 ON t1.x = t457.x JOIN (SELECT 1 AS x) t458 ON t1.x = t458.x JOIN (SELECT 1 AS x) t459 ON t1.x = t459.x JOIN (SELECT 1 AS x) t460 ON t1.x = t460.x JOIN (SELECT 1 AS x) t461 ON t1.x = t461.x JOIN (SELECT 1 AS x) t462 ON t1.x = t462.x JOIN (SELECT 1 AS x) t463 ON t1.x = t463.x JOIN (SELECT 1 AS x) t464 ON t1.x = t464.x JOIN (SELECT 1 AS x) t465 ON t1.x = t465.x JOIN (SELECT 1 AS x) t466 ON t1.x = t466.x JOIN (SELECT 1 AS x) t467 ON t1.x = t467.x JOIN (SELECT 1 AS x) t468 ON t1.x = t468.x JOIN (SELECT 1 AS x) t469 ON t1.x = t469.x JOIN (SELECT 1 AS x) t470 ON t1.x = t470.x JOIN (SELECT 1 AS x) t471 ON t1.x = t471.x JOIN (SELECT 1 AS x) t472 ON t1.x = t472.x JOIN (SELECT 1 AS x) t473 ON t1.x = t473.x JOIN (SELECT 1 AS x) t474 ON t1.x = t474.x JOIN (SELECT 1 AS x) t475 ON t1.x = t475.x JOIN (SELECT 1 AS x) t476 ON t1.x = t476.x JOIN (SELECT 1 AS x) t477 ON t1.x = t477.x JOIN (SELECT 1 AS x) t478 ON t1.x = t478.x JOIN (SELECT 1 AS x) t479 ON t1.x = t479.x JOIN (SELECT 1 AS x) t480 ON t1.x = t480.x JOIN (SELECT 1 AS x) t481 ON t1.x = t481.x JOIN (SELECT 1 AS x) t482 ON t1.x = t482.x JOIN (SELECT 1 AS x) t483 ON t1.x = t483.x JOIN (SELECT 1 AS x) t484 ON t1.x = t484.x JOIN (SELECT 1 AS x) t485 ON t1.x = t485.x JOIN (SELECT 1 AS x) t486 ON t1.x = t486.x JOIN (SELECT 1 AS x) t487 ON t1.x = t487.x JOIN (SELECT 1 AS x) t488 ON t1.x = t488.x JOIN (SELECT 1 AS x) t489 ON t1.x = t489.x JOIN (SELECT 1 AS x) t490 ON t1.x = t490.x JOIN (SELECT 1 AS x) t491 ON t1.x = t491.x JOIN (SELECT 1 AS x) t492 ON t1.x = t492.x JOIN (SELECT 1 AS x) t493 ON t1.x = t493.x JOIN (SELECT 1 AS x) t494 ON t1.x = t494.x JOIN (SELECT 1 AS x) t495 ON t1.x = t495.x JOIN (SELECT 1 AS x) t496 ON t1.x = t496.x JOIN (SELECT 1 AS x) t497 ON t1.x = t497.x JOIN (SELECT 1 AS x) t498 ON t1.x = t498.x JOIN (SELECT 1 AS x) t499 ON t1.x = t499.x JOIN (SELECT 1 AS x) t500 ON t1.x = t500.x JOIN (SELECT 1 AS x) t501 ON t1.x = t501.x JOIN (SELECT 1 AS x) t502 ON t1.x = t502.x JOIN (SELECT 1 AS x) t503 ON t1.x = t503.x JOIN (SELECT 1 AS x) t504 ON t1.x = t504.x JOIN (SELECT 1 AS x) t505 ON t1.x = t505.x JOIN (SELECT 1 AS x) t506 ON t1.x = t506.x JOIN (SELECT 1 AS x) t507 ON t1.x = t507.x JOIN (SELECT 1 AS x) t508 ON t1.x = t508.x JOIN (SELECT 1 AS x) t509 ON t1.x = t509.x JOIN (SELECT 1 AS x) t510 ON t1.x = t510.x JOIN (SELECT 1 AS x) t511 ON t1.x = t511.x JOIN (SELECT 1 AS x) t512 ON t1.x = t512.x JOIN (SELECT 1 AS x) t513 ON t1.x = t513.x JOIN (SELECT 1 AS x) t514 ON t1.x = t514.x JOIN (SELECT 1 AS x) t515 ON t1.x = t515.x JOIN (SELECT 1 AS x) t516 ON t1.x = t516.x JOIN (SELECT 1 AS x) t517 ON t1.x = t517.x JOIN (SELECT 1 AS x) t518 ON t1.x = t518.x JOIN (SELECT 1 AS x) t519 ON t1.x = t519.x JOIN (SELECT 1 AS x) t520 ON t1.x = t520.x JOIN (SELECT 1 AS x) t521 ON t1.x = t521.x JOIN (SELECT 1 AS x) t522 ON t1.x = t522.x JOIN (SELECT 1 AS x) t523 ON t1.x = t523.x JOIN (SELECT 1 AS x) t524 ON t1.x = t524.x JOIN (SELECT 1 AS x) t525 ON t1.x = t525.x JOIN (SELECT 1 AS x) t526 ON t1.x = t526.x JOIN (SELECT 1 AS x) t527 ON t1.x = t527.x JOIN (SELECT 1 AS x) t528 ON t1.x = t528.x JOIN (SELECT 1 AS x) t529 ON t1.x = t529.x JOIN (SELECT 1 AS x) t530 ON t1.x = t530.x JOIN (SELECT 1 AS x) t531 ON t1.x = t531.x JOIN (SELECT 1 AS x) t532 ON t1.x = t532.x JOIN (SELECT 1 AS x) t533 ON t1.x = t533.x JOIN (SELECT 1 AS x) t534 ON t1.x = t534.x JOIN (SELECT 1 AS x) t535 ON t1.x = t535.x JOIN (SELECT 1 AS x) t536 ON t1.x = t536.x JOIN (SELECT 1 AS x) t537 ON t1.x = t537.x JOIN (SELECT 1 AS x) t538 ON t1.x = t538.x JOIN (SELECT 1 AS x) t539 ON t1.x = t539.x JOIN (SELECT 1 AS x) t540 ON t1.x = t540.x JOIN (SELECT 1 AS x) t541 ON t1.x = t541.x JOIN (SELECT 1 AS x) t542 ON t1.x = t542.x JOIN (SELECT 1 AS x) t543 ON t1.x = t543.x JOIN (SELECT 1 AS x) t544 ON t1.x = t544.x JOIN (SELECT 1 AS x) t545 ON t1.x = t545.x JOIN (SELECT 1 AS x) t546 ON t1.x = t546.x JOIN (SELECT 1 AS x) t547 ON t1.x = t547.x JOIN (SELECT 1 AS x) t548 ON t1.x = t548.x JOIN (SELECT 1 AS x) t549 ON t1.x = t549.x JOIN (SELECT 1 AS x) t550 ON t1.x = t550.x JOIN (SELECT 1 AS x) t551 ON t1.x = t551.x JOIN (SELECT 1 AS x) t552 ON t1.x = t552.x JOIN (SELECT 1 AS x) t553 ON t1.x = t553.x JOIN (SELECT 1 AS x) t554 ON t1.x = t554.x JOIN (SELECT 1 AS x) t555 ON t1.x = t555.x JOIN (SELECT 1 AS x) t556 ON t1.x = t556.x JOIN (SELECT 1 AS x) t557 ON t1.x = t557.x JOIN (SELECT 1 AS x) t558 ON t1.x = t558.x JOIN (SELECT 1 AS x) t559 ON t1.x = t559.x JOIN (SELECT 1 AS x) t560 ON t1.x = t560.x JOIN (SELECT 1 AS x) t561 ON t1.x = t561.x JOIN (SELECT 1 AS x) t562 ON t1.x = t562.x JOIN (SELECT 1 AS x) t563 ON t1.x = t563.x JOIN (SELECT 1 AS x) t564 ON t1.x = t564.x JOIN (SELECT 1 AS x) t565 ON t1.x = t565.x JOIN (SELECT 1 AS x) t566 ON t1.x = t566.x JOIN (SELECT 1 AS x) t567 ON t1.x = t567.x JOIN (SELECT 1 AS x) t568 ON t1.x = t568.x JOIN (SELECT 1 AS x) t569 ON t1.x = t569.x JOIN (SELECT 1 AS x) t570 ON t1.x = t570.x JOIN (SELECT 1 AS x) t571 ON t1.x = t571.x JOIN (SELECT 1 AS x) t572 ON t1.x = t572.x JOIN (SELECT 1 AS x) t573 ON t1.x = t573.x JOIN (SELECT 1 AS x) t574 ON t1.x = t574.x JOIN (SELECT 1 AS x) t575 ON t1.x = t575.x JOIN (SELECT 1 AS x) t576 ON t1.x = t576.x JOIN (SELECT 1 AS x) t577 ON t1.x = t577.x JOIN (SELECT 1 AS x) t578 ON t1.x = t578.x JOIN (SELECT 1 AS x) t579 ON t1.x = t579.x JOIN (SELECT 1 AS x) t580 ON t1.x = t580.x JOIN (SELECT 1 AS x) t581 ON t1.x = t581.x JOIN (SELECT 1 AS x) t582 ON t1.x = t582.x JOIN (SELECT 1 AS x) t583 ON t1.x = t583.x JOIN (SELECT 1 AS x) t584 ON t1.x = t584.x JOIN (SELECT 1 AS x) t585 ON t1.x = t585.x JOIN (SELECT 1 AS x) t586 ON t1.x = t586.x JOIN (SELECT 1 AS x) t587 ON t1.x = t587.x JOIN (SELECT 1 AS x) t588 ON t1.x = t588.x JOIN (SELECT 1 AS x) t589 ON t1.x = t589.x JOIN (SELECT 1 AS x) t590 ON t1.x = t590.x JOIN (SELECT 1 AS x) t591 ON t1.x = t591.x JOIN (SELECT 1 AS x) t592 ON t1.x = t592.x JOIN (SELECT 1 AS x) t593 ON t1.x = t593.x JOIN (SELECT 1 AS x) t594 ON t1.x = t594.x JOIN (SELECT 1 AS x) t595 ON t1.x = t595.x JOIN (SELECT 1 AS x) t596 ON t1.x = t596.x JOIN (SELECT 1 AS x) t597 ON t1.x = t597.x JOIN (SELECT 1 AS x) t598 ON t1.x = t598.x JOIN (SELECT 1 AS x) t599 ON t1.x = t599.x JOIN (SELECT 1 AS x) t600 ON t1.x = t600.x JOIN (SELECT 1 AS x) t601 ON t1.x = t601.x JOIN (SELECT 1 AS x) t602 ON t1.x = t602.x JOIN (SELECT 1 AS x) t603 ON t1.x = t603.x JOIN (SELECT 1 AS x) t604 ON t1.x = t604.x JOIN (SELECT 1 AS x) t605 ON t1.x = t605.x JOIN (SELECT 1 AS x) t606 ON t1.x = t606.x JOIN (SELECT 1 AS x) t607 ON t1.x = t607.x JOIN (SELECT 1 AS x) t608 ON t1.x = t608.x JOIN (SELECT 1 AS x) t609 ON t1.x = t609.x JOIN (SELECT 1 AS x) t610 ON t1.x = t610.x JOIN (SELECT 1 AS x) t611 ON t1.x = t611.x JOIN (SELECT 1 AS x) t612 ON t1.x = t612.x JOIN (SELECT 1 AS x) t613 ON t1.x = t613.x JOIN (SELECT 1 AS x) t614 ON t1.x = t614.x JOIN (SELECT 1 AS x) t615 ON t1.x = t615.x JOIN (SELECT 1 AS x) t616 ON t1.x = t616.x JOIN (SELECT 1 AS x) t617 ON t1.x = t617.x JOIN (SELECT 1 AS x) t618 ON t1.x = t618.x JOIN (SELECT 1 AS x) t619 ON t1.x = t619.x JOIN (SELECT 1 AS x) t620 ON t1.x = t620.x JOIN (SELECT 1 AS x) t621 ON t1.x = t621.x JOIN (SELECT 1 AS x) t622 ON t1.x = t622.x JOIN (SELECT 1 AS x) t623 ON t1.x = t623.x JOIN (SELECT 1 AS x) t624 ON t1.x = t624.x JOIN (SELECT 1 AS x) t625 ON t1.x = t625.x JOIN (SELECT 1 AS x) t626 ON t1.x = t626.x JOIN (SELECT 1 AS x) t627 ON t1.x = t627.x JOIN (SELECT 1 AS x) t628 ON t1.x = t628.x JOIN (SELECT 1 AS x) t629 ON t1.x = t629.x JOIN (SELECT 1 AS x) t630 ON t1.x = t630.x JOIN (SELECT 1 AS x) t631 ON t1.x = t631.x JOIN (SELECT 1 AS x) t632 ON t1.x = t632.x JOIN (SELECT 1 AS x) t633 ON t1.x = t633.x JOIN (SELECT 1 AS x) t634 ON t1.x = t634.x JOIN (SELECT 1 AS x) t635 ON t1.x = t635.x JOIN (SELECT 1 AS x) t636 ON t1.x = t636.x JOIN (SELECT 1 AS x) t637 ON t1.x = t637.x JOIN (SELECT 1 AS x) t638 ON t1.x = t638.x JOIN (SELECT 1 AS x) t639 ON t1.x = t639.x JOIN (SELECT 1 AS x) t640 ON t1.x = t640.x JOIN (SELECT 1 AS x) t641 ON t1.x = t641.x JOIN (SELECT 1 AS x) t642 ON t1.x = t642.x JOIN (SELECT 1 AS x) t643 ON t1.x = t643.x JOIN (SELECT 1 AS x) t644 ON t1.x = t644.x JOIN (SELECT 1 AS x) t645 ON t1.x = t645.x JOIN (SELECT 1 AS x) t646 ON t1.x = t646.x JOIN (SELECT 1 AS x) t647 ON t1.x = t647.x JOIN (SELECT 1 AS x) t648 ON t1.x = t648.x JOIN (SELECT 1 AS x) t649 ON t1.x = t649.x JOIN (SELECT 1 AS x) t650 ON t1.x = t650.x JOIN (SELECT 1 AS x) t651 ON t1.x = t651.x JOIN (SELECT 1 AS x) t652 ON t1.x = t652.x JOIN (SELECT 1 AS x) t653 ON t1.x = t653.x JOIN (SELECT 1 AS x) t654 ON t1.x = t654.x JOIN (SELECT 1 AS x) t655 ON t1.x = t655.x JOIN (SELECT 1 AS x) t656 ON t1.x = t656.x JOIN (SELECT 1 AS x) t657 ON t1.x = t657.x JOIN (SELECT 1 AS x) t658 ON t1.x = t658.x JOIN (SELECT 1 AS x) t659 ON t1.x = t659.x JOIN (SELECT 1 AS x) t660 ON t1.x = t660.x JOIN (SELECT 1 AS x) t661 ON t1.x = t661.x JOIN (SELECT 1 AS x) t662 ON t1.x = t662.x JOIN (SELECT 1 AS x) t663 ON t1.x = t663.x JOIN (SELECT 1 AS x) t664 ON t1.x = t664.x JOIN (SELECT 1 AS x) t665 ON t1.x = t665.x JOIN (SELECT 1 AS x) t666 ON t1.x = t666.x diff --git a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql index ddefc322b4f..0a53bf03ad5 100644 --- a/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql +++ b/tests/queries/0_stateless/03130_convert_outer_join_to_inner_join.sql @@ -24,7 +24,7 @@ INSERT INTO test_table_2 VALUES (2, 'Value_2'), (3, 'Value_3'); EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -34,7 +34,7 @@ SELECT * FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs. SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; @@ -44,7 +44,7 @@ SELECT * FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs SELECT '--'; EXPLAIN header = 1, actions = 1 SELECT * FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id != 0 AND rhs.id != 0 -SETTINGS query_plan_join_inner_table_selection = 'right' +SETTINGS query_plan_join_swap_table = 'false' ; SELECT '--'; diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index eeb3ae85e84..20a8be6d499 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -11,7 +11,7 @@ CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; -SET query_plan_join_inner_table_selection = 'right'; +SET query_plan_join_swap_table = 'false'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, diff --git a/tests/queries/0_stateless/03267_join_swap_bug.reference b/tests/queries/0_stateless/03267_join_swap_bug.reference new file mode 100644 index 00000000000..68b8a88e2c1 --- /dev/null +++ b/tests/queries/0_stateless/03267_join_swap_bug.reference @@ -0,0 +1,4 @@ +1 +1 +0 +1 diff --git a/tests/queries/0_stateless/03267_join_swap_bug.sql b/tests/queries/0_stateless/03267_join_swap_bug.sql new file mode 100644 index 00000000000..1ccfa4924be --- /dev/null +++ b/tests/queries/0_stateless/03267_join_swap_bug.sql @@ -0,0 +1,33 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO TABLE t0 (c0) VALUES (1); + +SELECT 1 FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; +SELECT count() FROM t0 PASTE JOIN (SELECT 1 c0) tx PASTE JOIN t0 t1 GROUP BY tx.c0; + +SET allow_experimental_analyzer = 1; + +SELECT * +FROM +( + SELECT * + FROM system.one +) AS a +INNER JOIN +( + SELECT * + FROM system.one +) AS b USING (dummy) +INNER JOIN +( + SELECT * + FROM system.one +) AS c USING (dummy) +SETTINGS join_algorithm = 'full_sorting_merge'; + + +SELECT count(1) +FROM ( SELECT 1 AS x, x ) AS t1 +RIGHT JOIN (SELECT materialize(2) AS x) AS t2 +ON t1.x = t2.x +; From cfecdd60dda82abe50898360e4313c87051f16d2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 12 Nov 2024 17:12:07 +0100 Subject: [PATCH 099/433] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index f87052e5298..b1b686dfa52 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -77,7 +77,7 @@ static std::initializer_list Date: Tue, 12 Nov 2024 17:13:49 +0100 Subject: [PATCH 100/433] Update Settings.cpp --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 62423fca71b..e30663ae51d 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1223,7 +1223,7 @@ If true then data can be parsed directly to columns with custom serialization (e )", 0) \ \ DECLARE(Bool, merge_tree_use_v1_object_and_dynamic_serialization, false, R"( -When enabled, V1 serialization version of JSON and Dynamic types will be used in MergeTree instead of V2. +When enabled, V1 serialization version of JSON and Dynamic types will be used in MergeTree instead of V2. Changing this setting takes affect only after server restart. )", 0) \ DECLARE(UInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), R"( If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. From 7865eb1ce89f476ed19399e1d4636c164ae79b36 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Nov 2024 18:30:03 +0000 Subject: [PATCH 101/433] style --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 2 +- src/Processors/QueryPlan/JoinStep.h | 2 +- tests/clickhouse-test | 4 +--- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c4dda82a40e..5d95bec7aff 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -73,7 +73,7 @@ static std::initializer_listgetOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) { - /// We should add all duplicated columns, because join algorithm add either all colum with specified name or none + /// We should add all duplicated columns, because join algorithm add either all column with specified name or none auto set_used_column_with_duplicates = [&](const NamesAndTypesList & columns, JoinTableSide join_table_side) { const auto & column_name = columns.front().name; diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 4b9b8e825f8..cd171dbce87 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -37,7 +37,7 @@ public: void setJoin(JoinPtr join_, bool swap_streams_ = false); bool allowPushDownToRight() const; - /// Swap automatically if not set, othewise always or never, depending on the value + /// Swap automatically if not set, otherwise always or never, depending on the value std::optional swap_join_tables = {}; private: diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6ece1eb7f4b..7aaffc07e49 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -920,9 +920,7 @@ class SettingsRandomizer: "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), "parallel_replicas_local_plan": lambda: random.randint(0, 1), - "query_plan_join_swap_table": lambda: random.choice( - ["auto", "false", "true"] - ), + "query_plan_join_swap_table": lambda: random.choice(["auto", "false", "true"]), "output_format_native_write_json_as_string": lambda: random.randint(0, 1), "enable_vertical_final": lambda: random.randint(0, 1), } From ffd1183a1a28e0f82a3eebb34f53f16139733ef4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Nov 2024 10:27:10 +0000 Subject: [PATCH 102/433] remove join_use_nulls_for_system_columns --- src/Core/Settings.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9d2f448fb81..13589b56779 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1919,8 +1919,6 @@ DECLARE(BoolAuto, query_plan_join_swap_table, {}, R"( - 'true': Always swap tables (the left table is the build table). )", 0) \ \ - DECLARE(Bool, join_use_nulls_for_system_columns, false, R"( -)", 0) \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. )", 0) \ From 1c350a968c35212f7f18117a71220348fcfac2b7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Nov 2024 12:54:30 +0000 Subject: [PATCH 103/433] fix --- src/Processors/QueryPlan/JoinStep.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index cd171dbce87..fcebd5d64d7 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -38,7 +38,7 @@ public: bool allowPushDownToRight() const; /// Swap automatically if not set, otherwise always or never, depending on the value - std::optional swap_join_tables = {}; + std::optional swap_join_tables = false; private: void updateOutputHeader() override; From 0e70a375dc4462834947ec503a73b9e2c6be4c39 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Nov 2024 13:19:12 +0000 Subject: [PATCH 104/433] Restart CI From c73663b74baa45881027adb1b01f8ad36186e5fd Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 13 Nov 2024 14:23:21 +0000 Subject: [PATCH 105/433] upd reference --- .../03036_join_filter_push_down_equivalent_sets.reference | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 821f2d8a4dd..1bea145c50a 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -683,6 +683,7 @@ Positions: 4 1 0 2 Header: __table1.id UInt64 __table1.value String __table2.id UInt64 + __table2.value String AND column: equals(__table1.id, 5_UInt8) Actions: INPUT : 0 -> __table1.id UInt64 : 0 COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1 @@ -691,11 +692,11 @@ Positions: 4 1 0 2 Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed) Actions: INPUT :: 1 -> __table1.id UInt64 : 0 INPUT :: 2 -> __table1.value String : 1 - INPUT :: 3 -> __table2.value String : 2 - INPUT : 4 -> __table2.id UInt64 : 3 + INPUT : 3 -> __table2.id UInt64 : 2 + INPUT :: 4 -> __table2.value String : 3 COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4 INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5 - FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 + FUNCTION equals(__table2.id : 2, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6 FUNCTION and(equals(__table1.id, 5_UInt8) :: 5, equals(__table2.id, 6_UInt8) :: 6) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 4 Positions: 4 0 1 2 3 Join (JOIN FillRightFirst) From 650d98c8b656dc87798c118e7feb4551ffb9cdc7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 14 Nov 2024 09:04:35 +0800 Subject: [PATCH 106/433] set settings back --- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index ab1280eb8ac..3e2901c6cb3 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -883,7 +883,7 @@ Allows or restricts using [Variant](../../sql-reference/data-types/variant.md) a DECLARE(Bool, compile_expressions, false, R"( Compile some scalar functions and operators to native code. Due to a bug in the LLVM compiler infrastructure, on AArch64 machines, it is known to lead to a nullptr dereference and, consequently, server crash. Do not enable this setting. )", 0) \ - DECLARE(UInt64, min_count_to_compile_expression, 1, R"( + DECLARE(UInt64, min_count_to_compile_expression, 3, R"( Minimum count of executing same expression before it is get compiled. )", 0) \ DECLARE(Bool, compile_aggregate_expressions, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index af1145f5ccc..da95632ea9e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -122,8 +122,6 @@ static std::initializer_list Date: Thu, 14 Nov 2024 13:37:42 +0100 Subject: [PATCH 107/433] fix index_granularity_bytes to make test stable --- tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index a2198086b10..00a274caa66 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -9,7 +9,7 @@ CREATE TABLE url_na_log ENGINE = MergeTree PRIMARY KEY SiteId ORDER BY (SiteId, DateVisit) -SETTINGS index_granularity = 1000, min_bytes_for_wide_part = 0; +SETTINGS index_granularity_bytes = 1000000, index_granularity = 1000, min_bytes_for_wide_part = 0; CREATE ROW POLICY url_na_log_policy0 ON url_na_log FOR SELECT USING (DateVisit < '2022-08-11') OR (DateVisit > '2022-08-19') TO default; From 2c0f9dbfc5911a9fe69cd5e995922827b5a7d3b1 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Nov 2024 13:22:06 +0000 Subject: [PATCH 108/433] Fix Dynamic serialization in Pretty JSON formats --- src/DataTypes/Serializations/SerializationDynamic.cpp | 8 ++++++-- .../03273_dynamic_pretty_json_serialization.reference | 6 ++++++ .../03273_dynamic_pretty_json_serialization.sql | 6 ++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.reference create mode 100644 tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.sql diff --git a/src/DataTypes/Serializations/SerializationDynamic.cpp b/src/DataTypes/Serializations/SerializationDynamic.cpp index 91c8797d43f..1b999819f79 100644 --- a/src/DataTypes/Serializations/SerializationDynamic.cpp +++ b/src/DataTypes/Serializations/SerializationDynamic.cpp @@ -762,8 +762,12 @@ void SerializationDynamic::serializeTextJSON(const IColumn & column, size_t row_ void SerializationDynamic::serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const { - const auto & dynamic_column = assert_cast(column); - dynamic_column.getVariantInfo().variant_type->getDefaultSerialization()->serializeTextJSONPretty(dynamic_column.getVariantColumn(), row_num, ostr, settings, indent); + auto nested_serialize = [&settings, indent](const ISerialization & serialization, const IColumn & col, size_t row, WriteBuffer & buf) + { + serialization.serializeTextJSONPretty(col, row, buf, settings, indent); + }; + + serializeTextImpl(column, row_num, ostr, settings, nested_serialize); } void SerializationDynamic::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const diff --git a/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.reference b/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.reference new file mode 100644 index 00000000000..8d89910f707 --- /dev/null +++ b/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.reference @@ -0,0 +1,6 @@ +{ + "d": "Hello" +} +{ + "d": "Hello" +} diff --git a/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.sql b/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.sql new file mode 100644 index 00000000000..0c5ff73d8d4 --- /dev/null +++ b/tests/queries/0_stateless/03273_dynamic_pretty_json_serialization.sql @@ -0,0 +1,6 @@ +set allow_experimental_dynamic_type = 1; + +select 'Hello'::Dynamic(max_types=1) as d format PrettyJSONEachRow; +select 'Hello'::Dynamic(max_types=0) as d format PrettyJSONEachRow; + + From d92383feffb802689bc11e6d0def69122abc1061 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Nov 2024 09:44:52 +0000 Subject: [PATCH 109/433] fix --- src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp index 90e1f633899..ca858559886 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp @@ -57,10 +57,9 @@ void optimizeJoin(QueryPlan::Node & node, QueryPlan::Nodes &) const auto & table_join = join->getTableJoin(); - if (table_join.kind() != JoinKind::Inner - && table_join.kind() != JoinKind::Left - && table_join.kind() != JoinKind::Right - && table_join.kind() != JoinKind::Full) + /// Algorithms other than HashJoin may not support all JOIN kinds, so changing from LEFT to RIGHT is not always possible + bool allow_outer_join = typeid_cast(join.get()); + if (table_join.kind() != JoinKind::Inner && !allow_outer_join) return; /// fixme: USING clause handled specially in join algorithm, so swap breaks it From fcaa058c9271658a9e0bf180eaa2d6f0b298746d Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Sat, 16 Nov 2024 00:28:54 +0400 Subject: [PATCH 110/433] get rid of poco mongodb integration implementation --- base/poco/CMakeLists.txt | 5 - base/poco/MongoDB/CMakeLists.txt | 16 - .../poco/MongoDB/include/Poco/MongoDB/Array.h | 142 ----- .../MongoDB/include/Poco/MongoDB/BSONReader.h | 88 --- .../MongoDB/include/Poco/MongoDB/BSONWriter.h | 76 --- .../MongoDB/include/Poco/MongoDB/Binary.h | 158 ----- .../MongoDB/include/Poco/MongoDB/Connection.h | 191 ------ .../MongoDB/include/Poco/MongoDB/Cursor.h | 80 --- .../MongoDB/include/Poco/MongoDB/Database.h | 233 ------- .../include/Poco/MongoDB/DeleteRequest.h | 116 ---- .../MongoDB/include/Poco/MongoDB/Document.h | 296 --------- .../MongoDB/include/Poco/MongoDB/Element.h | 393 ------------ .../include/Poco/MongoDB/GetMoreRequest.h | 92 --- .../include/Poco/MongoDB/InsertRequest.h | 100 --- .../include/Poco/MongoDB/JavaScriptCode.h | 108 ---- .../include/Poco/MongoDB/KillCursorsRequest.h | 65 -- .../MongoDB/include/Poco/MongoDB/Message.h | 76 --- .../include/Poco/MongoDB/MessageHeader.h | 140 ----- .../MongoDB/include/Poco/MongoDB/MongoDB.h | 64 -- .../MongoDB/include/Poco/MongoDB/ObjectId.h | 151 ----- .../include/Poco/MongoDB/OpMsgCursor.h | 96 --- .../include/Poco/MongoDB/OpMsgMessage.h | 163 ----- .../Poco/MongoDB/PoolableConnectionFactory.h | 123 ---- .../include/Poco/MongoDB/QueryRequest.h | 190 ------ .../include/Poco/MongoDB/RegularExpression.h | 135 ---- .../MongoDB/include/Poco/MongoDB/ReplicaSet.h | 61 -- .../include/Poco/MongoDB/RequestMessage.h | 54 -- .../include/Poco/MongoDB/ResponseMessage.h | 114 ---- .../include/Poco/MongoDB/UpdateRequest.h | 117 ---- base/poco/MongoDB/src/Array.cpp | 75 --- base/poco/MongoDB/src/Binary.cpp | 89 --- base/poco/MongoDB/src/Connection.cpp | 348 ----------- base/poco/MongoDB/src/Cursor.cpp | 83 --- base/poco/MongoDB/src/Database.cpp | 482 --------------- base/poco/MongoDB/src/DeleteRequest.cpp | 54 -- base/poco/MongoDB/src/Document.cpp | 227 ------- base/poco/MongoDB/src/Element.cpp | 32 - base/poco/MongoDB/src/GetMoreRequest.cpp | 46 -- base/poco/MongoDB/src/InsertRequest.cpp | 49 -- base/poco/MongoDB/src/JavaScriptCode.cpp | 33 - base/poco/MongoDB/src/KillCursorsRequest.cpp | 44 -- base/poco/MongoDB/src/Message.cpp | 33 - base/poco/MongoDB/src/MessageHeader.cpp | 63 -- base/poco/MongoDB/src/ObjectId.cpp | 66 -- base/poco/MongoDB/src/OpMsgCursor.cpp | 187 ------ base/poco/MongoDB/src/OpMsgMessage.cpp | 412 ------------- base/poco/MongoDB/src/QueryRequest.cpp | 54 -- base/poco/MongoDB/src/RegularExpression.cpp | 71 --- base/poco/MongoDB/src/ReplicaSet.cpp | 89 --- base/poco/MongoDB/src/RequestMessage.cpp | 51 -- base/poco/MongoDB/src/ResponseMessage.cpp | 80 --- base/poco/MongoDB/src/UpdateRequest.cpp | 47 -- .../table-engines/integrations/mongodb.md | 5 - .../settings.md | 8 - programs/format/Format.cpp | 4 +- programs/local/LocalServer.cpp | 7 +- programs/server/Server.cpp | 7 +- src/CMakeLists.txt | 4 - src/Core/ServerSettings.cpp | 1 - src/Core/Settings.cpp | 2 +- src/Dictionaries/CMakeLists.txt | 4 - .../MongoDBPocoLegacyDictionarySource.cpp | 305 --------- .../MongoDBPocoLegacyDictionarySource.h | 93 --- src/Dictionaries/registerDictionaries.cpp | 7 +- src/Dictionaries/registerDictionaries.h | 2 +- .../Sources/MongoDBPocoLegacySource.cpp | 578 ------------------ .../Sources/MongoDBPocoLegacySource.h | 92 --- src/Storages/StorageMongoDBPocoLegacy.cpp | 327 ---------- src/Storages/StorageMongoDBPocoLegacy.h | 79 --- .../StorageMongoDBPocoLegacySocketFactory.cpp | 57 -- .../StorageMongoDBPocoLegacySocketFactory.h | 24 - src/Storages/registerStorages.cpp | 7 +- src/Storages/registerStorages.h | 2 +- .../TableFunctionMongoDBPocoLegacy.cpp | 128 ---- src/TableFunctions/registerTableFunctions.cpp | 7 +- src/TableFunctions/registerTableFunctions.h | 2 +- tests/integration/helpers/external_sources.py | 10 +- .../configs/mongo/legacy.xml | 3 - .../configs/mongo/new.xml | 3 - .../test_mongo.py | 37 +- .../test_mongo_uri.py | 30 +- .../configs/feature_flag.xml | 3 - .../integration/test_storage_mongodb/test.py | 1 - .../test_storage_mongodb_legacy/__init__.py | 0 .../configs/feature_flag.xml | 3 - .../configs/named_collections.xml | 12 - .../configs/users.xml | 9 - .../mongo_secure_config/cert.crt | 24 - .../mongo_secure_config/config.d/ssl_conf.xml | 8 - .../mongo_secure_config/key.pem | 52 -- .../mongo_secure_config/mongo_cert.pem | 52 -- .../mongo_secure_config/mongo_secure.conf | 6 - .../test_storage_mongodb_legacy/test.py | 509 --------------- .../configs/feature_flag.xml | 3 - .../test_table_function_mongodb/test.py | 1 - .../__init__.py | 0 .../configs/feature_flag.xml | 3 - .../configs/users.xml | 9 - .../mongo_secure_config/cert.crt | 24 - .../mongo_secure_config/config.d/ssl_conf.xml | 8 - .../mongo_secure_config/key.pem | 52 -- .../mongo_secure_config/mongo_secure.conf | 6 - .../test.py | 276 --------- ...new_table_functions_must_be_documented.sql | 1 - 104 files changed, 36 insertions(+), 9219 deletions(-) delete mode 100644 base/poco/MongoDB/CMakeLists.txt delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Array.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/BSONReader.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/BSONWriter.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Binary.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Connection.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Cursor.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Database.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Document.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Element.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/GetMoreRequest.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/InsertRequest.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/KillCursorsRequest.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/Message.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/QueryRequest.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/ReplicaSet.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/RequestMessage.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h delete mode 100644 base/poco/MongoDB/include/Poco/MongoDB/UpdateRequest.h delete mode 100644 base/poco/MongoDB/src/Array.cpp delete mode 100644 base/poco/MongoDB/src/Binary.cpp delete mode 100644 base/poco/MongoDB/src/Connection.cpp delete mode 100644 base/poco/MongoDB/src/Cursor.cpp delete mode 100644 base/poco/MongoDB/src/Database.cpp delete mode 100644 base/poco/MongoDB/src/DeleteRequest.cpp delete mode 100644 base/poco/MongoDB/src/Document.cpp delete mode 100644 base/poco/MongoDB/src/Element.cpp delete mode 100644 base/poco/MongoDB/src/GetMoreRequest.cpp delete mode 100644 base/poco/MongoDB/src/InsertRequest.cpp delete mode 100644 base/poco/MongoDB/src/JavaScriptCode.cpp delete mode 100644 base/poco/MongoDB/src/KillCursorsRequest.cpp delete mode 100644 base/poco/MongoDB/src/Message.cpp delete mode 100644 base/poco/MongoDB/src/MessageHeader.cpp delete mode 100644 base/poco/MongoDB/src/ObjectId.cpp delete mode 100644 base/poco/MongoDB/src/OpMsgCursor.cpp delete mode 100644 base/poco/MongoDB/src/OpMsgMessage.cpp delete mode 100644 base/poco/MongoDB/src/QueryRequest.cpp delete mode 100644 base/poco/MongoDB/src/RegularExpression.cpp delete mode 100644 base/poco/MongoDB/src/ReplicaSet.cpp delete mode 100644 base/poco/MongoDB/src/RequestMessage.cpp delete mode 100644 base/poco/MongoDB/src/ResponseMessage.cpp delete mode 100644 base/poco/MongoDB/src/UpdateRequest.cpp delete mode 100644 src/Dictionaries/MongoDBPocoLegacyDictionarySource.cpp delete mode 100644 src/Dictionaries/MongoDBPocoLegacyDictionarySource.h delete mode 100644 src/Processors/Sources/MongoDBPocoLegacySource.cpp delete mode 100644 src/Processors/Sources/MongoDBPocoLegacySource.h delete mode 100644 src/Storages/StorageMongoDBPocoLegacy.cpp delete mode 100644 src/Storages/StorageMongoDBPocoLegacy.h delete mode 100644 src/Storages/StorageMongoDBPocoLegacySocketFactory.cpp delete mode 100644 src/Storages/StorageMongoDBPocoLegacySocketFactory.h delete mode 100644 src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/legacy.xml delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/new.xml delete mode 100644 tests/integration/test_storage_mongodb/configs/feature_flag.xml delete mode 100644 tests/integration/test_storage_mongodb_legacy/__init__.py delete mode 100644 tests/integration/test_storage_mongodb_legacy/configs/feature_flag.xml delete mode 100644 tests/integration/test_storage_mongodb_legacy/configs/named_collections.xml delete mode 100644 tests/integration/test_storage_mongodb_legacy/configs/users.xml delete mode 100644 tests/integration/test_storage_mongodb_legacy/mongo_secure_config/cert.crt delete mode 100644 tests/integration/test_storage_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml delete mode 100644 tests/integration/test_storage_mongodb_legacy/mongo_secure_config/key.pem delete mode 100644 tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_cert.pem delete mode 100644 tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_secure.conf delete mode 100644 tests/integration/test_storage_mongodb_legacy/test.py delete mode 100644 tests/integration/test_table_function_mongodb/configs/feature_flag.xml delete mode 100644 tests/integration/test_table_function_mongodb_legacy/__init__.py delete mode 100644 tests/integration/test_table_function_mongodb_legacy/configs/feature_flag.xml delete mode 100644 tests/integration/test_table_function_mongodb_legacy/configs/users.xml delete mode 100644 tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/cert.crt delete mode 100644 tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml delete mode 100644 tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/key.pem delete mode 100644 tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/mongo_secure.conf delete mode 100644 tests/integration/test_table_function_mongodb_legacy/test.py diff --git a/base/poco/CMakeLists.txt b/base/poco/CMakeLists.txt index 434e24cf334..8cc66ff403c 100644 --- a/base/poco/CMakeLists.txt +++ b/base/poco/CMakeLists.txt @@ -3,11 +3,6 @@ add_subdirectory (Data) add_subdirectory (Data/ODBC) add_subdirectory (Foundation) add_subdirectory (JSON) - -if (USE_MONGODB) - add_subdirectory(MongoDB) -endif() - add_subdirectory (Net) add_subdirectory (NetSSL_OpenSSL) add_subdirectory (Redis) diff --git a/base/poco/MongoDB/CMakeLists.txt b/base/poco/MongoDB/CMakeLists.txt deleted file mode 100644 index bb6f90ed8f5..00000000000 --- a/base/poco/MongoDB/CMakeLists.txt +++ /dev/null @@ -1,16 +0,0 @@ -file (GLOB SRCS src/*.cpp) - -add_library (_poco_mongodb ${SRCS}) -add_library (Poco::MongoDB ALIAS _poco_mongodb) - -# TODO: remove these warning exclusions -target_compile_options (_poco_mongodb - PRIVATE - -Wno-old-style-cast - -Wno-unused-parameter - -Wno-zero-as-null-pointer-constant -) - -target_include_directories (_poco_mongodb SYSTEM PUBLIC "include") -target_link_libraries (_poco_mongodb PUBLIC Poco::Net) - diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Array.h b/base/poco/MongoDB/include/Poco/MongoDB/Array.h deleted file mode 100644 index 8a30c785b2d..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Array.h +++ /dev/null @@ -1,142 +0,0 @@ -// -// Array.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Array -// -// Definition of the Array class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Array_INCLUDED -#define MongoDB_Array_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/NumberFormatter.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Array : public Document - /// This class represents a BSON Array. - { - public: - using Ptr = SharedPtr; - - Array(); - /// Creates an empty Array. - - virtual ~Array(); - /// Destroys the Array. - - // Document template functions available for backward compatibility - using Document::add; - using Document::get; - - template - Document & add(T value) - /// Creates an element with the name from the current pos and value and - /// adds it to the array document. - /// - /// The active document is returned to allow chaining of the add methods. - { - return Document::add(Poco::NumberFormatter::format(size()), value); - } - - Document & add(const char * value) - /// Creates an element with a name from the current pos and value and - /// adds it to the array document. - /// - /// The active document is returned to allow chaining of the add methods. - { - return Document::add(Poco::NumberFormatter::format(size()), value); - } - - template - T get(std::size_t pos) const - /// Returns the element at the given index and tries to convert - /// it to the template type. If the element is not found, a - /// Poco::NotFoundException will be thrown. If the element cannot be - /// converted a BadCastException will be thrown. - { - return Document::get(Poco::NumberFormatter::format(pos)); - } - - template - T get(std::size_t pos, const T & deflt) const - /// Returns the element at the given index and tries to convert - /// it to the template type. If the element is not found, or - /// has the wrong type, the deflt argument will be returned. - { - return Document::get(Poco::NumberFormatter::format(pos), deflt); - } - - Element::Ptr get(std::size_t pos) const; - /// Returns the element at the given index. - /// An empty element will be returned if the element is not found. - - template - bool isType(std::size_t pos) const - /// Returns true if the type of the element equals the TypeId of ElementTrait, - /// otherwise false. - { - return Document::isType(Poco::NumberFormatter::format(pos)); - } - - std::string toString(int indent = 0) const; - /// Returns a string representation of the Array. - - private: - friend void BSONReader::read(Array::Ptr & to); - }; - - - // BSON Embedded Array - // spec: document - template <> - struct ElementTraits - { - enum - { - TypeId = 0x04 - }; - - static std::string toString(const Array::Ptr & value, int indent = 0) - { - //TODO: - return value.isNull() ? "null" : value->toString(indent); - } - }; - - - template <> - inline void BSONReader::read(Array::Ptr & to) - { - to->read(_reader); - } - - - template <> - inline void BSONWriter::write(Array::Ptr & from) - { - from->write(_writer); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Array_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/BSONReader.h b/base/poco/MongoDB/include/Poco/MongoDB/BSONReader.h deleted file mode 100644 index 5858226ee49..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/BSONReader.h +++ /dev/null @@ -1,88 +0,0 @@ -// -// BSONReader.h -// -// Library: MongoDB -// Package: MongoDB -// Module: BSONReader -// -// Definition of the BSONReader class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_BSONReader_INCLUDED -#define MongoDB_BSONReader_INCLUDED - - -#include "Poco/BinaryReader.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API BSONReader - /// Class for reading BSON using a Poco::BinaryReader - { - public: - BSONReader(const Poco::BinaryReader & reader) : _reader(reader) - /// Creates the BSONReader using the given BinaryWriter. - { - } - - virtual ~BSONReader() - /// Destroys the BSONReader. - { - } - - template - void read(T & t) - /// Reads the value from the reader. The default implementation uses the >> operator to - /// the given argument. Special types can write their own version. - { - _reader >> t; - } - - std::string readCString(); - /// Reads a cstring from the reader. - /// A cstring is a string terminated with a 0x00. - - private: - Poco::BinaryReader _reader; - }; - - - // - // inlines - // - inline std::string BSONReader::readCString() - { - std::string val; - while (_reader.good()) - { - char c; - _reader >> c; - if (_reader.good()) - { - if (c == 0x00) - return val; - else - val += c; - } - } - return val; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_BSONReader_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/BSONWriter.h b/base/poco/MongoDB/include/Poco/MongoDB/BSONWriter.h deleted file mode 100644 index 0a4e6e371b9..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/BSONWriter.h +++ /dev/null @@ -1,76 +0,0 @@ -// -// BSONWriter.h -// -// Library: MongoDB -// Package: MongoDB -// Module: BSONWriter -// -// Definition of the BSONWriter class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_BSONWriter_INCLUDED -#define MongoDB_BSONWriter_INCLUDED - - -#include "Poco/BinaryWriter.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API BSONWriter - /// Class for writing BSON using a Poco::BinaryWriter. - { - public: - BSONWriter(const Poco::BinaryWriter & writer) : _writer(writer) - /// Creates the BSONWriter. - { - } - - virtual ~BSONWriter() - /// Destroys the BSONWriter. - { - } - - template - void write(T & t) - /// Writes the value to the writer. The default implementation uses - /// the << operator. Special types can write their own version. - { - _writer << t; - } - - void writeCString(const std::string & value); - /// Writes a cstring to the writer. A cstring is a string - /// terminated a null character. - - private: - Poco::BinaryWriter _writer; - }; - - - // - // inlines - // - inline void BSONWriter::writeCString(const std::string & value) - { - _writer.writeRaw(value); - _writer << (unsigned char)0x00; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_BSONWriter_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Binary.h b/base/poco/MongoDB/include/Poco/MongoDB/Binary.h deleted file mode 100644 index aad8736e8b6..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Binary.h +++ /dev/null @@ -1,158 +0,0 @@ -// -// Binary.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Binary -// -// Definition of the Binary class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Binary_INCLUDED -#define MongoDB_Binary_INCLUDED - - -#include -#include "Poco/Base64Encoder.h" -#include "Poco/Buffer.h" -#include "Poco/MemoryStream.h" -#include "Poco/MongoDB/Element.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/StreamCopier.h" -#include "Poco/UUID.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Binary - /// Implements BSON Binary. - /// - /// A Binary stores its data in a Poco::Buffer. - { - public: - using Ptr = SharedPtr; - - Binary(); - /// Creates an empty Binary with subtype 0. - - Binary(Poco::Int32 size, unsigned char subtype); - /// Creates a Binary with a buffer of the given size and the given subtype. - - Binary(const UUID & uuid); - /// Creates a Binary containing an UUID. - - Binary(const std::string & data, unsigned char subtype = 0); - /// Creates a Binary with the contents of the given string and the given subtype. - - Binary(const void * data, Poco::Int32 size, unsigned char subtype = 0); - /// Creates a Binary with the contents of the given buffer and the given subtype. - - virtual ~Binary(); - /// Destroys the Binary. - - Buffer & buffer(); - /// Returns a reference to the internal buffer - - unsigned char subtype() const; - /// Returns the subtype. - - void subtype(unsigned char type); - /// Sets the subtype. - - std::string toString(int indent = 0) const; - /// Returns the contents of the Binary as Base64-encoded string. - - std::string toRawString() const; - /// Returns the raw content of the Binary as a string. - - UUID uuid() const; - /// Returns the UUID when the binary subtype is 0x04. - /// Otherwise, throws a Poco::BadCastException. - - private: - Buffer _buffer; - unsigned char _subtype; - }; - - - // - // inlines - // - inline unsigned char Binary::subtype() const - { - return _subtype; - } - - - inline void Binary::subtype(unsigned char type) - { - _subtype = type; - } - - - inline Buffer & Binary::buffer() - { - return _buffer; - } - - - inline std::string Binary::toRawString() const - { - return std::string(reinterpret_cast(_buffer.begin()), _buffer.size()); - } - - - // BSON Embedded Document - // spec: binary - template <> - struct ElementTraits - { - enum - { - TypeId = 0x05 - }; - - static std::string toString(const Binary::Ptr & value, int indent = 0) { return value.isNull() ? "" : value->toString(); } - }; - - - template <> - inline void BSONReader::read(Binary::Ptr & to) - { - Poco::Int32 size; - _reader >> size; - - to->buffer().resize(size); - - unsigned char subtype; - _reader >> subtype; - to->subtype(subtype); - - _reader.readRaw((char *)to->buffer().begin(), size); - } - - - template <> - inline void BSONWriter::write(Binary::Ptr & from) - { - _writer << (Poco::Int32)from->buffer().size(); - _writer << from->subtype(); - _writer.writeRaw((char *)from->buffer().begin(), from->buffer().size()); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Binary_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h deleted file mode 100644 index cf679d530aa..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ /dev/null @@ -1,191 +0,0 @@ -// -// Connection.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Connection -// -// Definition of the Connection class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Connection_INCLUDED -#define MongoDB_Connection_INCLUDED - - -#include "Poco/MongoDB/OpMsgMessage.h" -#include "Poco/MongoDB/RequestMessage.h" -#include "Poco/MongoDB/ResponseMessage.h" -#include "Poco/Mutex.h" -#include "Poco/Net/SocketAddress.h" -#include "Poco/Net/StreamSocket.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Connection - /// Represents a connection to a MongoDB server - /// using the MongoDB wire protocol. - /// - /// See https://docs.mongodb.com/manual/reference/mongodb-wire-protocol/ - /// for more information on the wire protocol. - { - public: - using Ptr = Poco::SharedPtr; - - class MongoDB_API SocketFactory - { - public: - SocketFactory(); - /// Creates the SocketFactory. - - virtual ~SocketFactory(); - /// Destroys the SocketFactory. - - virtual Poco::Net::StreamSocket createSocket(const std::string & host, int port, Poco::Timespan connectTimeout, bool secure); - /// Creates a Poco::Net::StreamSocket (if secure is false), or a - /// Poco::Net::SecureStreamSocket (if secure is true) connected to the - /// given host and port number. - /// - /// The default implementation will throw a Poco::NotImplementedException - /// if secure is true. - }; - - Connection(); - /// Creates an unconnected Connection. - /// - /// Use this when you want to connect later on. - - Connection(const std::string & hostAndPort); - /// Creates a Connection connected to the given MongoDB instance at host:port. - /// - /// The host and port must be separated with a colon. - - Connection(const std::string & uri, SocketFactory & socketFactory); - /// Creates a Connection connected to the given MongoDB instance at the - /// given URI. - /// - /// See the corresponding connect() method for more information. - - Connection(const std::string & host, int port); - /// Creates a Connection connected to the given MongoDB instance at host and port. - - Connection(const Poco::Net::SocketAddress & addrs); - /// Creates a Connection connected to the given MongoDB instance at the given address. - - Connection(const Poco::Net::StreamSocket & socket); - /// Creates a Connection connected to the given MongoDB instance using the given socket, - /// which must already be connected. - - virtual ~Connection(); - /// Destroys the Connection. - - Poco::Net::SocketAddress address() const; - /// Returns the address of the MongoDB server. - - const std::string & uri() const; - /// Returns the uri on which the connection was made. - - void connect(const std::string & hostAndPort); - /// Connects to the given MongoDB server. - /// - /// The host and port must be separated with a colon. - - void connect(const std::string & uri, SocketFactory & socketFactory); - /// Connects to the given MongoDB instance at the given URI. - /// - /// The URI must be in standard MongoDB connection string URI format: - /// - /// mongodb://:@hostname.com:/database-name?options - /// - /// The following options are supported: - /// - /// - ssl: If ssl=true is specified, a custom SocketFactory subclass creating - /// a SecureStreamSocket must be supplied. - /// - connectTimeoutMS: Socket connection timeout in milliseconds. - /// - socketTimeoutMS: Socket send/receive timeout in milliseconds. - /// - authMechanism: Authentication mechanism. Only "SCRAM-SHA-1" (default) - /// and "MONGODB-CR" are supported. - /// - /// Unknown options are silently ignored. - /// - /// Will also attempt to authenticate using the specified credentials, - /// using Database::authenticate(). - /// - /// Throws a Poco::NoPermissionException if authentication fails. - - void connect(const std::string & host, int port); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::SocketAddress & addrs); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::StreamSocket & socket); - /// Connects using an already connected socket. - - void disconnect(); - /// Disconnects from the MongoDB server. - - void sendRequest(RequestMessage & request); - /// Sends a request to the MongoDB server. - /// - /// Used for one-way requests without a response. - - void sendRequest(RequestMessage & request, ResponseMessage & response); - /// Sends a request to the MongoDB server and receives the response. - /// - /// Use this when a response is expected: only a "query" or "getmore" - /// request will return a response. - - void sendRequest(OpMsgMessage & request, OpMsgMessage & response); - /// Sends a request to the MongoDB server and receives the response - /// using newer wire protocol with OP_MSG. - - void sendRequest(OpMsgMessage & request); - /// Sends an unacknowledged request to the MongoDB server using newer - /// wire protocol with OP_MSG. - /// No response is sent by the server. - - void readResponse(OpMsgMessage & response); - /// Reads additional response data when previous message's flag moreToCome - /// indicates that server will send more data. - /// NOTE: See comments in OpMsgCursor code. - - - protected: - void connect(); - - private: - Poco::Net::SocketAddress _address; - Poco::Net::StreamSocket _socket; - std::string _uri; - }; - - - // - // inlines - // - inline Net::SocketAddress Connection::address() const - { - return _address; - } - inline const std::string & Connection::uri() const - { - return _uri; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Connection_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h b/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h deleted file mode 100644 index 8849d737a62..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Cursor.h +++ /dev/null @@ -1,80 +0,0 @@ -// -// Cursor.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Cursor -// -// Definition of the Cursor class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Cursor_INCLUDED -#define MongoDB_Cursor_INCLUDED - - -#include "Poco/MongoDB/Connection.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/QueryRequest.h" -#include "Poco/MongoDB/ResponseMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Cursor : public Document - /// Cursor is an helper class for querying multiple documents. - { - public: - Cursor(const std::string & dbname, const std::string & collectionName, QueryRequest::Flags flags = QueryRequest::QUERY_DEFAULT); - /// Creates a Cursor for the given database and collection, using the specified flags. - - Cursor(const std::string & fullCollectionName, QueryRequest::Flags flags = QueryRequest::QUERY_DEFAULT); - /// Creates a Cursor for the given database and collection ("database.collection"), using the specified flags. - - Cursor(const Document & aggregationResponse); - /// Creates a Cursor for the given aggregation query response. - - virtual ~Cursor(); - /// Destroys the Cursor. - - ResponseMessage & next(Connection & connection); - /// Tries to get the next documents. As long as ResponseMessage has a - /// cursor ID next can be called to retrieve the next bunch of documents. - /// - /// The cursor must be killed (see kill()) when not all documents are needed. - - QueryRequest & query(); - /// Returns the associated query. - - void kill(Connection & connection); - /// Kills the cursor and reset it so that it can be reused. - - private: - QueryRequest _query; - ResponseMessage _response; - }; - - - // - // inlines - // - inline QueryRequest & Cursor::query() - { - return _query; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Cursor_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Database.h b/base/poco/MongoDB/include/Poco/MongoDB/Database.h deleted file mode 100644 index 1fa91f4ca1a..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Database.h +++ /dev/null @@ -1,233 +0,0 @@ -// -// Database.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Database -// -// Definition of the Database class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Database_INCLUDED -#define MongoDB_Database_INCLUDED - - -#include "Poco/MongoDB/Connection.h" -#include "Poco/MongoDB/DeleteRequest.h" -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/InsertRequest.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/QueryRequest.h" -#include "Poco/MongoDB/UpdateRequest.h" - -#include "Poco/MongoDB/OpMsgCursor.h" -#include "Poco/MongoDB/OpMsgMessage.h" - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Database - /// Database is a helper class for creating requests. MongoDB works with - /// collection names and uses the part before the first dot as the name of - /// the database. - { - public: - explicit Database(const std::string & name); - /// Creates a Database for the database with the given name. - - virtual ~Database(); - /// Destroys the Database. - - const std::string & name() const; - /// Database name - - bool authenticate( - Connection & connection, - const std::string & username, - const std::string & password, - const std::string & method = AUTH_SCRAM_SHA1); - /// Authenticates against the database using the given connection, - /// username and password, as well as authentication method. - /// - /// "MONGODB-CR" (default prior to MongoDB 3.0) and - /// "SCRAM-SHA-1" (default starting in 3.0) are the only supported - /// authentication methods. - /// - /// Returns true if authentication was successful, otherwise false. - /// - /// May throw a Poco::ProtocolException if authentication fails for a reason other than - /// invalid credentials. - - Document::Ptr queryBuildInfo(Connection & connection) const; - /// Queries server build info (all wire protocols) - - Document::Ptr queryServerHello(Connection & connection, bool old = false) const; - /// Queries hello response from server (all wire protocols) - - Int64 count(Connection & connection, const std::string & collectionName) const; - /// Sends a count request for the given collection to MongoDB. (old wire protocol) - /// - /// If the command fails, -1 is returned. - - Poco::SharedPtr createCommand() const; - /// Creates a QueryRequest for a command. (old wire protocol) - - Poco::SharedPtr createCountRequest(const std::string & collectionName) const; - /// Creates a QueryRequest to count the given collection. - /// The collectionname must not contain the database name. (old wire protocol) - - Poco::SharedPtr createDeleteRequest(const std::string & collectionName) const; - /// Creates a DeleteRequest to delete documents in the given collection. - /// The collectionname must not contain the database name. (old wire protocol) - - Poco::SharedPtr createInsertRequest(const std::string & collectionName) const; - /// Creates an InsertRequest to insert new documents in the given collection. - /// The collectionname must not contain the database name. (old wire protocol) - - Poco::SharedPtr createQueryRequest(const std::string & collectionName) const; - /// Creates a QueryRequest. (old wire protocol) - /// The collectionname must not contain the database name. - - Poco::SharedPtr createUpdateRequest(const std::string & collectionName) const; - /// Creates an UpdateRequest. (old wire protocol) - /// The collectionname must not contain the database name. - - Poco::SharedPtr createOpMsgMessage(const std::string & collectionName) const; - /// Creates OpMsgMessage. (new wire protocol) - - Poco::SharedPtr createOpMsgMessage() const; - /// Creates OpMsgMessage for database commands that do not require collection as an argument. (new wire protocol) - - Poco::SharedPtr createOpMsgCursor(const std::string & collectionName) const; - /// Creates OpMsgCursor. (new wire protocol) - - Poco::MongoDB::Document::Ptr ensureIndex( - Connection & connection, - const std::string & collection, - const std::string & indexName, - Poco::MongoDB::Document::Ptr keys, - bool unique = false, - bool background = false, - int version = 0, - int ttl = 0); - /// Creates an index. The document returned is the result of a getLastError call. - /// For more info look at the ensureIndex information on the MongoDB website. (old wire protocol) - - Document::Ptr getLastErrorDoc(Connection & connection) const; - /// Sends the getLastError command to the database and returns the error document. - /// (old wire protocol) - - std::string getLastError(Connection & connection) const; - /// Sends the getLastError command to the database and returns the err element - /// from the error document. When err is null, an empty string is returned. - /// (old wire protocol) - - static const std::string AUTH_MONGODB_CR; - /// Default authentication mechanism prior to MongoDB 3.0. - - static const std::string AUTH_SCRAM_SHA1; - /// Default authentication mechanism for MongoDB 3.0. - - enum WireVersion - /// Wire version as reported by the command hello. - /// See details in MongoDB github, repository specifications. - /// @see queryServerHello - { - VER_26 = 1, - VER_26_2 = 2, - VER_30 = 3, - VER_32 = 4, - VER_34 = 5, - VER_36 = 6, ///< First wire version that supports OP_MSG - VER_40 = 7, - VER_42 = 8, - VER_44 = 9, - VER_50 = 13, - VER_51 = 14, ///< First wire version that supports only OP_MSG - VER_52 = 15, - VER_53 = 16, - VER_60 = 17 - }; - - protected: - bool authCR(Connection & connection, const std::string & username, const std::string & password); - bool authSCRAM(Connection & connection, const std::string & username, const std::string & password); - - private: - std::string _dbname; - }; - - - // - // inlines - // - inline const std::string & Database::name() const - { - return _dbname; - } - - - inline Poco::SharedPtr Database::createCommand() const - { - Poco::SharedPtr cmd = createQueryRequest("$cmd"); - cmd->setNumberToReturn(1); - return cmd; - } - - - inline Poco::SharedPtr Database::createDeleteRequest(const std::string & collectionName) const - { - return new Poco::MongoDB::DeleteRequest(_dbname + '.' + collectionName); - } - - - inline Poco::SharedPtr Database::createInsertRequest(const std::string & collectionName) const - { - return new Poco::MongoDB::InsertRequest(_dbname + '.' + collectionName); - } - - - inline Poco::SharedPtr Database::createQueryRequest(const std::string & collectionName) const - { - return new Poco::MongoDB::QueryRequest(_dbname + '.' + collectionName); - } - - - inline Poco::SharedPtr Database::createUpdateRequest(const std::string & collectionName) const - { - return new Poco::MongoDB::UpdateRequest(_dbname + '.' + collectionName); - } - - // -- New wire protocol commands - - inline Poco::SharedPtr Database::createOpMsgMessage(const std::string & collectionName) const - { - return new Poco::MongoDB::OpMsgMessage(_dbname, collectionName); - } - - inline Poco::SharedPtr Database::createOpMsgMessage() const - { - // Collection name for database commands is not needed. - return createOpMsgMessage(""); - } - - inline Poco::SharedPtr Database::createOpMsgCursor(const std::string & collectionName) const - { - return new Poco::MongoDB::OpMsgCursor(_dbname, collectionName); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Database_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h deleted file mode 100644 index a54f2a23121..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h +++ /dev/null @@ -1,116 +0,0 @@ -// -// DeleteRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: DeleteRequest -// -// Definition of the DeleteRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_DeleteRequest_INCLUDED -#define MongoDB_DeleteRequest_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API DeleteRequest : public RequestMessage - /// A DeleteRequest is used to delete one or more documents from a database. - /// - /// Specific flags for this request - /// - DELETE_DEFAULT: default delete operation - /// - DELETE_SINGLE_REMOVE: delete only the first document - { - public: - enum Flags - { - DELETE_DEFAULT = 0, - /// Default - - DELETE_SINGLE_REMOVE = 1 - /// Delete only the first document. - }; - - DeleteRequest(const std::string & collectionName, Flags flags = DELETE_DEFAULT); - /// Creates a DeleteRequest for the given collection using the given flags. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". - - DeleteRequest(const std::string & collectionName, bool justOne); - /// Creates a DeleteRequest for the given collection. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". - /// - /// If justOne is true, only the first matching document will - /// be removed (the same as using flag DELETE_SINGLE_REMOVE). - - virtual ~DeleteRequest(); - /// Destructor - - Flags flags() const; - /// Returns the flags. - - void flags(Flags flag); - /// Sets the flags. - - Document & selector(); - /// Returns the selector document. - - protected: - void buildRequest(BinaryWriter & writer); - /// Writes the OP_DELETE request to the writer. - - private: - Flags _flags; - std::string _fullCollectionName; - Document _selector; - }; - - - /// - /// inlines - /// - inline DeleteRequest::Flags DeleteRequest::flags() const - { - return _flags; - } - - - inline void DeleteRequest::flags(DeleteRequest::Flags flags) - { - _flags = flags; - } - - - inline Document & DeleteRequest::selector() - { - return _selector; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_DeleteRequest_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Document.h b/base/poco/MongoDB/include/Poco/MongoDB/Document.h deleted file mode 100644 index 9e1df349e20..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Document.h +++ /dev/null @@ -1,296 +0,0 @@ -// -// Document.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Document -// -// Definition of the Document class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Document_INCLUDED -#define MongoDB_Document_INCLUDED - - -#include -#include -#include "Poco/BinaryReader.h" -#include "Poco/BinaryWriter.h" -#include "Poco/MongoDB/Element.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - class Array; - - class ElementFindByName - { - public: - ElementFindByName(const std::string & name) : _name(name) { } - - bool operator()(const Element::Ptr & element) { return !element.isNull() && element->name() == _name; } - - private: - std::string _name; - }; - - - class MongoDB_API Document - /// Represents a MongoDB (BSON) document. - { - public: - using Ptr = SharedPtr; - using Vector = std::vector; - - Document(); - /// Creates an empty Document. - - virtual ~Document(); - /// Destroys the Document. - - Document & addElement(Element::Ptr element); - /// Add an element to the document. - /// - /// The active document is returned to allow chaining of the add methods. - - template - Document & add(const std::string & name, T value) - /// Creates an element with the given name and value and - /// adds it to the document. - /// - /// The active document is returned to allow chaining of the add methods. - { - return addElement(new ConcreteElement(name, value)); - } - - Document & add(const std::string & name, const char * value) - /// Creates an element with the given name and value and - /// adds it to the document. - /// - /// The active document is returned to allow chaining of the add methods. - { - return addElement(new ConcreteElement(name, std::string(value))); - } - - Document & addNewDocument(const std::string & name); - /// Create a new document and add it to this document. - /// Unlike the other add methods, this method returns - /// a reference to the new document. - - Array & addNewArray(const std::string & name); - /// Create a new array and add it to this document. - /// Method returns a reference to the new array. - - void clear(); - /// Removes all elements from the document. - - void elementNames(std::vector & keys) const; - /// Puts all element names into std::vector. - - bool empty() const; - /// Returns true if the document doesn't contain any documents. - - bool exists(const std::string & name) const; - /// Returns true if the document has an element with the given name. - - template - T get(const std::string & name) const - /// Returns the element with the given name and tries to convert - /// it to the template type. When the element is not found, a - /// NotFoundException will be thrown. When the element can't be - /// converted a BadCastException will be thrown. - { - Element::Ptr element = get(name); - if (element.isNull()) - { - throw NotFoundException(name); - } - else - { - if (ElementTraits::TypeId == element->type()) - { - ConcreteElement * concrete = dynamic_cast *>(element.get()); - if (concrete != 0) - { - return concrete->value(); - } - } - throw BadCastException("Invalid type mismatch!"); - } - } - - template - T get(const std::string & name, const T & def) const - /// Returns the element with the given name and tries to convert - /// it to the template type. When the element is not found, or - /// has the wrong type, the def argument will be returned. - { - Element::Ptr element = get(name); - if (element.isNull()) - { - return def; - } - - if (ElementTraits::TypeId == element->type()) - { - ConcreteElement * concrete = dynamic_cast *>(element.get()); - if (concrete != 0) - { - return concrete->value(); - } - } - - return def; - } - - Element::Ptr get(const std::string & name) const; - /// Returns the element with the given name. - /// An empty element will be returned when the element is not found. - - Int64 getInteger(const std::string & name) const; - /// Returns an integer. Useful when MongoDB returns Int32, Int64 - /// or double for a number (count for example). This method will always - /// return an Int64. When the element is not found, a - /// Poco::NotFoundException will be thrown. - - bool remove(const std::string & name); - /// Removes an element from the document. - - template - bool isType(const std::string & name) const - /// Returns true when the type of the element equals the TypeId of ElementTrait. - { - Element::Ptr element = get(name); - if (element.isNull()) - { - return false; - } - - return ElementTraits::TypeId == element->type(); - } - - void read(BinaryReader & reader); - /// Reads a document from the reader - - std::size_t size() const; - /// Returns the number of elements in the document. - - virtual std::string toString(int indent = 0) const; - /// Returns a String representation of the document. - - void write(BinaryWriter & writer); - /// Writes a document to the reader - - protected: - ElementSet _elements; - }; - - - // - // inlines - // - inline Document & Document::addElement(Element::Ptr element) - { - _elements.push_back(element); - return *this; - } - - - inline Document & Document::addNewDocument(const std::string & name) - { - Document::Ptr newDoc = new Document(); - add(name, newDoc); - return *newDoc; - } - - - inline void Document::clear() - { - _elements.clear(); - } - - - inline bool Document::empty() const - { - return _elements.empty(); - } - - - inline void Document::elementNames(std::vector & keys) const - { - for (ElementSet::const_iterator it = _elements.begin(); it != _elements.end(); ++it) - { - keys.push_back((*it)->name()); - } - } - - - inline bool Document::exists(const std::string & name) const - { - return std::find_if(_elements.begin(), _elements.end(), ElementFindByName(name)) != _elements.end(); - } - - - inline bool Document::remove(const std::string & name) - { - auto it = std::find_if(_elements.begin(), _elements.end(), ElementFindByName(name)); - if (it == _elements.end()) - return false; - - _elements.erase(it); - return true; - } - - - inline std::size_t Document::size() const - { - return _elements.size(); - } - - - // BSON Embedded Document - // spec: document - template <> - struct ElementTraits - { - enum - { - TypeId = 0x03 - }; - - static std::string toString(const Document::Ptr & value, int indent = 0) - { - return value.isNull() ? "null" : value->toString(indent); - } - }; - - - template <> - inline void BSONReader::read(Document::Ptr & to) - { - to->read(_reader); - } - - - template <> - inline void BSONWriter::write(Document::Ptr & from) - { - from->write(_writer); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Document_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Element.h b/base/poco/MongoDB/include/Poco/MongoDB/Element.h deleted file mode 100644 index 26525d7d02b..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Element.h +++ /dev/null @@ -1,393 +0,0 @@ -// -// Element.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Element -// -// Definition of the Element class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Element_INCLUDED -#define MongoDB_Element_INCLUDED - - -#include -#include -#include -#include -#include "Poco/BinaryReader.h" -#include "Poco/BinaryWriter.h" -#include "Poco/DateTimeFormatter.h" -#include "Poco/MongoDB/BSONReader.h" -#include "Poco/MongoDB/BSONWriter.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/Nullable.h" -#include "Poco/NumberFormatter.h" -#include "Poco/SharedPtr.h" -#include "Poco/Timestamp.h" -#include "Poco/UTF8String.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Element - /// Represents an Element of a Document or an Array. - { - public: - using Ptr = Poco::SharedPtr; - - explicit Element(const std::string & name); - /// Creates the Element with the given name. - - virtual ~Element(); - /// Destructor - - const std::string & name() const; - /// Returns the name of the element. - - virtual std::string toString(int indent = 0) const = 0; - /// Returns a string representation of the element. - - virtual int type() const = 0; - /// Returns the MongoDB type of the element. - - private: - virtual void read(BinaryReader & reader) = 0; - virtual void write(BinaryWriter & writer) = 0; - - friend class Document; - std::string _name; - }; - - - // - // inlines - // - inline const std::string & Element::name() const - { - return _name; - } - - - using ElementSet = std::list; - - - template - struct ElementTraits - { - }; - - - // BSON Floating point - // spec: double - template <> - struct ElementTraits - { - enum - { - TypeId = 0x01 - }; - - static std::string toString(const double & value, int indent = 0) { return Poco::NumberFormatter::format(value); } - }; - - - // BSON UTF-8 string - // spec: int32 (byte*) "\x00" - // int32 is the number bytes in byte* + 1 (for trailing "\x00") - template <> - struct ElementTraits - { - enum - { - TypeId = 0x02 - }; - - static std::string toString(const std::string & value, int indent = 0) - { - std::ostringstream oss; - - oss << '"'; - - for (std::string::const_iterator it = value.begin(); it != value.end(); ++it) - { - switch (*it) - { - case '"': - oss << "\\\""; - break; - case '\\': - oss << "\\\\"; - break; - case '\b': - oss << "\\b"; - break; - case '\f': - oss << "\\f"; - break; - case '\n': - oss << "\\n"; - break; - case '\r': - oss << "\\r"; - break; - case '\t': - oss << "\\t"; - break; - default: { - if (*it > 0 && *it <= 0x1F) - { - oss << "\\u" << std::hex << std::uppercase << std::setfill('0') << std::setw(4) << static_cast(*it); - } - else - { - oss << *it; - } - break; - } - } - } - oss << '"'; - return oss.str(); - } - }; - - - template <> - inline void BSONReader::read(std::string & to) - { - Poco::Int32 size; - _reader >> size; - _reader.readRaw(size, to); - to.erase(to.end() - 1); // remove terminating 0 - } - - - template <> - inline void BSONWriter::write(std::string & from) - { - _writer << (Poco::Int32)(from.length() + 1); - writeCString(from); - } - - - // BSON bool - // spec: "\x00" "\x01" - template <> - struct ElementTraits - { - enum - { - TypeId = 0x08 - }; - - static std::string toString(const bool & value, int indent = 0) { return value ? "true" : "false"; } - }; - - - template <> - inline void BSONReader::read(bool & to) - { - unsigned char b; - _reader >> b; - to = b != 0; - } - - - template <> - inline void BSONWriter::write(bool & from) - { - unsigned char b = from ? 0x01 : 0x00; - _writer << b; - } - - - // BSON 32-bit integer - // spec: int32 - template <> - struct ElementTraits - { - enum - { - TypeId = 0x10 - }; - - - static std::string toString(const Int32 & value, int indent = 0) { return Poco::NumberFormatter::format(value); } - }; - - - // BSON UTC datetime - // spec: int64 - template <> - struct ElementTraits - { - enum - { - TypeId = 0x09 - }; - - static std::string toString(const Timestamp & value, int indent = 0) - { - std::string result; - result.append(1, '"'); - result.append(DateTimeFormatter::format(value, "%Y-%m-%dT%H:%M:%s%z")); - result.append(1, '"'); - return result; - } - }; - - - template <> - inline void BSONReader::read(Timestamp & to) - { - Poco::Int64 value; - _reader >> value; - to = Timestamp::fromEpochTime(static_cast(value / 1000)); - to += (value % 1000 * 1000); - } - - - template <> - inline void BSONWriter::write(Timestamp & from) - { - _writer << (from.epochMicroseconds() / 1000); - } - - - using NullValue = Nullable; - - - // BSON Null Value - // spec: - template <> - struct ElementTraits - { - enum - { - TypeId = 0x0A - }; - - static std::string toString(const NullValue & value, int indent = 0) { return "null"; } - }; - - - template <> - inline void BSONReader::read(NullValue & to) - { - } - - - template <> - inline void BSONWriter::write(NullValue & from) - { - } - - - struct BSONTimestamp - { - Poco::Timestamp ts; - Poco::Int32 inc; - }; - - - // BSON Timestamp - // spec: int64 - template <> - struct ElementTraits - { - enum - { - TypeId = 0x11 - }; - - static std::string toString(const BSONTimestamp & value, int indent = 0) - { - std::string result; - result.append(1, '"'); - result.append(DateTimeFormatter::format(value.ts, "%Y-%m-%dT%H:%M:%s%z")); - result.append(1, ' '); - result.append(NumberFormatter::format(value.inc)); - result.append(1, '"'); - return result; - } - }; - - - template <> - inline void BSONReader::read(BSONTimestamp & to) - { - Poco::Int64 value; - _reader >> value; - to.inc = value & 0xffffffff; - value >>= 32; - to.ts = Timestamp::fromEpochTime(static_cast(value)); - } - - - template <> - inline void BSONWriter::write(BSONTimestamp & from) - { - Poco::Int64 value = from.ts.epochMicroseconds() / 1000; - value <<= 32; - value += from.inc; - _writer << value; - } - - - // BSON 64-bit integer - // spec: int64 - template <> - struct ElementTraits - { - enum - { - TypeId = 0x12 - }; - - static std::string toString(const Int64 & value, int indent = 0) { return NumberFormatter::format(value); } - }; - - - template - class ConcreteElement : public Element - { - public: - ConcreteElement(const std::string & name, const T & init) : Element(name), _value(init) { } - - virtual ~ConcreteElement() { } - - - T value() const { return _value; } - - - std::string toString(int indent = 0) const { return ElementTraits::toString(_value, indent); } - - - int type() const { return ElementTraits::TypeId; } - - void read(BinaryReader & reader) { BSONReader(reader).read(_value); } - - void write(BinaryWriter & writer) { BSONWriter(writer).write(_value); } - - private: - T _value; - }; - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Element_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/GetMoreRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/GetMoreRequest.h deleted file mode 100644 index a95ff4a63db..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/GetMoreRequest.h +++ /dev/null @@ -1,92 +0,0 @@ -// -// GetMoreRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: GetMoreRequest -// -// Definition of the GetMoreRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_GetMoreRequest_INCLUDED -#define MongoDB_GetMoreRequest_INCLUDED - - -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API GetMoreRequest : public RequestMessage - /// A GetMoreRequest is used to query the database for more documents in a collection - /// after a query request is send (OP_GETMORE). - { - public: - GetMoreRequest(const std::string & collectionName, Int64 cursorID); - /// Creates a GetMoreRequest for the give collection and cursor. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". The cursorID has been returned by the response on the query request. - /// By default the numberToReturn is set to 100. - - virtual ~GetMoreRequest(); - /// Destroys the GetMoreRequest. - - Int32 getNumberToReturn() const; - /// Returns the limit of returned documents. - - void setNumberToReturn(Int32 n); - /// Sets the limit of returned documents. - - Int64 cursorID() const; - /// Returns the cursor ID. - - protected: - void buildRequest(BinaryWriter & writer); - - private: - std::string _fullCollectionName; - Int32 _numberToReturn; - Int64 _cursorID; - }; - - - // - // inlines - // - inline Int32 GetMoreRequest::getNumberToReturn() const - { - return _numberToReturn; - } - - - inline void GetMoreRequest::setNumberToReturn(Int32 n) - { - _numberToReturn = n; - } - - - inline Int64 GetMoreRequest::cursorID() const - { - return _cursorID; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_GetMoreRequest_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/InsertRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/InsertRequest.h deleted file mode 100644 index f9bdcc624aa..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/InsertRequest.h +++ /dev/null @@ -1,100 +0,0 @@ -// -// InsertRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: InsertRequest -// -// Definition of the InsertRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_InsertRequest_INCLUDED -#define MongoDB_InsertRequest_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API InsertRequest : public RequestMessage - /// A request for inserting one or more documents to the database - /// (OP_INSERT). - { - public: - enum Flags - { - INSERT_DEFAULT = 0, - /// If specified, perform a normal insert operation. - - INSERT_CONTINUE_ON_ERROR = 1 - /// If set, the database will not stop processing a bulk insert if one - /// fails (e.g. due to duplicate IDs). This makes bulk insert behave similarly - /// to a series of single inserts, except lastError will be set if any insert - /// fails, not just the last one. If multiple errors occur, only the most - /// recent will be reported. - }; - - InsertRequest(const std::string & collectionName, Flags flags = INSERT_DEFAULT); - /// Creates an InsertRequest. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". - - virtual ~InsertRequest(); - /// Destroys the InsertRequest. - - Document & addNewDocument(); - /// Adds a new document for insertion. A reference to the empty document is - /// returned. InsertRequest is the owner of the Document and will free it - /// on destruction. - - Document::Vector & documents(); - /// Returns the documents to insert into the database. - - protected: - void buildRequest(BinaryWriter & writer); - - private: - Int32 _flags; - std::string _fullCollectionName; - Document::Vector _documents; - }; - - - // - // inlines - // - inline Document & InsertRequest::addNewDocument() - { - Document::Ptr doc = new Document(); - _documents.push_back(doc); - return *doc; - } - - - inline Document::Vector & InsertRequest::documents() - { - return _documents; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_InsertRequest_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h b/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h deleted file mode 100644 index c0f584b7c19..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/JavaScriptCode.h +++ /dev/null @@ -1,108 +0,0 @@ -// -// JavaScriptCode.h -// -// Library: MongoDB -// Package: MongoDB -// Module: JavaScriptCode -// -// Definition of the JavaScriptCode class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_JavaScriptCode_INCLUDED -#define MongoDB_JavaScriptCode_INCLUDED - - -#include "Poco/MongoDB/BSONReader.h" -#include "Poco/MongoDB/BSONWriter.h" -#include "Poco/MongoDB/Element.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/SharedPtr.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API JavaScriptCode - /// Represents JavaScript type in BSON. - { - public: - using Ptr = SharedPtr; - - JavaScriptCode(); - /// Creates an empty JavaScriptCode object. - - virtual ~JavaScriptCode(); - /// Destroys the JavaScriptCode. - - void setCode(const std::string & code); - /// Sets the JavaScript code. - - std::string getCode() const; - /// Returns the JavaScript code. - - private: - std::string _code; - }; - - - // - // inlines - // - inline void JavaScriptCode::setCode(const std::string & code) - { - _code = code; - } - - - inline std::string JavaScriptCode::getCode() const - { - return _code; - } - - - // BSON JavaScript code - // spec: string - template <> - struct ElementTraits - { - enum - { - TypeId = 0x0D - }; - - static std::string toString(const JavaScriptCode::Ptr & value, int indent = 0) { return value.isNull() ? "" : value->getCode(); } - }; - - - template <> - inline void BSONReader::read(JavaScriptCode::Ptr & to) - { - std::string code; - BSONReader(_reader).read(code); - to = new JavaScriptCode(); - to->setCode(code); - } - - - template <> - inline void BSONWriter::write(JavaScriptCode::Ptr & from) - { - std::string code = from->getCode(); - BSONWriter(_writer).write(code); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_JavaScriptCode_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/KillCursorsRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/KillCursorsRequest.h deleted file mode 100644 index e6cdd3df34c..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/KillCursorsRequest.h +++ /dev/null @@ -1,65 +0,0 @@ -// -// KillCursorsRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: KillCursorsRequest -// -// Definition of the KillCursorsRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_KillCursorsRequest_INCLUDED -#define MongoDB_KillCursorsRequest_INCLUDED - - -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API KillCursorsRequest : public RequestMessage - /// Class for creating an OP_KILL_CURSORS client request. This - /// request is used to kill cursors, which are still open, - /// returned by query requests. - { - public: - KillCursorsRequest(); - /// Creates a KillCursorsRequest. - - virtual ~KillCursorsRequest(); - /// Destroys the KillCursorsRequest. - - std::vector & cursors(); - /// The internal list of cursors. - - protected: - void buildRequest(BinaryWriter & writer); - std::vector _cursors; - }; - - - // - // inlines - // - inline std::vector & KillCursorsRequest::cursors() - { - return _cursors; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_KillCursorsRequest_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Message.h b/base/poco/MongoDB/include/Poco/MongoDB/Message.h deleted file mode 100644 index f2585db5f49..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/Message.h +++ /dev/null @@ -1,76 +0,0 @@ -// -// Message.h -// -// Library: MongoDB -// Package: MongoDB -// Module: Message -// -// Definition of the Message class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_Message_INCLUDED -#define MongoDB_Message_INCLUDED - - -#include -#include "Poco/BinaryReader.h" -#include "Poco/BinaryWriter.h" -#include "Poco/MongoDB/MessageHeader.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/Net/Socket.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API Message - /// Base class for all messages send or retrieved from MongoDB server. - { - public: - explicit Message(MessageHeader::OpCode opcode); - /// Creates a Message using the given OpCode. - - virtual ~Message(); - /// Destructor - - MessageHeader & header(); - /// Returns the message header - - protected: - MessageHeader _header; - - void messageLength(Poco::Int32 length); - /// Sets the message length in the message header - }; - - - // - // inlines - // - inline MessageHeader & Message::header() - { - return _header; - } - - - inline void Message::messageLength(Poco::Int32 length) - { - poco_assert(length > 0); - _header.setMessageLength(length); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_Message_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h b/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h deleted file mode 100644 index 98f45e876c1..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/MessageHeader.h +++ /dev/null @@ -1,140 +0,0 @@ -// -// MessageHeader.h -// -// Library: MongoDB -// Package: MongoDB -// Module: MessageHeader -// -// Definition of the MessageHeader class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_MessageHeader_INCLUDED -#define MongoDB_MessageHeader_INCLUDED - - -#include "Poco/MongoDB/MessageHeader.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class Message; // Required to disambiguate friend declaration in MessageHeader. - - - class MongoDB_API MessageHeader - /// Represents the message header which is always prepended to a - /// MongoDB request or response message. - { - public: - static const unsigned int MSG_HEADER_SIZE = 16; - - enum OpCode - { - // Opcodes deprecated in MongoDB 5.0 - OP_REPLY = 1, - OP_UPDATE = 2001, - OP_INSERT = 2002, - OP_QUERY = 2004, - OP_GET_MORE = 2005, - OP_DELETE = 2006, - OP_KILL_CURSORS = 2007, - - /// Opcodes supported in MongoDB 5.1 and later - OP_COMPRESSED = 2012, - OP_MSG = 2013 - }; - - explicit MessageHeader(OpCode); - /// Creates the MessageHeader using the given OpCode. - - virtual ~MessageHeader(); - /// Destroys the MessageHeader. - - void read(BinaryReader & reader); - /// Reads the header using the given BinaryReader. - - void write(BinaryWriter & writer); - /// Writes the header using the given BinaryWriter. - - Int32 getMessageLength() const; - /// Returns the message length. - - OpCode opCode() const; - /// Returns the OpCode. - - Int32 getRequestID() const; - /// Returns the request ID of the current message. - - void setRequestID(Int32 id); - /// Sets the request ID of the current message. - - Int32 responseTo() const; - /// Returns the request id from the original request. - - private: - void setMessageLength(Int32 length); - /// Sets the message length. - - Int32 _messageLength; - Int32 _requestID; - Int32 _responseTo; - OpCode _opCode; - - friend class Message; - }; - - - // - // inlines - // - inline MessageHeader::OpCode MessageHeader::opCode() const - { - return _opCode; - } - - - inline Int32 MessageHeader::getMessageLength() const - { - return _messageLength; - } - - - inline void MessageHeader::setMessageLength(Int32 length) - { - poco_assert(_messageLength >= 0); - _messageLength = MSG_HEADER_SIZE + length; - } - - - inline void MessageHeader::setRequestID(Int32 id) - { - _requestID = id; - } - - - inline Int32 MessageHeader::getRequestID() const - { - return _requestID; - } - - inline Int32 MessageHeader::responseTo() const - { - return _responseTo; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_MessageHeader_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h b/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h deleted file mode 100644 index de246ddc9dd..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/MongoDB.h +++ /dev/null @@ -1,64 +0,0 @@ -// -// MongoDB.h -// -// Library: MongoDB -// Package: MongoDB -// Module: MongoDB -// -// Basic definitions for the Poco MongoDB library. -// This file must be the first file included by every other MongoDB -// header file. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDBMongoDB_INCLUDED -#define MongoDBMongoDB_INCLUDED - - -#include "Poco/Foundation.h" - - -// -// The following block is the standard way of creating macros which make exporting -// from a DLL simpler. All files within this DLL are compiled with the MongoDB_EXPORTS -// symbol defined on the command line. this symbol should not be defined on any project -// that uses this DLL. This way any other project whose source files include this file see -// MongoDB_API functions as being imported from a DLL, whereas this DLL sees symbols -// defined with this macro as being exported. -// - - -#if defined(_WIN32) && defined(POCO_DLL) -# if defined(MongoDB_EXPORTS) -# define MongoDB_API __declspec(dllexport) -# else -# define MongoDB_API __declspec(dllimport) -# endif -#endif - - -#if !defined(MongoDB_API) -# if !defined(POCO_NO_GCC_API_ATTRIBUTE) && defined(__GNUC__) && (__GNUC__ >= 4) -# define MongoDB_API __attribute__((visibility("default"))) -# else -# define MongoDB_API -# endif -#endif - - -// -// Automatically link MongoDB library. -// -#if defined(_MSC_VER) -# if !defined(POCO_NO_AUTOMATIC_LIBS) && !defined(MongoDB_EXPORTS) -# pragma comment(lib, "PocoMongoDB" POCO_LIB_SUFFIX) -# endif -#endif - - -#endif // MongoDBMongoDB_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h b/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h deleted file mode 100644 index 8a335320ea0..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/ObjectId.h +++ /dev/null @@ -1,151 +0,0 @@ -// -// Array.h -// -// Library: MongoDB -// Package: MongoDB -// Module: ObjectId -// -// Definition of the ObjectId class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_ObjectId_INCLUDED -#define MongoDB_ObjectId_INCLUDED - - -#include "Poco/MongoDB/Element.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/Timestamp.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API ObjectId - /// ObjectId is a 12-byte BSON type, constructed using: - /// - /// - a 4-byte timestamp, - /// - a 3-byte machine identifier, - /// - a 2-byte process id, and - /// - a 3-byte counter, starting with a random value. - /// - /// In MongoDB, documents stored in a collection require a unique _id field that acts - /// as a primary key. Because ObjectIds are small, most likely unique, and fast to generate, - /// MongoDB uses ObjectIds as the default value for the _id field if the _id field is not - /// specified; i.e., the mongod adds the _id field and generates a unique ObjectId to assign - /// as its value. - { - public: - using Ptr = SharedPtr; - - explicit ObjectId(const std::string & id); - /// Creates an ObjectId from a string. - /// - /// The string must contain a hexadecimal representation - /// of an object ID. This means a string of 24 characters. - - ObjectId(const ObjectId & copy); - /// Creates an ObjectId by copying another one. - - virtual ~ObjectId(); - /// Destroys the ObjectId. - - Timestamp timestamp() const; - /// Returns the timestamp which is stored in the first four bytes of the id - - std::string toString(const std::string & fmt = "%02x") const; - /// Returns the id in string format. The fmt parameter - /// specifies the formatting used for individual members - /// of the ID char array. - - private: - ObjectId(); - - static int fromHex(char c); - static char fromHex(const char * c); - - unsigned char _id[12]; - - friend class BSONWriter; - friend class BSONReader; - friend class Document; - }; - - - // - // inlines - // - inline Timestamp ObjectId::timestamp() const - { - int time; - char * T = (char *)&time; - T[0] = _id[3]; - T[1] = _id[2]; - T[2] = _id[1]; - T[3] = _id[0]; - return Timestamp::fromEpochTime((time_t)time); - } - - - inline int ObjectId::fromHex(char c) - { - if ('0' <= c && c <= '9') - return c - '0'; - if ('a' <= c && c <= 'f') - return c - 'a' + 10; - if ('A' <= c && c <= 'F') - return c - 'A' + 10; - return 0xff; - } - - - inline char ObjectId::fromHex(const char * c) - { - return (char)((fromHex(c[0]) << 4) | fromHex(c[1])); - } - - - // BSON Embedded Document - // spec: ObjectId - template <> - struct ElementTraits - { - enum - { - TypeId = 0x07 - }; - - static std::string toString(const ObjectId::Ptr & id, int indent = 0, const std::string & fmt = "%02x") - { - return id->toString(fmt); - } - }; - - - template <> - inline void BSONReader::read(ObjectId::Ptr & to) - { - _reader.readRaw((char *)to->_id, 12); - } - - - template <> - inline void BSONWriter::write(ObjectId::Ptr & from) - { - _writer.writeRaw((char *)from->_id, 12); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_ObjectId_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h deleted file mode 100644 index a465a71bb1c..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgCursor.h +++ /dev/null @@ -1,96 +0,0 @@ -// -// OpMsgCursor.h -// -// Library: MongoDB -// Package: MongoDB -// Module: OpMsgCursor -// -// Definition of the OpMsgCursor class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_OpMsgCursor_INCLUDED -#define MongoDB_OpMsgCursor_INCLUDED - - -#include "Poco/MongoDB/Connection.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/OpMsgMessage.h" - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API OpMsgCursor : public Document - /// OpMsgCursor is an helper class for querying multiple documents using OpMsgMessage. - { - public: - OpMsgCursor(const std::string & dbname, const std::string & collectionName); - /// Creates a OpMsgCursor for the given database and collection. - - virtual ~OpMsgCursor(); - /// Destroys the OpMsgCursor. - - void setEmptyFirstBatch(bool empty); - /// Empty first batch is used to get error response faster with little server processing - - bool emptyFirstBatch() const; - - void setBatchSize(Int32 batchSize); - /// Set non-default batch size - - Int32 batchSize() const; - /// Current batch size (zero or negative number indicates default batch size) - - Int64 cursorID() const; - - OpMsgMessage & next(Connection & connection); - /// Tries to get the next documents. As long as response message has a - /// cursor ID next can be called to retrieve the next bunch of documents. - /// - /// The cursor must be killed (see kill()) when not all documents are needed. - - OpMsgMessage & query(); - /// Returns the associated query. - - void kill(Connection & connection); - /// Kills the cursor and reset it so that it can be reused. - - private: - OpMsgMessage _query; - OpMsgMessage _response; - - bool _emptyFirstBatch{false}; - Int32 _batchSize{-1}; - /// Batch size used in the cursor. Zero or negative value means that default shall be used. - - Int64 _cursorID{0}; - }; - - - // - // inlines - // - inline OpMsgMessage & OpMsgCursor::query() - { - return _query; - } - - inline Int64 OpMsgCursor::cursorID() const - { - return _cursorID; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_OpMsgCursor_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h b/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h deleted file mode 100644 index 699c7fc4e12..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/OpMsgMessage.h +++ /dev/null @@ -1,163 +0,0 @@ -// -// OpMsgMessage.h -// -// Library: MongoDB -// Package: MongoDB -// Module: OpMsgMessage -// -// Definition of the OpMsgMessage class. -// -// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_OpMsgMessage_INCLUDED -#define MongoDB_OpMsgMessage_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/Message.h" -#include "Poco/MongoDB/MongoDB.h" - -#include - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API OpMsgMessage : public Message - /// This class represents a request/response (OP_MSG) to send requests and receive responses to/from MongoDB. - { - public: - // Constants for most often used MongoDB commands that can be sent using OP_MSG - // For complete list see: https://www.mongodb.com/docs/manual/reference/command/ - - // Query and write - static const std::string CMD_INSERT; - static const std::string CMD_DELETE; - static const std::string CMD_UPDATE; - static const std::string CMD_FIND; - static const std::string CMD_FIND_AND_MODIFY; - static const std::string CMD_GET_MORE; - - // Aggregation - static const std::string CMD_AGGREGATE; - static const std::string CMD_COUNT; - static const std::string CMD_DISTINCT; - static const std::string CMD_MAP_REDUCE; - - // Replication and administration - static const std::string CMD_HELLO; - static const std::string CMD_REPL_SET_GET_STATUS; - static const std::string CMD_REPL_SET_GET_CONFIG; - - static const std::string CMD_CREATE; - static const std::string CMD_CREATE_INDEXES; - static const std::string CMD_DROP; - static const std::string CMD_DROP_DATABASE; - static const std::string CMD_KILL_CURSORS; - static const std::string CMD_LIST_DATABASES; - static const std::string CMD_LIST_INDEXES; - - // Diagnostic - static const std::string CMD_BUILD_INFO; - static const std::string CMD_COLL_STATS; - static const std::string CMD_DB_STATS; - static const std::string CMD_HOST_INFO; - - - enum Flags : UInt32 - { - MSG_FLAGS_DEFAULT = 0, - - MSG_CHECKSUM_PRESENT = (1 << 0), - - MSG_MORE_TO_COME = (1 << 1), - /// Sender will send another message and is not prepared for overlapping messages - - MSG_EXHAUST_ALLOWED = (1 << 16) - /// Client is prepared for multiple replies (using the moreToCome bit) to this request - }; - - OpMsgMessage(); - /// Creates an OpMsgMessage for response. - - OpMsgMessage(const std::string & databaseName, const std::string & collectionName, UInt32 flags = MSG_FLAGS_DEFAULT); - /// Creates an OpMsgMessage for requests. - - virtual ~OpMsgMessage(); - - const std::string & databaseName() const; - - const std::string & collectionName() const; - - void setCommandName(const std::string & command); - /// Sets the command name and clears the command document - - void setCursor(Poco::Int64 cursorID, Poco::Int32 batchSize = -1); - /// Sets the command "getMore" for the cursor id with batch size (if it is not negative). - - const std::string & commandName() const; - /// Current command name. - - void setAcknowledgedRequest(bool ack); - /// Set false to create request that does not return response. - /// It has effect only for commands that write or delete documents. - /// Default is true (request returns acknowledge response). - - bool acknowledgedRequest() const; - - UInt32 flags() const; - - Document & body(); - /// Access to body document. - /// Additional query arguments shall be added after setting the command name. - - const Document & body() const; - - Document::Vector & documents(); - /// Documents prepared for request or retrieved in response. - - const Document::Vector & documents() const; - /// Documents prepared for request or retrieved in response. - - bool responseOk() const; - /// Reads "ok" status from the response message. - - void clear(); - /// Clears the message. - - void send(std::ostream & ostr); - /// Writes the request to stream. - - void read(std::istream & istr); - /// Reads the response from the stream. - - private: - enum PayloadType : UInt8 - { - PAYLOAD_TYPE_0 = 0, - PAYLOAD_TYPE_1 = 1 - }; - - std::string _databaseName; - std::string _collectionName; - UInt32 _flags{MSG_FLAGS_DEFAULT}; - std::string _commandName; - bool _acknowledged{true}; - - Document _body; - Document::Vector _documents; - }; - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_OpMsgMessage_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h b/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h deleted file mode 100644 index 53f4a5127ef..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/PoolableConnectionFactory.h +++ /dev/null @@ -1,123 +0,0 @@ -// -// PoolableConnectionFactory.h -// -// Library: MongoDB -// Package: MongoDB -// Module: PoolableConnectionFactory -// -// Definition of the PoolableConnectionFactory class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_PoolableConnectionFactory_INCLUDED -#define MongoDB_PoolableConnectionFactory_INCLUDED - - -#include "Poco/MongoDB/Connection.h" -#include "Poco/ObjectPool.h" - - -namespace Poco -{ - - -template <> -class PoolableObjectFactory -/// PoolableObjectFactory specialisation for Connection. New connections -/// are created with the given address or URI. -/// -/// If a Connection::SocketFactory is given, it must live for the entire -/// lifetime of the PoolableObjectFactory. -{ -public: - PoolableObjectFactory(Net::SocketAddress & address) : _address(address), _pSocketFactory(0) { } - - PoolableObjectFactory(const std::string & address) : _address(address), _pSocketFactory(0) { } - - PoolableObjectFactory(const std::string & uri, MongoDB::Connection::SocketFactory & socketFactory) - : _uri(uri), _pSocketFactory(&socketFactory) - { - } - - MongoDB::Connection::Ptr createObject() - { - if (_pSocketFactory) - return new MongoDB::Connection(_uri, *_pSocketFactory); - else - return new MongoDB::Connection(_address); - } - - bool validateObject(MongoDB::Connection::Ptr pObject) { return true; } - - void activateObject(MongoDB::Connection::Ptr pObject) { } - - void deactivateObject(MongoDB::Connection::Ptr pObject) { } - - void destroyObject(MongoDB::Connection::Ptr pObject) { } - -private: - Net::SocketAddress _address; - std::string _uri; - MongoDB::Connection::SocketFactory * _pSocketFactory; -}; - - -namespace MongoDB -{ - - - class PooledConnection - /// Helper class for borrowing and returning a connection automatically from a pool. - { - public: - PooledConnection(Poco::ObjectPool & pool) : _pool(pool) { _connection = _pool.borrowObject(); } - - virtual ~PooledConnection() - { - try - { - if (_connection) - { - _pool.returnObject(_connection); - } - } - catch (...) - { - poco_unexpected(); - } - } - - operator Connection::Ptr() { return _connection; } - -#if defined(POCO_ENABLE_CPP11) - // Disable copy to prevent unwanted release of resources: C++11 way - PooledConnection(const PooledConnection &) = delete; - PooledConnection & operator=(const PooledConnection &) = delete; - - // Enable move semantics - PooledConnection(PooledConnection && other) = default; - PooledConnection & operator=(PooledConnection &&) = default; -#endif - - private: -#if !defined(POCO_ENABLE_CPP11) - // Disable copy to prevent unwanted release of resources: pre C++11 way - PooledConnection(const PooledConnection &); - PooledConnection & operator=(const PooledConnection &); -#endif - - Poco::ObjectPool & _pool; - Connection::Ptr _connection; - }; - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_PoolableConnectionFactory_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/QueryRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/QueryRequest.h deleted file mode 100644 index 6a9e80f6d3c..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/QueryRequest.h +++ /dev/null @@ -1,190 +0,0 @@ -// -// QueryRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: QueryRequest -// -// Definition of the QueryRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_QueryRequest_INCLUDED -#define MongoDB_QueryRequest_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API QueryRequest : public RequestMessage - /// A request to query documents in a MongoDB database - /// using an OP_QUERY request. - { - public: - enum Flags - { - QUERY_DEFAULT = 0, - /// Do not set any flags. - - QUERY_TAILABLE_CURSOR = 2, - /// Tailable means cursor is not closed when the last data is retrieved. - /// Rather, the cursor marks the final object’s position. - /// You can resume using the cursor later, from where it was located, - /// if more data were received. Like any "latent cursor", the cursor may - /// become invalid at some point (CursorNotFound) – for example if the final - /// object it references were deleted. - - QUERY_SLAVE_OK = 4, - /// Allow query of replica slave. Normally these return an error except - /// for namespace "local". - - // QUERY_OPLOG_REPLAY = 8 (internal replication use only - drivers should not implement) - - QUERY_NO_CURSOR_TIMEOUT = 16, - /// The server normally times out idle cursors after an inactivity period - /// (10 minutes) to prevent excess memory use. Set this option to prevent that. - - QUERY_AWAIT_DATA = 32, - /// Use with QUERY_TAILABLECURSOR. If we are at the end of the data, block for - /// a while rather than returning no data. After a timeout period, we do - /// return as normal. - - QUERY_EXHAUST = 64, - /// Stream the data down full blast in multiple "more" packages, on the - /// assumption that the client will fully read all data queried. - /// Faster when you are pulling a lot of data and know you want to pull - /// it all down. - /// Note: the client is not allowed to not read all the data unless it - /// closes the connection. - - QUERY_PARTIAL = 128 - /// Get partial results from a mongos if some shards are down - /// (instead of throwing an error). - }; - - QueryRequest(const std::string & collectionName, Flags flags = QUERY_DEFAULT); - /// Creates a QueryRequest. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". - - virtual ~QueryRequest(); - /// Destroys the QueryRequest. - - Flags getFlags() const; - /// Returns the flags. - - void setFlags(Flags flag); - /// Set the flags. - - std::string fullCollectionName() const; - /// Returns the . used for this query. - - Int32 getNumberToSkip() const; - /// Returns the number of documents to skip. - - void setNumberToSkip(Int32 n); - /// Sets the number of documents to skip. - - Int32 getNumberToReturn() const; - /// Returns the number of documents to return. - - void setNumberToReturn(Int32 n); - /// Sets the number of documents to return (limit). - - Document & selector(); - /// Returns the selector document. - - Document & returnFieldSelector(); - /// Returns the field selector document. - - protected: - void buildRequest(BinaryWriter & writer); - - private: - Flags _flags; - std::string _fullCollectionName; - Int32 _numberToSkip; - Int32 _numberToReturn; - Document _selector; - Document _returnFieldSelector; - }; - - - // - // inlines - // - inline QueryRequest::Flags QueryRequest::getFlags() const - { - return _flags; - } - - - inline void QueryRequest::setFlags(QueryRequest::Flags flags) - { - _flags = flags; - } - - - inline std::string QueryRequest::fullCollectionName() const - { - return _fullCollectionName; - } - - - inline Document & QueryRequest::selector() - { - return _selector; - } - - - inline Document & QueryRequest::returnFieldSelector() - { - return _returnFieldSelector; - } - - - inline Int32 QueryRequest::getNumberToSkip() const - { - return _numberToSkip; - } - - - inline void QueryRequest::setNumberToSkip(Int32 n) - { - _numberToSkip = n; - } - - - inline Int32 QueryRequest::getNumberToReturn() const - { - return _numberToReturn; - } - - - inline void QueryRequest::setNumberToReturn(Int32 n) - { - _numberToReturn = n; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_QueryRequest_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h b/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h deleted file mode 100644 index 244b8c14163..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/RegularExpression.h +++ /dev/null @@ -1,135 +0,0 @@ -// -// RegularExpression.h -// -// Library: MongoDB -// Package: MongoDB -// Module: RegularExpression -// -// Definition of the RegularExpression class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_RegularExpression_INCLUDED -#define MongoDB_RegularExpression_INCLUDED - - -#include "Poco/MongoDB/Element.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/RegularExpression.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API RegularExpression - /// Represents a regular expression in BSON format. - { - public: - using Ptr = SharedPtr; - - RegularExpression(); - /// Creates an empty RegularExpression. - - RegularExpression(const std::string & pattern, const std::string & options); - /// Creates a RegularExpression using the given pattern and options. - - virtual ~RegularExpression(); - /// Destroys the RegularExpression. - - SharedPtr createRE() const; - /// Tries to create a Poco::RegularExpression from the MongoDB regular expression. - - std::string getOptions() const; - /// Returns the options string. - - void setOptions(const std::string & options); - /// Sets the options string. - - std::string getPattern() const; - /// Returns the pattern. - - void setPattern(const std::string & pattern); - /// Sets the pattern. - - private: - std::string _pattern; - std::string _options; - }; - - - /// - /// inlines - /// - inline std::string RegularExpression::getPattern() const - { - return _pattern; - } - - - inline void RegularExpression::setPattern(const std::string & pattern) - { - _pattern = pattern; - } - - - inline std::string RegularExpression::getOptions() const - { - return _options; - } - - - inline void RegularExpression::setOptions(const std::string & options) - { - _options = options; - } - - - // BSON Regex - // spec: cstring cstring - template <> - struct ElementTraits - { - enum - { - TypeId = 0x0B - }; - - static std::string toString(const RegularExpression::Ptr & value, int indent = 0) - { - //TODO - return "RE: not implemented yet"; - } - }; - - - template <> - inline void BSONReader::read(RegularExpression::Ptr & to) - { - std::string pattern = readCString(); - std::string options = readCString(); - - to = new RegularExpression(pattern, options); - } - - - template <> - inline void BSONWriter::write(RegularExpression::Ptr & from) - { - writeCString(from->getPattern()); - writeCString(from->getOptions()); - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_RegularExpression_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/ReplicaSet.h b/base/poco/MongoDB/include/Poco/MongoDB/ReplicaSet.h deleted file mode 100644 index 61c96fd8d00..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/ReplicaSet.h +++ /dev/null @@ -1,61 +0,0 @@ -// -// ReplicaSet.h -// -// Library: MongoDB -// Package: MongoDB -// Module: ReplicaSet -// -// Definition of the ReplicaSet class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_ReplicaSet_INCLUDED -#define MongoDB_ReplicaSet_INCLUDED - - -#include -#include "Poco/MongoDB/Connection.h" -#include "Poco/Net/SocketAddress.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API ReplicaSet - /// Class for working with a MongoDB replica set. - { - public: - explicit ReplicaSet(const std::vector & addresses); - /// Creates the ReplicaSet using the given server addresses. - - virtual ~ReplicaSet(); - /// Destroys the ReplicaSet. - - Connection::Ptr findMaster(); - /// Tries to find the master MongoDB instance from the addresses - /// passed to the constructor. - /// - /// Returns the Connection to the master, or null if no master - /// instance was found. - - protected: - Connection::Ptr isMaster(const Net::SocketAddress & host); - - private: - std::vector _addresses; - }; - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_ReplicaSet_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/RequestMessage.h b/base/poco/MongoDB/include/Poco/MongoDB/RequestMessage.h deleted file mode 100644 index 5ac750a7fe0..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/RequestMessage.h +++ /dev/null @@ -1,54 +0,0 @@ -// -// RequestMessage.h -// -// Library: MongoDB -// Package: MongoDB -// Module: RequestMessage -// -// Definition of the RequestMessage class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_RequestMessage_INCLUDED -#define MongoDB_RequestMessage_INCLUDED - - -#include -#include "Poco/MongoDB/Message.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API RequestMessage : public Message - /// Base class for a request sent to the MongoDB server. - { - public: - explicit RequestMessage(MessageHeader::OpCode opcode); - /// Creates a RequestMessage using the given opcode. - - virtual ~RequestMessage(); - /// Destroys the RequestMessage. - - void send(std::ostream & ostr); - /// Writes the request to stream. - - protected: - virtual void buildRequest(BinaryWriter & ss) = 0; - }; - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_RequestMessage_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h b/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h deleted file mode 100644 index 9cb92cb16c4..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/ResponseMessage.h +++ /dev/null @@ -1,114 +0,0 @@ -// -// ResponseMessage.h -// -// Library: MongoDB -// Package: MongoDB -// Module: ResponseMessage -// -// Definition of the ResponseMessage class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_ResponseMessage_INCLUDED -#define MongoDB_ResponseMessage_INCLUDED - - -#include -#include -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/Message.h" -#include "Poco/MongoDB/MongoDB.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class MongoDB_API ResponseMessage : public Message - /// This class represents a response (OP_REPLY) from MongoDB. - { - public: - ResponseMessage(); - /// Creates an empty ResponseMessage. - - ResponseMessage(const Int64 & cursorID); - /// Creates an ResponseMessage for existing cursor ID. - - virtual ~ResponseMessage(); - /// Destroys the ResponseMessage. - - Int64 cursorID() const; - /// Returns the cursor ID. - - void clear(); - /// Clears the response. - - std::size_t count() const; - /// Returns the number of documents in the response. - - Document::Vector & documents(); - /// Returns a vector containing the received documents. - - bool empty() const; - /// Returns true if the response does not contain any documents. - - bool hasDocuments() const; - /// Returns true if there is at least one document in the response. - - void read(std::istream & istr); - /// Reads the response from the stream. - - private: - Int32 _responseFlags; - Int64 _cursorID; - Int32 _startingFrom; - Int32 _numberReturned; - Document::Vector _documents; - }; - - - // - // inlines - // - inline std::size_t ResponseMessage::count() const - { - return _documents.size(); - } - - - inline bool ResponseMessage::empty() const - { - return _documents.size() == 0; - } - - - inline Int64 ResponseMessage::cursorID() const - { - return _cursorID; - } - - - inline Document::Vector & ResponseMessage::documents() - { - return _documents; - } - - - inline bool ResponseMessage::hasDocuments() const - { - return _documents.size() > 0; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_ResponseMessage_INCLUDED diff --git a/base/poco/MongoDB/include/Poco/MongoDB/UpdateRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/UpdateRequest.h deleted file mode 100644 index 0f61c4a3e29..00000000000 --- a/base/poco/MongoDB/include/Poco/MongoDB/UpdateRequest.h +++ /dev/null @@ -1,117 +0,0 @@ -// -// UpdateRequest.h -// -// Library: MongoDB -// Package: MongoDB -// Module: UpdateRequest -// -// Definition of the UpdateRequest class. -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef MongoDB_UpdateRequest_INCLUDED -#define MongoDB_UpdateRequest_INCLUDED - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/MongoDB.h" -#include "Poco/MongoDB/RequestMessage.h" - - -namespace Poco -{ -namespace MongoDB -{ - - - class UpdateRequest : public RequestMessage - /// This request is used to update a document in a database - /// using the OP_UPDATE client request. - { - public: - enum Flags - { - UPDATE_DEFAULT = 0, - /// If set, the database will insert the supplied object into the - /// collection if no matching document is found. - - UPDATE_UPSERT = 1, - /// If set, the database will update all matching objects in the collection. - /// Otherwise only updates first matching doc. - - UPDATE_MULTIUPDATE = 2 - /// If set to, updates multiple documents that meet the query criteria. - /// Otherwise only updates one document. - }; - - UpdateRequest(const std::string & collectionName, Flags flags = UPDATE_DEFAULT); - /// Creates the UpdateRequest. - /// - /// The full collection name is the concatenation of the database - /// name with the collection name, using a "." for the concatenation. For example, - /// for the database "foo" and the collection "bar", the full collection name is - /// "foo.bar". - - virtual ~UpdateRequest(); - /// Destroys the UpdateRequest. - - Document & selector(); - /// Returns the selector document. - - Document & update(); - /// Returns the document to update. - - Flags flags() const; - /// Returns the flags - - void flags(Flags flags); - /// Sets the flags - - protected: - void buildRequest(BinaryWriter & writer); - - private: - Flags _flags; - std::string _fullCollectionName; - Document _selector; - Document _update; - }; - - - // - // inlines - // - inline UpdateRequest::Flags UpdateRequest::flags() const - { - return _flags; - } - - - inline void UpdateRequest::flags(UpdateRequest::Flags flags) - { - _flags = flags; - } - - - inline Document & UpdateRequest::selector() - { - return _selector; - } - - - inline Document & UpdateRequest::update() - { - return _update; - } - - -} -} // namespace Poco::MongoDB - - -#endif // MongoDB_UpdateRequest_INCLUDED diff --git a/base/poco/MongoDB/src/Array.cpp b/base/poco/MongoDB/src/Array.cpp deleted file mode 100644 index 6fff0994d82..00000000000 --- a/base/poco/MongoDB/src/Array.cpp +++ /dev/null @@ -1,75 +0,0 @@ -// -// Array.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Array -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Array.h" -#include - - -namespace Poco { -namespace MongoDB { - - -Array::Array(): - Document() -{ -} - - -Array::~Array() -{ -} - - -Element::Ptr Array::get(std::size_t pos) const -{ - std::string name = Poco::NumberFormatter::format(pos); - return Document::get(name); -} - - -std::string Array::toString(int indent) const -{ - std::ostringstream oss; - - oss << "["; - - if (indent > 0) oss << std::endl; - - for (ElementSet::const_iterator it = _elements.begin(); it != _elements.end(); ++it) - { - if (it != _elements.begin()) - { - oss << ","; - if (indent > 0) oss << std::endl; - } - - for (int i = 0; i < indent; ++i) oss << ' '; - - oss << (*it)->toString(indent > 0 ? indent + 2 : 0); - } - - if (indent > 0) - { - oss << std::endl; - if (indent >= 2) indent -= 2; - for (int i = 0; i < indent; ++i) oss << ' '; - } - - oss << "]"; - - return oss.str(); -} - - -} } // Namespace Poco::Mongo diff --git a/base/poco/MongoDB/src/Binary.cpp b/base/poco/MongoDB/src/Binary.cpp deleted file mode 100644 index 8b0e6baeccb..00000000000 --- a/base/poco/MongoDB/src/Binary.cpp +++ /dev/null @@ -1,89 +0,0 @@ -// -// Binary.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Binary -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Binary.h" - - -namespace Poco { -namespace MongoDB { - - -Binary::Binary(): - _buffer(0), - _subtype(0) -{ -} - - -Binary::Binary(Poco::Int32 size, unsigned char subtype): - _buffer(size), - _subtype(subtype) -{ -} - - -Binary::Binary(const UUID& uuid): - _buffer(128 / 8), - _subtype(0x04) -{ - unsigned char szUUID[16]; - uuid.copyTo((char*) szUUID); - _buffer.assign(szUUID, 16); -} - - - -Binary::Binary(const std::string& data, unsigned char subtype): - _buffer(reinterpret_cast(data.data()), data.size()), - _subtype(subtype) -{ -} - - -Binary::Binary(const void* data, Poco::Int32 size, unsigned char subtype): - _buffer(reinterpret_cast(data), size), - _subtype(subtype) -{ -} - - -Binary::~Binary() -{ -} - - -std::string Binary::toString(int indent) const -{ - std::ostringstream oss; - Base64Encoder encoder(oss); - MemoryInputStream mis((const char*) _buffer.begin(), _buffer.size()); - StreamCopier::copyStream(mis, encoder); - encoder.close(); - return oss.str(); -} - - -UUID Binary::uuid() const -{ - if ((_subtype == 0x04 || _subtype == 0x03) && _buffer.size() == 16) - { - UUID uuid; - uuid.copyFrom((const char*) _buffer.begin()); - return uuid; - } - throw BadCastException("Invalid subtype: " + std::to_string(_subtype) + ", size: " + std::to_string(_buffer.size())); -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/Connection.cpp b/base/poco/MongoDB/src/Connection.cpp deleted file mode 100644 index fa20887054b..00000000000 --- a/base/poco/MongoDB/src/Connection.cpp +++ /dev/null @@ -1,348 +0,0 @@ -// -// Connection.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Connection -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/Net/SocketStream.h" -#include "Poco/MongoDB/Connection.h" -#include "Poco/MongoDB/Database.h" -#include "Poco/URI.h" -#include "Poco/Format.h" -#include "Poco/NumberParser.h" -#include "Poco/Exception.h" - - -namespace Poco { -namespace MongoDB { - - -Connection::SocketFactory::SocketFactory() -{ -} - - -Connection::SocketFactory::~SocketFactory() -{ -} - - -Poco::Net::StreamSocket Connection::SocketFactory::createSocket(const std::string& host, int port, Poco::Timespan connectTimeout, bool secure) -{ - if (!secure) - { - Poco::Net::SocketAddress addr(host, port); - Poco::Net::StreamSocket socket; - if (connectTimeout > 0) - socket.connect(addr, connectTimeout); - else - socket.connect(addr); - return socket; - } - else throw Poco::NotImplementedException("Default SocketFactory implementation does not support SecureStreamSocket"); -} - - -Connection::Connection(): - _address(), - _socket() -{ -} - - -Connection::Connection(const std::string& hostAndPort): - _address(hostAndPort), - _socket() -{ - connect(); -} - - -Connection::Connection(const std::string& uri, SocketFactory& socketFactory): - _address(), - _socket() -{ - connect(uri, socketFactory); -} - - -Connection::Connection(const std::string& host, int port): - _address(host, port), - _socket() -{ - connect(); -} - - -Connection::Connection(const Poco::Net::SocketAddress& addrs): - _address(addrs), - _socket() -{ - connect(); -} - - -Connection::Connection(const Poco::Net::StreamSocket& socket): - _address(socket.peerAddress()), - _socket(socket) -{ -} - - -Connection::~Connection() -{ - try - { - disconnect(); - } - catch (...) - { - } -} - - -void Connection::connect() -{ - _socket.connect(_address); -} - - -void Connection::connect(const std::string& hostAndPort) -{ - _address = Poco::Net::SocketAddress(hostAndPort); - connect(); -} - - -void Connection::connect(const std::string& host, int port) -{ - _address = Poco::Net::SocketAddress(host, port); - connect(); -} - - -void Connection::connect(const Poco::Net::SocketAddress& addrs) -{ - _address = addrs; - connect(); -} - - -void Connection::connect(const Poco::Net::StreamSocket& socket) -{ - _address = socket.peerAddress(); - _socket = socket; -} - - -void Connection::connect(const std::string& uri, SocketFactory& socketFactory) -{ - std::vector strAddresses; - std::string newURI; - - if (uri.find(',') != std::string::npos) - { - size_t pos; - size_t head = 0; - if ((pos = uri.find("@")) != std::string::npos) - { - head = pos + 1; - } - else if ((pos = uri.find("://")) != std::string::npos) - { - head = pos + 3; - } - - std::string tempstr; - std::string::const_iterator it = uri.begin(); - it += head; - size_t tail = head; - for (;it != uri.end() && *it != '?' && *it != '/'; ++it) - { - tempstr += *it; - tail++; - } - - it = tempstr.begin(); - std::string token; - for (;it != tempstr.end(); ++it) - { - if (*it == ',') - { - newURI = uri.substr(0, head) + token + uri.substr(tail, uri.length()); - strAddresses.push_back(newURI); - token = ""; - } - else - { - token += *it; - } - } - newURI = uri.substr(0, head) + token + uri.substr(tail, uri.length()); - strAddresses.push_back(newURI); - } - else - { - strAddresses.push_back(uri); - } - - newURI = strAddresses.front(); - Poco::URI theURI(newURI); - if (theURI.getScheme() != "mongodb") throw Poco::UnknownURISchemeException(uri); - - std::string userInfo = theURI.getUserInfo(); - std::string databaseName = theURI.getPath(); - if (!databaseName.empty() && databaseName[0] == '/') databaseName.erase(0, 1); - if (databaseName.empty()) databaseName = "admin"; - - bool ssl = false; - Poco::Timespan connectTimeout; - Poco::Timespan socketTimeout; - std::string authMechanism = Database::AUTH_SCRAM_SHA1; - std::string readPreference="primary"; - - Poco::URI::QueryParameters params = theURI.getQueryParameters(); - for (Poco::URI::QueryParameters::const_iterator it = params.begin(); it != params.end(); ++it) - { - if (it->first == "ssl") - { - ssl = (it->second == "true"); - } - else if (it->first == "connectTimeoutMS") - { - connectTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); - } - else if (it->first == "socketTimeoutMS") - { - socketTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); - } - else if (it->first == "authMechanism") - { - authMechanism = it->second; - } - else if (it->first == "readPreference") - { - readPreference= it->second; - } - } - - for (std::vector::const_iterator it = strAddresses.cbegin();it != strAddresses.cend(); ++it) - { - newURI = *it; - theURI = Poco::URI(newURI); - - std::string host = theURI.getHost(); - Poco::UInt16 port = theURI.getPort(); - if (port == 0) port = 27017; - - connect(socketFactory.createSocket(host, port, connectTimeout, ssl)); - _uri = newURI; - if (socketTimeout > 0) - { - _socket.setSendTimeout(socketTimeout); - _socket.setReceiveTimeout(socketTimeout); - } - if (strAddresses.size() > 1) - { - Poco::MongoDB::QueryRequest request("admin.$cmd"); - request.setNumberToReturn(1); - request.selector().add("isMaster", 1); - Poco::MongoDB::ResponseMessage response; - - sendRequest(request, response); - _uri = newURI; - if (!response.documents().empty()) - { - Poco::MongoDB::Document::Ptr doc = response.documents()[0]; - if (doc->get("ismaster") && readPreference == "primary") - { - break; - } - else if (!doc->get("ismaster") && readPreference == "secondary") - { - break; - } - else if (it + 1 == strAddresses.cend()) - { - throw Poco::URISyntaxException(uri); - } - } - } - } - if (!userInfo.empty()) - { - std::string username; - std::string password; - std::string::size_type pos = userInfo.find(':'); - if (pos != std::string::npos) - { - username.assign(userInfo, 0, pos++); - password.assign(userInfo, pos, userInfo.size() - pos); - } - else username = userInfo; - - Database database(databaseName); - - if (!database.authenticate(*this, username, password, authMechanism)) - throw Poco::NoPermissionException(Poco::format("Access to MongoDB database %s denied for user %s", databaseName, username)); - } -} - - -void Connection::disconnect() -{ - _socket.close(); -} - - -void Connection::sendRequest(RequestMessage& request) -{ - Poco::Net::SocketOutputStream sos(_socket); - request.send(sos); -} - - -void Connection::sendRequest(RequestMessage& request, ResponseMessage& response) -{ - sendRequest(request); - - Poco::Net::SocketInputStream sis(_socket); - response.read(sis); -} - - -void Connection::sendRequest(OpMsgMessage& request, OpMsgMessage& response) -{ - Poco::Net::SocketOutputStream sos(_socket); - request.send(sos); - - response.clear(); - readResponse(response); -} - - -void Connection::sendRequest(OpMsgMessage& request) -{ - request.setAcknowledgedRequest(false); - Poco::Net::SocketOutputStream sos(_socket); - request.send(sos); -} - - -void Connection::readResponse(OpMsgMessage& response) -{ - Poco::Net::SocketInputStream sis(_socket); - response.read(sis); -} - - - -} } // Poco::MongoDB diff --git a/base/poco/MongoDB/src/Cursor.cpp b/base/poco/MongoDB/src/Cursor.cpp deleted file mode 100644 index ef7a4ca961d..00000000000 --- a/base/poco/MongoDB/src/Cursor.cpp +++ /dev/null @@ -1,83 +0,0 @@ -// -// Cursor.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Cursor -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Cursor.h" -#include "Poco/MongoDB/GetMoreRequest.h" -#include "Poco/MongoDB/KillCursorsRequest.h" - - -namespace Poco { -namespace MongoDB { - - -Cursor::Cursor(const std::string& db, const std::string& collection, QueryRequest::Flags flags): - _query(db + '.' + collection, flags) -{ -} - - -Cursor::Cursor(const std::string& fullCollectionName, QueryRequest::Flags flags): - _query(fullCollectionName, flags) -{ -} - - -Cursor::Cursor(const Document& aggregationResponse) : - _query(aggregationResponse.get("cursor")->get("ns")), - _response(aggregationResponse.get("cursor")->get("id")) -{ -} - -Cursor::~Cursor() -{ - try - { - poco_assert_dbg(!_response.cursorID()); - } - catch (...) - { - } -} - - -ResponseMessage& Cursor::next(Connection& connection) -{ - if (_response.cursorID() == 0) - { - connection.sendRequest(_query, _response); - } - else - { - Poco::MongoDB::GetMoreRequest getMore(_query.fullCollectionName(), _response.cursorID()); - getMore.setNumberToReturn(_query.getNumberToReturn()); - _response.clear(); - connection.sendRequest(getMore, _response); - } - return _response; -} - - -void Cursor::kill(Connection& connection) -{ - if (_response.cursorID() != 0) - { - KillCursorsRequest killRequest; - killRequest.cursors().push_back(_response.cursorID()); - connection.sendRequest(killRequest); - } - _response.clear(); -} - - -} } // Namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/Database.cpp b/base/poco/MongoDB/src/Database.cpp deleted file mode 100644 index 15c46b17251..00000000000 --- a/base/poco/MongoDB/src/Database.cpp +++ /dev/null @@ -1,482 +0,0 @@ -// -// Database.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Database -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Database.h" -#include "Poco/MongoDB/Binary.h" -#include "Poco/MD5Engine.h" -#include "Poco/SHA1Engine.h" -#include "Poco/PBKDF2Engine.h" -#include "Poco/HMACEngine.h" -#include "Poco/Base64Decoder.h" -#include "Poco/MemoryStream.h" -#include "Poco/StreamCopier.h" -#include "Poco/Exception.h" -#include "Poco/RandomStream.h" -#include "Poco/Random.h" -#include "Poco/Format.h" -#include "Poco/NumberParser.h" -#include -#include - - -namespace Poco { -namespace MongoDB { - - -const std::string Database::AUTH_MONGODB_CR("MONGODB-CR"); -const std::string Database::AUTH_SCRAM_SHA1("SCRAM-SHA-1"); - - -namespace -{ - std::map parseKeyValueList(const std::string& str) - { - std::map kvm; - std::string::const_iterator it = str.begin(); - std::string::const_iterator end = str.end(); - while (it != end) - { - std::string k; - std::string v; - while (it != end && *it != '=') k += *it++; - if (it != end) ++it; - while (it != end && *it != ',') v += *it++; - if (it != end) ++it; - kvm[k] = v; - } - return kvm; - } - - std::string decodeBase64(const std::string& base64) - { - Poco::MemoryInputStream istr(base64.data(), base64.size()); - Poco::Base64Decoder decoder(istr); - std::string result; - Poco::StreamCopier::copyToString(decoder, result); - return result; - } - - std::string encodeBase64(const std::string& data) - { - std::ostringstream ostr; - Poco::Base64Encoder encoder(ostr); - encoder.rdbuf()->setLineLength(0); - encoder << data; - encoder.close(); - return ostr.str(); - } - - std::string digestToBinaryString(Poco::DigestEngine& engine) - { - Poco::DigestEngine::Digest d = engine.digest(); - return std::string(reinterpret_cast(&d[0]), d.size()); - } - - std::string digestToHexString(Poco::DigestEngine& engine) - { - Poco::DigestEngine::Digest d = engine.digest(); - return Poco::DigestEngine::digestToHex(d); - } - - std::string digestToBase64(Poco::DigestEngine& engine) - { - return encodeBase64(digestToBinaryString(engine)); - } - - std::string hashCredentials(const std::string& username, const std::string& password) - { - Poco::MD5Engine md5; - md5.update(username); - md5.update(std::string(":mongo:")); - md5.update(password); - return digestToHexString(md5); - } - - std::string createNonce() - { - Poco::MD5Engine md5; - Poco::RandomInputStream randomStream; - Poco::Random random; - for (int i = 0; i < 4; i++) - { - md5.update(randomStream.get()); - md5.update(random.nextChar()); - } - return digestToHexString(md5); - } -} - - -Database::Database(const std::string& db): - _dbname(db) -{ -} - - -Database::~Database() -{ -} - - -bool Database::authenticate(Connection& connection, const std::string& username, const std::string& password, const std::string& method) -{ - if (username.empty()) throw Poco::InvalidArgumentException("empty username"); - if (password.empty()) throw Poco::InvalidArgumentException("empty password"); - - if (method == AUTH_MONGODB_CR) - return authCR(connection, username, password); - else if (method == AUTH_SCRAM_SHA1) - return authSCRAM(connection, username, password); - else - throw Poco::InvalidArgumentException("authentication method", method); -} - - -bool Database::authCR(Connection& connection, const std::string& username, const std::string& password) -{ - std::string nonce; - Poco::SharedPtr pCommand = createCommand(); - pCommand->selector().add("getnonce", 1); - - ResponseMessage response; - connection.sendRequest(*pCommand, response); - if (response.documents().size() > 0) - { - Document::Ptr pDoc = response.documents()[0]; - if (pDoc->getInteger("ok") != 1) return false; - nonce = pDoc->get("nonce", ""); - if (nonce.empty()) throw Poco::ProtocolException("no nonce received"); - } - else throw Poco::ProtocolException("empty response for getnonce"); - - std::string credsDigest = hashCredentials(username, password); - - Poco::MD5Engine md5; - md5.update(nonce); - md5.update(username); - md5.update(credsDigest); - std::string key = digestToHexString(md5); - - pCommand = createCommand(); - pCommand->selector() - .add("authenticate", 1) - .add("user", username) - .add("nonce", nonce) - .add("key", key); - - connection.sendRequest(*pCommand, response); - if (response.documents().size() > 0) - { - Document::Ptr pDoc = response.documents()[0]; - return pDoc->getInteger("ok") == 1; - } - else throw Poco::ProtocolException("empty response for authenticate"); -} - - -bool Database::authSCRAM(Connection& connection, const std::string& username, const std::string& password) -{ - std::string clientNonce(createNonce()); - std::string clientFirstMsg = Poco::format("n=%s,r=%s", username, clientNonce); - - Poco::SharedPtr pCommand = createCommand(); - pCommand->selector() - .add("saslStart", 1) - .add("mechanism", AUTH_SCRAM_SHA1) - .add("payload", new Binary(Poco::format("n,,%s", clientFirstMsg))); - - ResponseMessage response; - connection.sendRequest(*pCommand, response); - - Int32 conversationId = 0; - std::string serverFirstMsg; - - if (response.documents().size() > 0) - { - Document::Ptr pDoc = response.documents()[0]; - if (pDoc->getInteger("ok") == 1) - { - Binary::Ptr pPayload = pDoc->get("payload"); - serverFirstMsg = pPayload->toRawString(); - conversationId = pDoc->get("conversationId"); - } - else - { - if (pDoc->exists("errmsg")) - { - const Poco::MongoDB::Element::Ptr value = pDoc->get("errmsg"); - auto message = static_cast &>(*value).value(); - throw Poco::RuntimeException(message); - } - else - return false; - } - } - else throw Poco::ProtocolException("empty response for saslStart"); - - std::map kvm = parseKeyValueList(serverFirstMsg); - const std::string serverNonce = kvm["r"]; - const std::string salt = decodeBase64(kvm["s"]); - const unsigned iterations = Poco::NumberParser::parseUnsigned(kvm["i"]); - const Poco::UInt32 dkLen = 20; - - std::string hashedPassword = hashCredentials(username, password); - - Poco::PBKDF2Engine > pbkdf2(salt, iterations, dkLen); - pbkdf2.update(hashedPassword); - std::string saltedPassword = digestToBinaryString(pbkdf2); - - std::string clientFinalNoProof = Poco::format("c=biws,r=%s", serverNonce); - std::string authMessage = Poco::format("%s,%s,%s", clientFirstMsg, serverFirstMsg, clientFinalNoProof); - - Poco::HMACEngine hmacKey(saltedPassword); - hmacKey.update(std::string("Client Key")); - std::string clientKey = digestToBinaryString(hmacKey); - - Poco::SHA1Engine sha1; - sha1.update(clientKey); - std::string storedKey = digestToBinaryString(sha1); - - Poco::HMACEngine hmacSig(storedKey); - hmacSig.update(authMessage); - std::string clientSignature = digestToBinaryString(hmacSig); - - std::string clientProof(clientKey); - for (std::size_t i = 0; i < clientProof.size(); i++) - { - clientProof[i] ^= clientSignature[i]; - } - - std::string clientFinal = Poco::format("%s,p=%s", clientFinalNoProof, encodeBase64(clientProof)); - - pCommand = createCommand(); - pCommand->selector() - .add("saslContinue", 1) - .add("conversationId", conversationId) - .add("payload", new Binary(clientFinal)); - - std::string serverSecondMsg; - connection.sendRequest(*pCommand, response); - if (response.documents().size() > 0) - { - Document::Ptr pDoc = response.documents()[0]; - if (pDoc->getInteger("ok") == 1) - { - Binary::Ptr pPayload = pDoc->get("payload"); - serverSecondMsg = pPayload->toRawString(); - } - else - { - if (pDoc->exists("errmsg")) - { - const Poco::MongoDB::Element::Ptr value = pDoc->get("errmsg"); - auto message = static_cast &>(*value).value(); - throw Poco::RuntimeException(message); - } - else - return false; - } - } - else throw Poco::ProtocolException("empty response for saslContinue"); - - Poco::HMACEngine hmacSKey(saltedPassword); - hmacSKey.update(std::string("Server Key")); - std::string serverKey = digestToBinaryString(hmacSKey); - - Poco::HMACEngine hmacSSig(serverKey); - hmacSSig.update(authMessage); - std::string serverSignature = digestToBase64(hmacSSig); - - kvm = parseKeyValueList(serverSecondMsg); - std::string serverSignatureReceived = kvm["v"]; - - if (serverSignature != serverSignatureReceived) - throw Poco::ProtocolException("server signature verification failed"); - - pCommand = createCommand(); - pCommand->selector() - .add("saslContinue", 1) - .add("conversationId", conversationId) - .add("payload", new Binary); - - connection.sendRequest(*pCommand, response); - if (response.documents().size() > 0) - { - Document::Ptr pDoc = response.documents()[0]; - if (pDoc->getInteger("ok") == 1) - { - return true; - } - else - { - if (pDoc->exists("errmsg")) - { - const Poco::MongoDB::Element::Ptr value = pDoc->get("errmsg"); - auto message = static_cast &>(*value).value(); - throw Poco::RuntimeException(message); - } - else - return false; - } - } - else throw Poco::ProtocolException("empty response for saslContinue"); -} - - -Document::Ptr Database::queryBuildInfo(Connection& connection) const -{ - // build info can be issued on "config" system database - Poco::SharedPtr request = createCommand(); - request->selector().add("buildInfo", 1); - - Poco::MongoDB::ResponseMessage response; - connection.sendRequest(*request, response); - - Document::Ptr buildInfo; - if ( response.documents().size() > 0 ) - { - buildInfo = response.documents()[0]; - } - else - { - throw Poco::ProtocolException("Didn't get a response from the buildinfo command"); - } - return buildInfo; -} - - -Document::Ptr Database::queryServerHello(Connection& connection, bool old) const -{ - // hello can be issued on "config" system database - Poco::SharedPtr request = createCommand(); - - // 'hello' command was previously called 'isMaster' - std::string command_name; - if (old) - command_name = "isMaster"; - else - command_name = "hello"; - - request->selector().add(command_name, 1); - - Poco::MongoDB::ResponseMessage response; - connection.sendRequest(*request, response); - - Document::Ptr hello; - if ( response.documents().size() > 0 ) - { - hello = response.documents()[0]; - } - else - { - throw Poco::ProtocolException("Didn't get a response from the hello command"); - } - return hello; -} - - -Int64 Database::count(Connection& connection, const std::string& collectionName) const -{ - Poco::SharedPtr countRequest = createCountRequest(collectionName); - - Poco::MongoDB::ResponseMessage response; - connection.sendRequest(*countRequest, response); - - if (response.documents().size() > 0) - { - Poco::MongoDB::Document::Ptr doc = response.documents()[0]; - return doc->getInteger("n"); - } - - return -1; -} - - -Poco::MongoDB::Document::Ptr Database::ensureIndex(Connection& connection, const std::string& collection, const std::string& indexName, Poco::MongoDB::Document::Ptr keys, bool unique, bool background, int version, int ttl) -{ - Poco::MongoDB::Document::Ptr index = new Poco::MongoDB::Document(); - index->add("ns", _dbname + "." + collection); - index->add("name", indexName); - index->add("key", keys); - - if (version > 0) - { - index->add("version", version); - } - - if (unique) - { - index->add("unique", true); - } - - if (background) - { - index->add("background", true); - } - - if (ttl > 0) - { - index->add("expireAfterSeconds", ttl); - } - - Poco::SharedPtr insertRequest = createInsertRequest("system.indexes"); - insertRequest->documents().push_back(index); - connection.sendRequest(*insertRequest); - - return getLastErrorDoc(connection); -} - - -Document::Ptr Database::getLastErrorDoc(Connection& connection) const -{ - Document::Ptr errorDoc; - - Poco::SharedPtr request = createCommand(); - request->setNumberToReturn(1); - request->selector().add("getLastError", 1); - - Poco::MongoDB::ResponseMessage response; - connection.sendRequest(*request, response); - - if (response.documents().size() > 0) - { - errorDoc = response.documents()[0]; - } - - return errorDoc; -} - - -std::string Database::getLastError(Connection& connection) const -{ - Document::Ptr errorDoc = getLastErrorDoc(connection); - if (!errorDoc.isNull() && errorDoc->isType("err")) - { - return errorDoc->get("err"); - } - - return ""; -} - - -Poco::SharedPtr Database::createCountRequest(const std::string& collectionName) const -{ - Poco::SharedPtr request = createCommand(); - request->setNumberToReturn(1); - request->selector().add("count", collectionName); - return request; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/DeleteRequest.cpp b/base/poco/MongoDB/src/DeleteRequest.cpp deleted file mode 100644 index ba75beb55fb..00000000000 --- a/base/poco/MongoDB/src/DeleteRequest.cpp +++ /dev/null @@ -1,54 +0,0 @@ -// -// DeleteRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: DeleteRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/DeleteRequest.h" - - -namespace Poco { -namespace MongoDB { - - -DeleteRequest::DeleteRequest(const std::string& collectionName, DeleteRequest::Flags flags): - RequestMessage(MessageHeader::OP_DELETE), - _flags(flags), - _fullCollectionName(collectionName), - _selector() -{ -} - - -DeleteRequest::DeleteRequest(const std::string& collectionName, bool justOne): - RequestMessage(MessageHeader::OP_DELETE), - _flags(justOne ? DELETE_SINGLE_REMOVE : DELETE_DEFAULT), - _fullCollectionName(collectionName), - _selector() -{ -} - - -DeleteRequest::~DeleteRequest() -{ -} - - -void DeleteRequest::buildRequest(BinaryWriter& writer) -{ - writer << 0; // 0 - reserved for future use - BSONWriter(writer).writeCString(_fullCollectionName); - writer << _flags; - _selector.write(writer); -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/Document.cpp b/base/poco/MongoDB/src/Document.cpp deleted file mode 100644 index f7c5c9c5dc6..00000000000 --- a/base/poco/MongoDB/src/Document.cpp +++ /dev/null @@ -1,227 +0,0 @@ -// -// Document.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Document -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Document.h" -#include "Poco/MongoDB/Binary.h" -#include "Poco/MongoDB/ObjectId.h" -#include "Poco/MongoDB/Array.h" -#include "Poco/MongoDB/RegularExpression.h" -#include "Poco/MongoDB/JavaScriptCode.h" -#include - - -namespace Poco { -namespace MongoDB { - - -Document::Document() -{ -} - - -Document::~Document() -{ -} - - -Array& Document::addNewArray(const std::string& name) -{ - Array::Ptr newArray = new Array(); - add(name, newArray); - return *newArray; -} - - -Element::Ptr Document::get(const std::string& name) const -{ - Element::Ptr element; - - ElementSet::const_iterator it = std::find_if(_elements.begin(), _elements.end(), ElementFindByName(name)); - if (it != _elements.end()) - { - return *it; - } - - return element; -} - - -Int64 Document::getInteger(const std::string& name) const -{ - Element::Ptr element = get(name); - if (element.isNull()) throw Poco::NotFoundException(name); - - if (ElementTraits::TypeId == element->type()) - { - ConcreteElement* concrete = dynamic_cast*>(element.get()); - if (concrete) return static_cast(concrete->value()); - } - else if (ElementTraits::TypeId == element->type()) - { - ConcreteElement* concrete = dynamic_cast*>(element.get()); - if (concrete) return concrete->value(); - } - else if (ElementTraits::TypeId == element->type()) - { - ConcreteElement* concrete = dynamic_cast*>(element.get()); - if (concrete) return concrete->value(); - } - throw Poco::BadCastException("Invalid type mismatch!"); -} - - -void Document::read(BinaryReader& reader) -{ - int size; - reader >> size; - - unsigned char type; - reader >> type; - - while (type != '\0') - { - Element::Ptr element; - - std::string name = BSONReader(reader).readCString(); - - switch (type) - { - case ElementTraits::TypeId: - element = new ConcreteElement(name, 0); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, 0); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, ""); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new Document); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new Array); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new Binary); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new ObjectId); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, false); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, Poco::Timestamp()); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, BSONTimestamp()); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, NullValue(0)); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new RegularExpression()); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, new JavaScriptCode()); - break; - case ElementTraits::TypeId: - element = new ConcreteElement(name, 0); - break; - default: - { - std::stringstream ss; - ss << "Element " << name << " contains an unsupported type 0x" << std::hex << (int) type; - throw Poco::NotImplementedException(ss.str()); - } - //TODO: x0F -> JavaScript code with scope - // xFF -> Min Key - // x7F -> Max Key - } - - element->read(reader); - _elements.push_back(element); - - reader >> type; - } -} - - -std::string Document::toString(int indent) const -{ - std::ostringstream oss; - - oss << '{'; - - if (indent > 0) oss << std::endl; - - - for (ElementSet::const_iterator it = _elements.begin(); it != _elements.end(); ++it) - { - if (it != _elements.begin()) - { - oss << ','; - if (indent > 0) oss << std::endl; - } - - for (int i = 0; i < indent; ++i) oss << ' '; - - oss << '"' << (*it)->name() << '"'; - oss << (indent > 0 ? " : " : ":"); - - oss << (*it)->toString(indent > 0 ? indent + 2 : 0); - } - - if (indent > 0) - { - oss << std::endl; - if (indent >= 2) indent -= 2; - - for (int i = 0; i < indent; ++i) oss << ' '; - } - - oss << '}'; - - return oss.str(); -} - - -void Document::write(BinaryWriter& writer) -{ - if (_elements.empty()) - { - writer << 5; - } - else - { - std::stringstream sstream; - Poco::BinaryWriter tempWriter(sstream, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - for (ElementSet::iterator it = _elements.begin(); it != _elements.end(); ++it) - { - tempWriter << static_cast((*it)->type()); - BSONWriter(tempWriter).writeCString((*it)->name()); - Element::Ptr element = *it; - element->write(tempWriter); - } - tempWriter.flush(); - - Poco::Int32 len = static_cast(5 + sstream.tellp()); /* 5 = sizeof(len) + 0-byte */ - writer << len; - writer.writeRaw(sstream.str()); - } - writer << '\0'; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/Element.cpp b/base/poco/MongoDB/src/Element.cpp deleted file mode 100644 index f91ce264493..00000000000 --- a/base/poco/MongoDB/src/Element.cpp +++ /dev/null @@ -1,32 +0,0 @@ -// -// Element.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Element -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Element.h" - - -namespace Poco { -namespace MongoDB { - - -Element::Element(const std::string& name) : _name(name) -{ -} - - -Element::~Element() -{ -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/GetMoreRequest.cpp b/base/poco/MongoDB/src/GetMoreRequest.cpp deleted file mode 100644 index 2c1f6909eb7..00000000000 --- a/base/poco/MongoDB/src/GetMoreRequest.cpp +++ /dev/null @@ -1,46 +0,0 @@ -// -// GetMoreRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: GetMoreRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/GetMoreRequest.h" -#include "Poco/MongoDB/Element.h" - - -namespace Poco { -namespace MongoDB { - - -GetMoreRequest::GetMoreRequest(const std::string& collectionName, Int64 cursorID): - RequestMessage(MessageHeader::OP_GET_MORE), - _fullCollectionName(collectionName), - _numberToReturn(100), - _cursorID(cursorID) -{ -} - - -GetMoreRequest::~GetMoreRequest() -{ -} - - -void GetMoreRequest::buildRequest(BinaryWriter& writer) -{ - writer << 0; // 0 - reserved for future use - BSONWriter(writer).writeCString(_fullCollectionName); - writer << _numberToReturn; - writer << _cursorID; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/InsertRequest.cpp b/base/poco/MongoDB/src/InsertRequest.cpp deleted file mode 100644 index 65be5654b3e..00000000000 --- a/base/poco/MongoDB/src/InsertRequest.cpp +++ /dev/null @@ -1,49 +0,0 @@ -// -// InsertRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: InsertRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/InsertRequest.h" - - -namespace Poco { -namespace MongoDB { - - -InsertRequest::InsertRequest(const std::string& collectionName, Flags flags): - RequestMessage(MessageHeader::OP_INSERT), - _flags(flags), - _fullCollectionName(collectionName) -{ -} - - -InsertRequest::~InsertRequest() -{ -} - - -void InsertRequest::buildRequest(BinaryWriter& writer) -{ - poco_assert (!_documents.empty()); - - writer << _flags; - BSONWriter bsonWriter(writer); - bsonWriter.writeCString(_fullCollectionName); - for (Document::Vector::iterator it = _documents.begin(); it != _documents.end(); ++it) - { - bsonWriter.write(*it); - } -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/JavaScriptCode.cpp b/base/poco/MongoDB/src/JavaScriptCode.cpp deleted file mode 100644 index 41f5fcabe6b..00000000000 --- a/base/poco/MongoDB/src/JavaScriptCode.cpp +++ /dev/null @@ -1,33 +0,0 @@ -// -// JavaScriptCode.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: JavaScriptCode -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/JavaScriptCode.h" - - -namespace Poco { -namespace MongoDB { - - -JavaScriptCode::JavaScriptCode() -{ - -} - - -JavaScriptCode::~JavaScriptCode() -{ -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/KillCursorsRequest.cpp b/base/poco/MongoDB/src/KillCursorsRequest.cpp deleted file mode 100644 index 448002aa16a..00000000000 --- a/base/poco/MongoDB/src/KillCursorsRequest.cpp +++ /dev/null @@ -1,44 +0,0 @@ -// -// KillCursorsRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: KillCursorsRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/KillCursorsRequest.h" - - -namespace Poco { -namespace MongoDB { - - -KillCursorsRequest::KillCursorsRequest(): - RequestMessage(MessageHeader::OP_KILL_CURSORS) -{ -} - - -KillCursorsRequest::~KillCursorsRequest() -{ -} - - -void KillCursorsRequest::buildRequest(BinaryWriter& writer) -{ - writer << 0; // 0 - reserved for future use - writer << static_cast(_cursors.size()); - for (std::vector::iterator it = _cursors.begin(); it != _cursors.end(); ++it) - { - writer << *it; - } -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/Message.cpp b/base/poco/MongoDB/src/Message.cpp deleted file mode 100644 index 7b1cb23bab6..00000000000 --- a/base/poco/MongoDB/src/Message.cpp +++ /dev/null @@ -1,33 +0,0 @@ -// -// Message.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: Message -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Message.h" - - -namespace Poco { -namespace MongoDB { - - -Message::Message(MessageHeader::OpCode opcode): - _header(opcode) -{ -} - - -Message::~Message() -{ -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/MessageHeader.cpp b/base/poco/MongoDB/src/MessageHeader.cpp deleted file mode 100644 index b472bcec465..00000000000 --- a/base/poco/MongoDB/src/MessageHeader.cpp +++ /dev/null @@ -1,63 +0,0 @@ -// -// MessageHeader.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: MessageHeader -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/Message.h" -#include "Poco/Exception.h" - - -namespace Poco { -namespace MongoDB { - - -MessageHeader::MessageHeader(OpCode opCode): - _messageLength(0), - _requestID(0), - _responseTo(0), - _opCode(opCode) -{ -} - - -MessageHeader::~MessageHeader() -{ -} - - -void MessageHeader::read(BinaryReader& reader) -{ - reader >> _messageLength; - reader >> _requestID; - reader >> _responseTo; - - Int32 opCode; - reader >> opCode; - _opCode = static_cast(opCode); - - if (!reader.good()) - { - throw IOException("Failed to read from socket"); - } -} - - -void MessageHeader::write(BinaryWriter& writer) -{ - writer << _messageLength; - writer << _requestID; - writer << _responseTo; - writer << static_cast(_opCode); -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/ObjectId.cpp b/base/poco/MongoDB/src/ObjectId.cpp deleted file mode 100644 index e360d129843..00000000000 --- a/base/poco/MongoDB/src/ObjectId.cpp +++ /dev/null @@ -1,66 +0,0 @@ -// -// ObjectId.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: ObjectId -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/ObjectId.h" -#include "Poco/Format.h" -#include - - -namespace Poco { -namespace MongoDB { - - -ObjectId::ObjectId() -{ - std::memset(_id, 0, sizeof(_id)); -} - - -ObjectId::ObjectId(const std::string& id) -{ - poco_assert_dbg(id.size() == 24); - - const char* p = id.c_str(); - for (std::size_t i = 0; i < 12; ++i) - { - _id[i] = fromHex(p); - p += 2; - } -} - - -ObjectId::ObjectId(const ObjectId& copy) -{ - std::memcpy(_id, copy._id, sizeof(_id)); -} - - -ObjectId::~ObjectId() -{ -} - - -std::string ObjectId::toString(const std::string& fmt) const -{ - std::string s; - - for (int i = 0; i < 12; ++i) - { - s += Poco::format(fmt, (unsigned int) _id[i]); - } - return s; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/OpMsgCursor.cpp b/base/poco/MongoDB/src/OpMsgCursor.cpp deleted file mode 100644 index 6abd45ecf76..00000000000 --- a/base/poco/MongoDB/src/OpMsgCursor.cpp +++ /dev/null @@ -1,187 +0,0 @@ -// -// OpMsgCursor.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: OpMsgCursor -// -// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/OpMsgCursor.h" -#include "Poco/MongoDB/Array.h" - -// -// NOTE: -// -// MongoDB specification indicates that the flag MSG_EXHAUST_ALLOWED shall be -// used in the request when the receiver is ready to receive multiple messages -// without sending additional requests in between. Sender (MongoDB) indicates -// that more messages follow with flag MSG_MORE_TO_COME. -// -// It seems that this does not work properly. MSG_MORE_TO_COME is set and reading -// next messages sometimes works, however often the data is missing in response -// or the message header contains wrong message length and reading blocks. -// Opcode in the header is correct. -// -// Using MSG_EXHAUST_ALLOWED is therefore currently disabled. -// -// It seems that related JIRA ticket is: -// -// https://jira.mongodb.org/browse/SERVER-57297 -// -// https://github.com/mongodb/specifications/blob/master/source/message/OP_MSG.rst -// - -#define MONGODB_EXHAUST_ALLOWED_WORKS false - -namespace Poco { -namespace MongoDB { - - -[[ maybe_unused ]] static const std::string keyCursor {"cursor"}; -[[ maybe_unused ]] static const std::string keyFirstBatch {"firstBatch"}; -[[ maybe_unused ]] static const std::string keyNextBatch {"nextBatch"}; - -static Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc); - - -OpMsgCursor::OpMsgCursor(const std::string& db, const std::string& collection): -#if MONGODB_EXHAUST_ALLOWED_WORKS - _query(db, collection, OpMsgMessage::MSG_EXHAUST_ALLOWED) -#else - _query(db, collection) -#endif -{ -} - -OpMsgCursor::~OpMsgCursor() -{ - try - { - poco_assert_dbg(_cursorID == 0); - } - catch (...) - { - } -} - - -void OpMsgCursor::setEmptyFirstBatch(bool empty) -{ - _emptyFirstBatch = empty; -} - - -bool OpMsgCursor::emptyFirstBatch() const -{ - return _emptyFirstBatch; -} - - -void OpMsgCursor::setBatchSize(Int32 batchSize) -{ - _batchSize = batchSize; -} - - -Int32 OpMsgCursor::batchSize() const -{ - return _batchSize; -} - - -OpMsgMessage& OpMsgCursor::next(Connection& connection) -{ - if (_cursorID == 0) - { - _response.clear(); - - if (_emptyFirstBatch || _batchSize > 0) - { - Int32 bsize = _emptyFirstBatch ? 0 : _batchSize; - if (_query.commandName() == OpMsgMessage::CMD_FIND) - { - _query.body().add("batchSize", bsize); - } - else if (_query.commandName() == OpMsgMessage::CMD_AGGREGATE) - { - auto& cursorDoc = _query.body().addNewDocument("cursor"); - cursorDoc.add("batchSize", bsize); - } - } - - connection.sendRequest(_query, _response); - - const auto& rdoc = _response.body(); - _cursorID = cursorIdFromResponse(rdoc); - } - else - { -#if MONGODB_EXHAUST_ALLOWED_WORKS - std::cout << "Response flags: " << _response.flags() << std::endl; - if (_response.flags() & OpMsgMessage::MSG_MORE_TO_COME) - { - std::cout << "More to come. Reading more response: " << std::endl; - _response.clear(); - connection.readResponse(_response); - } - else -#endif - { - _response.clear(); - _query.setCursor(_cursorID, _batchSize); - connection.sendRequest(_query, _response); - } - } - - const auto& rdoc = _response.body(); - _cursorID = cursorIdFromResponse(rdoc); - - return _response; -} - - -void OpMsgCursor::kill(Connection& connection) -{ - _response.clear(); - if (_cursorID != 0) - { - _query.setCommandName(OpMsgMessage::CMD_KILL_CURSORS); - - MongoDB::Array::Ptr cursors = new MongoDB::Array(); - cursors->add(_cursorID); - _query.body().add("cursors", cursors); - - connection.sendRequest(_query, _response); - - const auto killed = _response.body().get("cursorsKilled", nullptr); - if (!killed || killed->size() != 1 || killed->get(0, -1) != _cursorID) - { - throw Poco::ProtocolException("Cursor not killed as expected: " + std::to_string(_cursorID)); - } - - _cursorID = 0; - _query.clear(); - _response.clear(); - } -} - - -Poco::Int64 cursorIdFromResponse(const MongoDB::Document& doc) -{ - Poco::Int64 id {0}; - auto cursorDoc = doc.get(keyCursor, nullptr); - if(cursorDoc) - { - id = cursorDoc->get("id", 0); - } - return id; -} - - -} } // Namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/OpMsgMessage.cpp b/base/poco/MongoDB/src/OpMsgMessage.cpp deleted file mode 100644 index 2b55772ca59..00000000000 --- a/base/poco/MongoDB/src/OpMsgMessage.cpp +++ /dev/null @@ -1,412 +0,0 @@ -// -// OpMsgMessage.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: OpMsgMessage -// -// Copyright (c) 2022, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - -#include "Poco/MongoDB/OpMsgMessage.h" -#include "Poco/MongoDB/MessageHeader.h" -#include "Poco/MongoDB/Array.h" -#include "Poco/StreamCopier.h" -#include "Poco/Logger.h" - -#define POCO_MONGODB_DUMP false - -namespace Poco { -namespace MongoDB { - -// Query and write -const std::string OpMsgMessage::CMD_INSERT { "insert" }; -const std::string OpMsgMessage::CMD_DELETE { "delete" }; -const std::string OpMsgMessage::CMD_UPDATE { "update" }; -const std::string OpMsgMessage::CMD_FIND { "find" }; -const std::string OpMsgMessage::CMD_FIND_AND_MODIFY { "findAndModify" }; -const std::string OpMsgMessage::CMD_GET_MORE { "getMore" }; - -// Aggregation -const std::string OpMsgMessage::CMD_AGGREGATE { "aggregate" }; -const std::string OpMsgMessage::CMD_COUNT { "count" }; -const std::string OpMsgMessage::CMD_DISTINCT { "distinct" }; -const std::string OpMsgMessage::CMD_MAP_REDUCE { "mapReduce" }; - -// Replication and administration -const std::string OpMsgMessage::CMD_HELLO { "hello" }; -const std::string OpMsgMessage::CMD_REPL_SET_GET_STATUS { "replSetGetStatus" }; -const std::string OpMsgMessage::CMD_REPL_SET_GET_CONFIG { "replSetGetConfig" }; - -const std::string OpMsgMessage::CMD_CREATE { "create" }; -const std::string OpMsgMessage::CMD_CREATE_INDEXES { "createIndexes" }; -const std::string OpMsgMessage::CMD_DROP { "drop" }; -const std::string OpMsgMessage::CMD_DROP_DATABASE { "dropDatabase" }; -const std::string OpMsgMessage::CMD_KILL_CURSORS { "killCursors" }; -const std::string OpMsgMessage::CMD_LIST_DATABASES { "listDatabases" }; -const std::string OpMsgMessage::CMD_LIST_INDEXES { "listIndexes" }; - -// Diagnostic -const std::string OpMsgMessage::CMD_BUILD_INFO { "buildInfo" }; -const std::string OpMsgMessage::CMD_COLL_STATS { "collStats" }; -const std::string OpMsgMessage::CMD_DB_STATS { "dbStats" }; -const std::string OpMsgMessage::CMD_HOST_INFO { "hostInfo" }; - - -static const std::string& commandIdentifier(const std::string& command); - /// Commands have different names for the payload that is sent in a separate section - - -static const std::string keyCursor {"cursor"}; -static const std::string keyFirstBatch {"firstBatch"}; -static const std::string keyNextBatch {"nextBatch"}; - - -OpMsgMessage::OpMsgMessage() : - Message(MessageHeader::OP_MSG) -{ -} - - -OpMsgMessage::OpMsgMessage(const std::string& databaseName, const std::string& collectionName, UInt32 flags) : - Message(MessageHeader::OP_MSG), - _databaseName(databaseName), - _collectionName(collectionName), - _flags(flags) -{ -} - - -OpMsgMessage::~OpMsgMessage() -{ -} - -const std::string& OpMsgMessage::databaseName() const -{ - return _databaseName; -} - - -const std::string& OpMsgMessage::collectionName() const -{ - return _collectionName; -} - - -void OpMsgMessage::setCommandName(const std::string& command) -{ - _commandName = command; - _body.clear(); - - // IMPORTANT: Command name must be first - if (_collectionName.empty()) - { - // Collection is not specified. It is assumed that this particular command does - // not need it. - _body.add(_commandName, Int32(1)); - } - else - { - _body.add(_commandName, _collectionName); - } - _body.add("$db", _databaseName); -} - - -void OpMsgMessage::setCursor(Poco::Int64 cursorID, Poco::Int32 batchSize) -{ - _commandName = OpMsgMessage::CMD_GET_MORE; - _body.clear(); - - // IMPORTANT: Command name must be first - _body.add(_commandName, cursorID); - _body.add("$db", _databaseName); - _body.add("collection", _collectionName); - if (batchSize > 0) - { - _body.add("batchSize", batchSize); - } -} - - -const std::string& OpMsgMessage::commandName() const -{ - return _commandName; -} - - -void OpMsgMessage::setAcknowledgedRequest(bool ack) -{ - const auto& id = commandIdentifier(_commandName); - if (id.empty()) - return; - - _acknowledged = ack; - - auto writeConcern = _body.get("writeConcern", nullptr); - if (writeConcern) - writeConcern->remove("w"); - - if (ack) - { - _flags = _flags & (~MSG_MORE_TO_COME); - } - else - { - _flags = _flags | MSG_MORE_TO_COME; - if (!writeConcern) - _body.addNewDocument("writeConcern").add("w", 0); - else - writeConcern->add("w", 0); - } - -} - - -bool OpMsgMessage::acknowledgedRequest() const -{ - return _acknowledged; -} - - -UInt32 OpMsgMessage::flags() const -{ - return _flags; -} - - -Document& OpMsgMessage::body() -{ - return _body; -} - - -const Document& OpMsgMessage::body() const -{ - return _body; -} - - -Document::Vector& OpMsgMessage::documents() -{ - return _documents; -} - - -const Document::Vector& OpMsgMessage::documents() const -{ - return _documents; -} - - -bool OpMsgMessage::responseOk() const -{ - Poco::Int64 ok {false}; - if (_body.exists("ok")) - { - ok = _body.getInteger("ok"); - } - return (ok != 0); -} - - -void OpMsgMessage::clear() -{ - _flags = MSG_FLAGS_DEFAULT; - _commandName.clear(); - _body.clear(); - _documents.clear(); -} - - -void OpMsgMessage::send(std::ostream& ostr) -{ - BinaryWriter socketWriter(ostr, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - - // Serialise the body - std::stringstream ss; - BinaryWriter writer(ss, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - writer << _flags; - - writer << PAYLOAD_TYPE_0; - _body.write(writer); - - if (!_documents.empty()) - { - // Serialise attached documents - - std::stringstream ssdoc; - BinaryWriter wdoc(ssdoc, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - for (auto& doc: _documents) - { - doc->write(wdoc); - } - wdoc.flush(); - - const std::string& identifier = commandIdentifier(_commandName); - const Poco::Int32 size = static_cast(sizeof(size) + identifier.size() + 1 + ssdoc.tellp()); - writer << PAYLOAD_TYPE_1; - writer << size; - writer.writeCString(identifier.c_str()); - StreamCopier::copyStream(ssdoc, ss); - } - writer.flush(); - -#if POCO_MONGODB_DUMP - const std::string section = ss.str(); - std::string dump; - Logger::formatDump(dump, section.data(), section.length()); - std::cout << dump << std::endl; -#endif - - messageLength(static_cast(ss.tellp())); - - _header.write(socketWriter); - StreamCopier::copyStream(ss, ostr); - - ostr.flush(); -} - - -void OpMsgMessage::read(std::istream& istr) -{ - std::string message; - { - BinaryReader reader(istr, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); - _header.read(reader); - - poco_assert_dbg(_header.opCode() == _header.OP_MSG); - - const std::streamsize remainingSize {_header.getMessageLength() - _header.MSG_HEADER_SIZE }; - message.reserve(remainingSize); - -#if POCO_MONGODB_DUMP - std::cout - << "Message hdr: " << _header.getMessageLength() << " " << remainingSize << " " - << _header.opCode() << " " << _header.getRequestID() << " " << _header.responseTo() - << std::endl; -#endif - - reader.readRaw(remainingSize, message); - -#if POCO_MONGODB_DUMP - std::string dump; - Logger::formatDump(dump, message.data(), message.length()); - std::cout << dump << std::endl; -#endif - } - // Read complete message and then interpret it. - - std::istringstream msgss(message); - BinaryReader reader(msgss, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); - - Poco::UInt8 payloadType {0xFF}; - - reader >> _flags; - reader >> payloadType; - poco_assert_dbg(payloadType == PAYLOAD_TYPE_0); - - _body.read(reader); - - // Read next sections from the buffer - while (msgss.good()) - { - // NOTE: Not tested yet with database, because it returns everything in the body. - // Does MongoDB ever return documents as Payload type 1? - reader >> payloadType; - if (!msgss.good()) - { - break; - } - poco_assert_dbg(payloadType == PAYLOAD_TYPE_1); -#if POCO_MONGODB_DUMP - std::cout << "section payload: " << payloadType << std::endl; -#endif - - Poco::Int32 sectionSize {0}; - reader >> sectionSize; - poco_assert_dbg(sectionSize > 0); - -#if POCO_MONGODB_DUMP - std::cout << "section size: " << sectionSize << std::endl; -#endif - std::streamoff offset = sectionSize - sizeof(sectionSize); - std::streampos endOfSection = msgss.tellg() + offset; - - std::string identifier; - reader.readCString(identifier); -#if POCO_MONGODB_DUMP - std::cout << "section identifier: " << identifier << std::endl; -#endif - - // Loop to read documents from this section. - while (msgss.tellg() < endOfSection) - { -#if POCO_MONGODB_DUMP - std::cout << "section doc: " << msgss.tellg() << " " << endOfSection << std::endl; -#endif - Document::Ptr doc = new Document(); - doc->read(reader); - _documents.push_back(doc); - if (msgss.tellg() < 0) - { - break; - } - } - } - - // Extract documents from the cursor batch if they are there. - MongoDB::Array::Ptr batch; - auto curDoc = _body.get(keyCursor, nullptr); - if (curDoc) - { - batch = curDoc->get(keyFirstBatch, nullptr); - if (!batch) - { - batch = curDoc->get(keyNextBatch, nullptr); - } - } - if (batch) - { - for(std::size_t i = 0; i < batch->size(); i++) - { - const auto& d = batch->get(i, nullptr); - if (d) - { - _documents.push_back(d); - } - } - } - -} - -const std::string& commandIdentifier(const std::string& command) -{ - // Names of identifiers for commands that send bulk documents in the request - // The identifier is set in the section type 1. - static std::map identifiers { - { OpMsgMessage::CMD_INSERT, "documents" }, - { OpMsgMessage::CMD_DELETE, "deletes" }, - { OpMsgMessage::CMD_UPDATE, "updates" }, - - // Not sure if create index can send document section - { OpMsgMessage::CMD_CREATE_INDEXES, "indexes" } - }; - - const auto i = identifiers.find(command); - if (i != identifiers.end()) - { - return i->second; - } - - // This likely means that documents are incorrectly set for a command - // that does not send list of documents in section type 1. - static const std::string emptyIdentifier; - return emptyIdentifier; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/QueryRequest.cpp b/base/poco/MongoDB/src/QueryRequest.cpp deleted file mode 100644 index 6d7d23a8456..00000000000 --- a/base/poco/MongoDB/src/QueryRequest.cpp +++ /dev/null @@ -1,54 +0,0 @@ -// -// QueryRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: QueryRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/QueryRequest.h" - - -namespace Poco { -namespace MongoDB { - - -QueryRequest::QueryRequest(const std::string& collectionName, QueryRequest::Flags flags): - RequestMessage(MessageHeader::OP_QUERY), - _flags(flags), - _fullCollectionName(collectionName), - _numberToSkip(0), - _numberToReturn(100), - _selector(), - _returnFieldSelector() -{ -} - - -QueryRequest::~QueryRequest() -{ -} - - -void QueryRequest::buildRequest(BinaryWriter& writer) -{ - writer << _flags; - BSONWriter(writer).writeCString(_fullCollectionName); - writer << _numberToSkip; - writer << _numberToReturn; - _selector.write(writer); - - if (!_returnFieldSelector.empty()) - { - _returnFieldSelector.write(writer); - } -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/RegularExpression.cpp b/base/poco/MongoDB/src/RegularExpression.cpp deleted file mode 100644 index 5f7eb6bb51b..00000000000 --- a/base/poco/MongoDB/src/RegularExpression.cpp +++ /dev/null @@ -1,71 +0,0 @@ -// -// RegularExpression.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: RegularExpression -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/RegularExpression.h" -#include - - -namespace Poco { -namespace MongoDB { - - -RegularExpression::RegularExpression() -{ -} - - -RegularExpression::RegularExpression(const std::string& pattern, const std::string& options): - _pattern(pattern), - _options(options) -{ -} - - -RegularExpression::~RegularExpression() -{ -} - - -SharedPtr RegularExpression::createRE() const -{ - int options = 0; - for (std::string::const_iterator optIt = _options.begin(); optIt != _options.end(); ++optIt) - { - switch (*optIt) - { - case 'i': // Case Insensitive - options |= Poco::RegularExpression::RE_CASELESS; - break; - case 'm': // Multiline matching - options |= Poco::RegularExpression::RE_MULTILINE; - break; - case 'x': // Verbose mode - //No equivalent in Poco - break; - case 'l': // \w \W Locale dependent - //No equivalent in Poco - break; - case 's': // Dotall mode - options |= Poco::RegularExpression::RE_DOTALL; - break; - case 'u': // \w \W Unicode - //No equivalent in Poco - break; - } - } - return new Poco::RegularExpression(_pattern, options); -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/ReplicaSet.cpp b/base/poco/MongoDB/src/ReplicaSet.cpp deleted file mode 100644 index fce2f2bdada..00000000000 --- a/base/poco/MongoDB/src/ReplicaSet.cpp +++ /dev/null @@ -1,89 +0,0 @@ -// -// ReplicaSet.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: ReplicaSet -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/ReplicaSet.h" -#include "Poco/MongoDB/QueryRequest.h" -#include "Poco/MongoDB/ResponseMessage.h" - - -namespace Poco { -namespace MongoDB { - - -ReplicaSet::ReplicaSet(const std::vector &addresses): - _addresses(addresses) -{ -} - - -ReplicaSet::~ReplicaSet() -{ -} - - -Connection::Ptr ReplicaSet::findMaster() -{ - Connection::Ptr master; - - for (std::vector::iterator it = _addresses.begin(); it != _addresses.end(); ++it) - { - master = isMaster(*it); - if (!master.isNull()) - { - break; - } - } - - return master; -} - - -Connection::Ptr ReplicaSet::isMaster(const Net::SocketAddress& address) -{ - Connection::Ptr conn = new Connection(); - - try - { - conn->connect(address); - - QueryRequest request("admin.$cmd"); - request.setNumberToReturn(1); - request.selector().add("isMaster", 1); - - ResponseMessage response; - conn->sendRequest(request, response); - - if (response.documents().size() > 0) - { - Document::Ptr doc = response.documents()[0]; - if (doc->get("ismaster")) - { - return conn; - } - else if (doc->exists("primary")) - { - return isMaster(Net::SocketAddress(doc->get("primary"))); - } - } - } - catch (...) - { - conn = 0; - } - - return 0; -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/RequestMessage.cpp b/base/poco/MongoDB/src/RequestMessage.cpp deleted file mode 100644 index 999ed8a6ba1..00000000000 --- a/base/poco/MongoDB/src/RequestMessage.cpp +++ /dev/null @@ -1,51 +0,0 @@ -// -// RequestMessage.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: RequestMessage -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/RequestMessage.h" -#include "Poco/Net/SocketStream.h" -#include "Poco/StreamCopier.h" - - -namespace Poco { -namespace MongoDB { - - -RequestMessage::RequestMessage(MessageHeader::OpCode opcode): - Message(opcode) -{ -} - - -RequestMessage::~RequestMessage() -{ -} - - -void RequestMessage::send(std::ostream& ostr) -{ - std::stringstream ss; - BinaryWriter requestWriter(ss, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - buildRequest(requestWriter); - requestWriter.flush(); - - messageLength(static_cast(ss.tellp())); - - BinaryWriter socketWriter(ostr, BinaryWriter::LITTLE_ENDIAN_BYTE_ORDER); - _header.write(socketWriter); - StreamCopier::copyStream(ss, ostr); - ostr.flush(); -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/ResponseMessage.cpp b/base/poco/MongoDB/src/ResponseMessage.cpp deleted file mode 100644 index e8216767494..00000000000 --- a/base/poco/MongoDB/src/ResponseMessage.cpp +++ /dev/null @@ -1,80 +0,0 @@ -// -// ResponseMessage.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: ResponseMessage -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/ResponseMessage.h" -#include "Poco/Net/SocketStream.h" - - -namespace Poco { -namespace MongoDB { - - -ResponseMessage::ResponseMessage(): - Message(MessageHeader::OP_REPLY), - _responseFlags(0), - _cursorID(0), - _startingFrom(0), - _numberReturned(0) -{ -} - - -ResponseMessage::ResponseMessage(const Int64& cursorID): - Message(MessageHeader::OP_REPLY), - _responseFlags(0), - _cursorID(cursorID), - _startingFrom(0), - _numberReturned(0) -{ -} - - -ResponseMessage::~ResponseMessage() -{ -} - - -void ResponseMessage::clear() -{ - _responseFlags = 0; - _startingFrom = 0; - _cursorID = 0; - _numberReturned = 0; - _documents.clear(); -} - - -void ResponseMessage::read(std::istream& istr) -{ - clear(); - - BinaryReader reader(istr, BinaryReader::LITTLE_ENDIAN_BYTE_ORDER); - - _header.read(reader); - - reader >> _responseFlags; - reader >> _cursorID; - reader >> _startingFrom; - reader >> _numberReturned; - - for (int i = 0; i < _numberReturned; ++i) - { - Document::Ptr doc = new Document(); - doc->read(reader); - _documents.push_back(doc); - } -} - - -} } // namespace Poco::MongoDB diff --git a/base/poco/MongoDB/src/UpdateRequest.cpp b/base/poco/MongoDB/src/UpdateRequest.cpp deleted file mode 100644 index 7477fc752d5..00000000000 --- a/base/poco/MongoDB/src/UpdateRequest.cpp +++ /dev/null @@ -1,47 +0,0 @@ -// -// UpdateRequest.cpp -// -// Library: MongoDB -// Package: MongoDB -// Module: UpdateRequest -// -// Copyright (c) 2012, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MongoDB/UpdateRequest.h" - - -namespace Poco { -namespace MongoDB { - - -UpdateRequest::UpdateRequest(const std::string& collectionName, UpdateRequest::Flags flags): - RequestMessage(MessageHeader::OP_UPDATE), - _flags(flags), - _fullCollectionName(collectionName), - _selector(), - _update() -{ -} - - -UpdateRequest::~UpdateRequest() -{ -} - - -void UpdateRequest::buildRequest(BinaryWriter& writer) -{ - writer << 0; // 0 - reserved for future use - BSONWriter(writer).writeCString(_fullCollectionName); - writer << _flags; - _selector.write(writer); - _update.write(writer); -} - - -} } // namespace Poco::MongoDB diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index 834b57757b9..151edeb0a00 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -11,11 +11,6 @@ MongoDB engine is read-only table engine which allows to read data from remote [ Only MongoDB v3.6+ servers are supported. [Seed list(`mongodb+srv`)](https://www.mongodb.com/docs/manual/reference/glossary/#std-term-seed-list) is not yet supported. -:::note -If you're facing troubles, please report the issue, and try to use [the legacy implementation](../../../operations/server-configuration-parameters/settings.md#use_legacy_mongodb_integration). -Keep in mind that it is deprecated, and will be removed in next releases. -::: - ## Creating a Table {#creating-a-table} ``` sql diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ca4938b1a47..c2abc721882 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -3277,11 +3277,3 @@ Type: UInt64 Default value: 100 Zero means unlimited - -## use_legacy_mongodb_integration - -Use the legacy MongoDB integration implementation. Deprecated. - -Type: Bool - -Default value: `true`. diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 900d9cb8e01..522b9a74cff 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -188,9 +188,9 @@ int mainEntryClickHouseFormat(int argc, char ** argv) registerInterpreters(); registerFunctions(); registerAggregateFunctions(); - registerTableFunctions(false); + registerTableFunctions(); registerDatabases(); - registerStorages(false); + registerStorages(); registerFormats(); std::unordered_set additional_names; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e6f8ecef097..51c41248dfa 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -110,7 +110,6 @@ namespace ServerSetting extern const ServerSettingsString uncompressed_cache_policy; extern const ServerSettingsUInt64 uncompressed_cache_size; extern const ServerSettingsDouble uncompressed_cache_size_ratio; - extern const ServerSettingsBool use_legacy_mongodb_integration; } namespace ErrorCodes @@ -549,10 +548,10 @@ try /// Don't initialize DateLUT registerFunctions(); registerAggregateFunctions(); - registerTableFunctions(server_settings[ServerSetting::use_legacy_mongodb_integration]); + registerTableFunctions(); registerDatabases(); - registerStorages(server_settings[ServerSetting::use_legacy_mongodb_integration]); - registerDictionaries(server_settings[ServerSetting::use_legacy_mongodb_integration]); + registerStorages(); + registerDictionaries(); registerDisks(/* global_skip_access_check= */ true); registerFormats(); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 68f262079ff..b1b81d8da4c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -279,7 +279,6 @@ namespace ServerSetting extern const ServerSettingsString uncompressed_cache_policy; extern const ServerSettingsUInt64 uncompressed_cache_size; extern const ServerSettingsDouble uncompressed_cache_size_ratio; - extern const ServerSettingsBool use_legacy_mongodb_integration; } } @@ -912,10 +911,10 @@ try registerInterpreters(); registerFunctions(); registerAggregateFunctions(); - registerTableFunctions(server_settings[ServerSetting::use_legacy_mongodb_integration]); + registerTableFunctions(); registerDatabases(); - registerStorages(server_settings[ServerSetting::use_legacy_mongodb_integration]); - registerDictionaries(server_settings[ServerSetting::use_legacy_mongodb_integration]); + registerStorages(); + registerDictionaries(); registerDisks(/* global_skip_access_check= */ false); registerFormats(); registerRemoteFileMetadatas(); diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3627d760d4c..8ecdf0995a5 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -433,10 +433,6 @@ dbms_target_link_libraries ( Poco::Redis ) -if (USE_MONGODB) - dbms_target_link_libraries (PUBLIC Poco::MongoDB) -endif() - if (TARGET ch_contrib::mongocxx) dbms_target_link_libraries( PUBLIC diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 2f8e7b6843a..163db07f7b7 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -194,7 +194,6 @@ namespace DB DECLARE(UInt64, parts_kill_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to kill_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, parts_killer_pool_size, 128, "Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \ - DECLARE(Bool, use_legacy_mongodb_integration, true, "Use the legacy MongoDB integration implementation. Note: it's highly recommended to set this option to false, since legacy implementation will be removed in the future. Please submit any issues you encounter with the new implementation.", 0) \ \ DECLARE(UInt64, prefetch_threadpool_pool_size, 100, "Size of background pool for prefetches for remote object storages", 0) \ DECLARE(UInt64, prefetch_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0e2e1f8a3f0..874728aeb31 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5715,7 +5715,7 @@ Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting If enabled, only allow identifiers containing alphanumeric characters and underscores. )", 0) \ DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( -If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option does not apply to the legacy implementation or when 'allow_experimental_analyzer=0'. +If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option is not applied when 'allow_experimental_analyzer=0'. )", 0) \ DECLARE(Bool, implicit_select, false, R"( Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index a5bb0a7543c..d1d8a824651 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -40,10 +40,6 @@ target_link_libraries(clickhouse_dictionaries Poco::Redis ) -if (USE_MONGODB) - target_link_libraries(clickhouse_dictionaries PRIVATE Poco::MongoDB) -endif() - target_link_libraries(clickhouse_dictionaries PUBLIC ch_contrib::abseil_swiss_tables) if (TARGET ch_contrib::cassandra) diff --git a/src/Dictionaries/MongoDBPocoLegacyDictionarySource.cpp b/src/Dictionaries/MongoDBPocoLegacyDictionarySource.cpp deleted file mode 100644 index 4495215d826..00000000000 --- a/src/Dictionaries/MongoDBPocoLegacyDictionarySource.cpp +++ /dev/null @@ -1,305 +0,0 @@ -#include "config.h" - -#include "DictionarySourceFactory.h" -#if USE_MONGODB -#include -#include "MongoDBPocoLegacyDictionarySource.h" -#include "DictionaryStructure.h" -#include "registerDictionaries.h" -#include -#include -#endif - -namespace DB -{ - -namespace ErrorCodes -{ -#if USE_MONGODB -extern const int NOT_IMPLEMENTED; -extern const int UNSUPPORTED_METHOD; -extern const int MONGODB_CANNOT_AUTHENTICATE; -#else -extern const int SUPPORT_IS_DISABLED; -#endif -} - -void registerDictionarySourceMongoDBPocoLegacy(DictionarySourceFactory & factory) -{ - #if USE_MONGODB - auto create_mongo_db_dictionary = []( - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & root_config_prefix, - Block & sample_block, - ContextPtr context, - const std::string & /* default_database */, - bool created_from_ddl) - { - const auto config_prefix = root_config_prefix + ".mongodb"; - auto named_collection = created_from_ddl ? tryGetNamedCollectionWithOverrides(config, config_prefix, context) : nullptr; - - String host, username, password, database, method, options, collection; - UInt16 port; - if (named_collection) - { - validateNamedCollection( - *named_collection, - /* required_keys */{"collection"}, - /* optional_keys */ValidateKeysMultiset{ - "host", "port", "user", "password", "db", "database", "uri", "name", "method", "options"}); - - host = named_collection->getOrDefault("host", ""); - port = static_cast(named_collection->getOrDefault("port", 0)); - username = named_collection->getOrDefault("user", ""); - password = named_collection->getOrDefault("password", ""); - database = named_collection->getAnyOrDefault({"db", "database"}, ""); - method = named_collection->getOrDefault("method", ""); - collection = named_collection->getOrDefault("collection", ""); - options = named_collection->getOrDefault("options", ""); - } - else - { - host = config.getString(config_prefix + ".host", ""); - port = config.getUInt(config_prefix + ".port", 0); - username = config.getString(config_prefix + ".user", ""); - password = config.getString(config_prefix + ".password", ""); - database = config.getString(config_prefix + ".db", ""); - method = config.getString(config_prefix + ".method", ""); - collection = config.getString(config_prefix + ".collection"); - options = config.getString(config_prefix + ".options", ""); - } - - if (created_from_ddl) - context->getRemoteHostFilter().checkHostAndPort(host, toString(port)); - - return std::make_unique(dict_struct, - config.getString(config_prefix + ".uri", ""), - host, - port, - username, - password, - method, - database, - collection, - options, - sample_block); - }; - #else - auto create_mongo_db_dictionary = []( - const DictionaryStructure & /* dict_struct */, - const Poco::Util::AbstractConfiguration & /* config */, - const std::string & /* root_config_prefix */, - Block & /* sample_block */, - ContextPtr /* context */, - const std::string & /* default_database */, - bool /* created_from_ddl */) -> DictionarySourcePtr - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Dictionary source of type `mongodb` is disabled because ClickHouse was built without mongodb support."); - }; - #endif - - factory.registerSource("mongodb", create_mongo_db_dictionary); -} - -} - -#if USE_MONGODB -#include -#include -#include -#include -#include -#include -#include -#include - -// only after poco -// naming conflict: -// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); -// src/IO/WriteHelpers.h:146 #define writeCString(s, buf) -#include - - -namespace DB -{ -static const UInt64 max_block_size = 8192; - - -MongoDBPocoLegacyDictionarySource::MongoDBPocoLegacyDictionarySource( - const DictionaryStructure & dict_struct_, - const std::string & uri_, - const std::string & host_, - UInt16 port_, - const std::string & user_, - const std::string & password_, - const std::string & method_, - const std::string & db_, - const std::string & collection_, - const std::string & options_, - const Block & sample_block_) - : dict_struct{dict_struct_} - , uri{uri_} - , host{host_} - , port{port_} - , user{user_} - , password{password_} - , method{method_} - , db{db_} - , collection{collection_} - , options(options_) - , sample_block{sample_block_} - , connection{std::make_shared()} -{ - - StorageMongoDBPocoLegacySocketFactory socket_factory; - if (!uri.empty()) - { - // Connect with URI. - connection->connect(uri, socket_factory); - - Poco::URI poco_uri(connection->uri()); - - // Parse database from URI. This is required for correctness -- the - // cursor is created using database name and collection name, so we have - // to specify them properly. - db = poco_uri.getPath(); - // getPath() may return a leading slash, remove it. - if (!db.empty() && db[0] == '/') - { - db.erase(0, 1); - } - - // Parse some other parts from URI, for logging and display purposes. - host = poco_uri.getHost(); - port = poco_uri.getPort(); - user = poco_uri.getUserInfo(); - if (size_t separator = user.find(':'); separator != std::string::npos) - { - user.resize(separator); - } - } - else - { - // Connect with host/port/user/etc through constructing the uri - std::string uri_constructed("mongodb://" + host + ":" + std::to_string(port) + "/" + db + (options.empty() ? "" : "?" + options)); - connection->connect(uri_constructed, socket_factory); - - if (!user.empty()) - { - Poco::MongoDB::Database poco_db(db); - if (!poco_db.authenticate(*connection, user, password, method.empty() ? Poco::MongoDB::Database::AUTH_SCRAM_SHA1 : method)) - throw Exception(ErrorCodes::MONGODB_CANNOT_AUTHENTICATE, "Cannot authenticate in MongoDB, incorrect user or password"); - } - } -} - - -MongoDBPocoLegacyDictionarySource::MongoDBPocoLegacyDictionarySource(const MongoDBPocoLegacyDictionarySource & other) - : MongoDBPocoLegacyDictionarySource{ - other.dict_struct, other.uri, other.host, other.port, other.user, other.password, other.method, other.db, - other.collection, other.options, other.sample_block - } -{ -} - -MongoDBPocoLegacyDictionarySource::~MongoDBPocoLegacyDictionarySource() = default; - -QueryPipeline MongoDBPocoLegacyDictionarySource::loadAll() -{ - return QueryPipeline(std::make_shared(connection, db, collection, Poco::MongoDB::Document{}, sample_block, max_block_size)); -} - -QueryPipeline MongoDBPocoLegacyDictionarySource::loadIds(const std::vector & ids) -{ - if (!dict_struct.id) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); - - Poco::MongoDB::Document query; - - /** NOTE: While building array, Poco::MongoDB requires passing of different unused element names, along with values. - * In general, Poco::MongoDB is quite inefficient and bulky. - */ - - Poco::MongoDB::Array::Ptr ids_array(new Poco::MongoDB::Array); - for (const UInt64 id : ids) - ids_array->add(DB::toString(id), static_cast(id)); - - query.addNewDocument(dict_struct.id->name).add("$in", ids_array); - - return QueryPipeline(std::make_shared(connection, db, collection, query, sample_block, max_block_size)); -} - - -QueryPipeline MongoDBPocoLegacyDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) -{ - if (!dict_struct.key) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading"); - - Poco::MongoDB::Document query; - Poco::MongoDB::Array::Ptr keys_array(new Poco::MongoDB::Array); - - for (const auto row_idx : requested_rows) - { - auto & key = keys_array->addNewDocument(DB::toString(row_idx)); - - const auto & key_attributes = *dict_struct.key; - for (size_t attribute_index = 0; attribute_index < key_attributes.size(); ++attribute_index) - { - const auto & key_attribute = key_attributes[attribute_index]; - - switch (key_attribute.underlying_type) - { - case AttributeUnderlyingType::UInt8: - case AttributeUnderlyingType::UInt16: - case AttributeUnderlyingType::UInt32: - case AttributeUnderlyingType::UInt64: - case AttributeUnderlyingType::Int8: - case AttributeUnderlyingType::Int16: - case AttributeUnderlyingType::Int32: - case AttributeUnderlyingType::Int64: - { - key.add(key_attribute.name, static_cast(key_columns[attribute_index]->get64(row_idx))); - break; - } - case AttributeUnderlyingType::Float32: - case AttributeUnderlyingType::Float64: - { - key.add(key_attribute.name, key_columns[attribute_index]->getFloat64(row_idx)); - break; - } - case AttributeUnderlyingType::String: - { - String loaded_str((*key_columns[attribute_index])[row_idx].safeGet()); - /// Convert string to ObjectID - if (key_attribute.is_object_id) - { - Poco::MongoDB::ObjectId::Ptr loaded_id(new Poco::MongoDB::ObjectId(loaded_str)); - key.add(key_attribute.name, loaded_id); - } - else - { - key.add(key_attribute.name, loaded_str); - } - break; - } - default: - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported dictionary attribute type for MongoDB dictionary source"); - } - } - } - - /// If more than one key we should use $or - query.add("$or", keys_array); - - return QueryPipeline(std::make_shared(connection, db, collection, query, sample_block, max_block_size)); -} - -std::string MongoDBPocoLegacyDictionarySource::toString() const -{ - return fmt::format("MongoDB: {}.{},{}{}:{}", db, collection, (user.empty() ? " " : " " + user + '@'), host, port); -} - -} -#endif diff --git a/src/Dictionaries/MongoDBPocoLegacyDictionarySource.h b/src/Dictionaries/MongoDBPocoLegacyDictionarySource.h deleted file mode 100644 index 95dc1194981..00000000000 --- a/src/Dictionaries/MongoDBPocoLegacyDictionarySource.h +++ /dev/null @@ -1,93 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_MONGODB -#include -#include - -#include "DictionaryStructure.h" -#include "IDictionarySource.h" - -namespace Poco -{ -namespace Util -{ - class AbstractConfiguration; -} - -namespace MongoDB -{ - class Connection; -} -} - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -/// Allows loading dictionaries from a MongoDB collection. Deprecated, will be removed soon. -class MongoDBPocoLegacyDictionarySource final : public IDictionarySource -{ -public: - MongoDBPocoLegacyDictionarySource( - const DictionaryStructure & dict_struct_, - const std::string & uri_, - const std::string & host_, - UInt16 port_, - const std::string & user_, - const std::string & password_, - const std::string & method_, - const std::string & db_, - const std::string & collection_, - const std::string & options, - const Block & sample_block_); - - MongoDBPocoLegacyDictionarySource(const MongoDBPocoLegacyDictionarySource & other); - - ~MongoDBPocoLegacyDictionarySource() override; - - QueryPipeline loadAll() override; - - QueryPipeline loadUpdatedAll() override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for MongoDBDictionarySource"); - } - - bool supportsSelectiveLoad() const override { return true; } - - QueryPipeline loadIds(const std::vector & ids) override; - - QueryPipeline loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; - - /// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field - bool isModified() const override { return true; } - - ///Not yet supported - bool hasUpdateField() const override { return false; } - - DictionarySourcePtr clone() const override { return std::make_shared(*this); } - - std::string toString() const override; - -private: - const DictionaryStructure dict_struct; - const std::string uri; - std::string host; - UInt16 port; - std::string user; - const std::string password; - const std::string method; - std::string db; - const std::string collection; - const std::string options; - Block sample_block; - - std::shared_ptr connection; -}; - -} -#endif diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index fdecd540f13..67da6ca0c05 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -36,7 +36,7 @@ void registerDictionaryPolygon(DictionaryFactory & factory); void registerDictionaryDirect(DictionaryFactory & factory); -void registerDictionaries(bool use_legacy_mongodb_integration) +void registerDictionaries() { { auto & source_factory = DictionarySourceFactory::instance(); @@ -45,10 +45,7 @@ void registerDictionaries(bool use_legacy_mongodb_integration) registerDictionarySourceMysql(source_factory); registerDictionarySourceClickHouse(source_factory); - if (use_legacy_mongodb_integration) - registerDictionarySourceMongoDBPocoLegacy(source_factory); - else - registerDictionarySourceMongoDB(source_factory); + registerDictionarySourceMongoDB(source_factory); registerDictionarySourceRedis(source_factory); registerDictionarySourceCassandra(source_factory); diff --git a/src/Dictionaries/registerDictionaries.h b/src/Dictionaries/registerDictionaries.h index 4f82f7b5d29..e8480277c2c 100644 --- a/src/Dictionaries/registerDictionaries.h +++ b/src/Dictionaries/registerDictionaries.h @@ -2,5 +2,5 @@ namespace DB { -void registerDictionaries(bool use_legacy_mongodb_integration); +void registerDictionaries(); } diff --git a/src/Processors/Sources/MongoDBPocoLegacySource.cpp b/src/Processors/Sources/MongoDBPocoLegacySource.cpp deleted file mode 100644 index d39b857ee28..00000000000 --- a/src/Processors/Sources/MongoDBPocoLegacySource.cpp +++ /dev/null @@ -1,578 +0,0 @@ -#include "config.h" - -#if USE_MONGODB -#include "MongoDBPocoLegacySource.h" - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include "base/types.h" -#include -#include - -#include -#include - -// only after poco -// naming conflict: -// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); -// src/IO/WriteHelpers.h:146 #define writeCString(s, buf) -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TYPE_MISMATCH; - extern const int UNKNOWN_TYPE; - extern const int MONGODB_ERROR; - extern const int BAD_ARGUMENTS; -} - -namespace -{ - using ValueType = ExternalResultDescription::ValueType; - using ObjectId = Poco::MongoDB::ObjectId; - using MongoArray = Poco::MongoDB::Array; - using MongoUUID = Poco::MongoDB::Binary::Ptr; - - - UUID parsePocoUUID(const Poco::UUID & src) - { - UUID uuid; - - std::array src_node = src.getNode(); - UInt64 node = 0; - node |= UInt64(src_node[0]) << 40; - node |= UInt64(src_node[1]) << 32; - node |= UInt64(src_node[2]) << 24; - node |= UInt64(src_node[3]) << 16; - node |= UInt64(src_node[4]) << 8; - node |= src_node[5]; - - UUIDHelpers::getHighBytes(uuid) = UInt64(src.getTimeLow()) << 32 | UInt32(src.getTimeMid() << 16 | src.getTimeHiAndVersion()); - UUIDHelpers::getLowBytes(uuid) = UInt64(src.getClockSeq()) << 48 | node; - - return uuid; - } - - template - Field getNumber(const Poco::MongoDB::Element & value, const std::string & name) - { - switch (value.type()) - { - case Poco::MongoDB::ElementTraits::TypeId: - return static_cast(static_cast &>(value).value()); - case Poco::MongoDB::ElementTraits::TypeId: - return static_cast(static_cast &>(value).value()); - case Poco::MongoDB::ElementTraits::TypeId: - return static_cast(static_cast &>(value).value()); - case Poco::MongoDB::ElementTraits::TypeId: - return static_cast(static_cast &>(value).value()); - case Poco::MongoDB::ElementTraits::TypeId: - return Field(); - case Poco::MongoDB::ElementTraits::TypeId: - return parse(static_cast &>(value).value()); - default: - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected a number, got type id = {} for column {}", - toString(value.type()), name); - } - } - - void prepareMongoDBArrayInfo( - std::unordered_map & array_info, size_t column_idx, const DataTypePtr data_type) - { - const auto * array_type = assert_cast(data_type.get()); - auto nested = array_type->getNestedType(); - - size_t count_dimensions = 1; - while (isArray(nested)) - { - ++count_dimensions; - nested = assert_cast(nested.get())->getNestedType(); - } - - Field default_value = nested->getDefault(); - if (nested->isNullable()) - nested = assert_cast(nested.get())->getNestedType(); - - WhichDataType which(nested); - std::function parser; - - if (which.isUInt8()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isUInt16()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isUInt32()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isUInt64()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isInt8()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isInt16()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isInt32()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isInt64()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isFloat32()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isFloat64()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field { return getNumber(value, name); }; - else if (which.isString() || which.isFixedString()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field - { - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - String string_id = value.toString(); - return Field(string_id.data(), string_id.size()); - } - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - String string = static_cast &>(value).value(); - return Field(string.data(), string.size()); - } - - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String, got type id = {} for column {}", - toString(value.type()), name); - }; - else if (which.isDate()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field - { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected Timestamp, got type id = {} for column {}", - toString(value.type()), name); - - return static_cast(DateLUT::instance().toDayNum( - static_cast &>(value).value().epochTime())); - }; - else if (which.isDateTime()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field - { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected Timestamp, got type id = {} for column {}", - toString(value.type()), name); - - return static_cast(static_cast &>(value).value().epochTime()); - }; - else if (which.isUUID()) - parser = [](const Poco::MongoDB::Element & value, const std::string & name) -> Field - { - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - String string = static_cast &>(value).value(); - return parse(string); - } - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); - return parsePocoUUID(poco_uuid); - } - - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", - toString(value.type()), name); - }; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); - - array_info[column_idx] = {count_dimensions, default_value, parser}; - } - - template - void insertNumber(IColumn & column, const Poco::MongoDB::Element & value, const std::string & name) - { - switch (value.type()) - { - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().push_back( - static_cast &>(value).value()); - break; - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().push_back( - static_cast(static_cast &>(value).value())); - break; - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().push_back(static_cast( - static_cast &>(value).value())); - break; - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().push_back( - static_cast &>(value).value()); - break; - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().emplace_back(); - break; - case Poco::MongoDB::ElementTraits::TypeId: - assert_cast &>(column).getData().push_back( - parse(static_cast &>(value).value())); - break; - default: - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected a number, got type id = {} for column {}", - toString(value.type()), name); - } - } - - void insertValue( - IColumn & column, - const ValueType type, - const Poco::MongoDB::Element & value, - const std::string & name, - std::unordered_map & array_info, - size_t idx) - { - switch (type) - { - case ValueType::vtUInt8: - insertNumber(column, value, name); - break; - case ValueType::vtUInt16: - insertNumber(column, value, name); - break; - case ValueType::vtUInt32: - insertNumber(column, value, name); - break; - case ValueType::vtUInt64: - insertNumber(column, value, name); - break; - case ValueType::vtInt8: - insertNumber(column, value, name); - break; - case ValueType::vtInt16: - insertNumber(column, value, name); - break; - case ValueType::vtInt32: - insertNumber(column, value, name); - break; - case ValueType::vtInt64: - insertNumber(column, value, name); - break; - case ValueType::vtFloat32: - insertNumber(column, value, name); - break; - case ValueType::vtFloat64: - insertNumber(column, value, name); - break; - - case ValueType::vtEnum8: - case ValueType::vtEnum16: - case ValueType::vtString: - { - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - std::string string_id = value.toString(); - assert_cast(column).insertData(string_id.data(), string_id.size()); - break; - } - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - String string = static_cast &>(value).value(); - assert_cast(column).insertData(string.data(), string.size()); - break; - } - - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String, got type id = {} for column {}", - toString(value.type()), name); - } - - case ValueType::vtDate: - { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected Timestamp, got type id = {} for column {}", - toString(value.type()), name); - - assert_cast(column).getData().push_back(static_cast(DateLUT::instance().toDayNum( - static_cast &>(value).value().epochTime()))); - break; - } - - case ValueType::vtDateTime: - { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected Timestamp, got type id = {} for column {}", - toString(value.type()), name); - - assert_cast(column).getData().push_back( - static_cast(static_cast &>(value).value().epochTime())); - break; - } - case ValueType::vtUUID: - { - if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - String string = static_cast &>(value).value(); - assert_cast(column).getData().push_back(parse(string)); - } - else if (value.type() == Poco::MongoDB::ElementTraits::TypeId) - { - const Poco::UUID & poco_uuid = static_cast &>(value).value()->uuid(); - UUID uuid = parsePocoUUID(poco_uuid); - assert_cast(column).getData().push_back(uuid); - } - else - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected String/UUID, got type id = {} for column {}", - toString(value.type()), name); - break; - } - case ValueType::vtArray: - { - if (value.type() != Poco::MongoDB::ElementTraits::TypeId) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch, expected Array, got type id = {} for column {}", - toString(value.type()), name); - - size_t expected_dimensions = array_info[idx].num_dimensions; - const auto parse_value = array_info[idx].parser; - std::vector dimensions(expected_dimensions + 1); - - auto array = static_cast &>(value).value(); - - std::vector> arrays; - arrays.emplace_back(&value, 0); - - while (!arrays.empty()) - { - size_t dimension_idx = arrays.size() - 1; - - if (dimension_idx + 1 > expected_dimensions) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got more dimensions than expected"); - - auto [parent_ptr, child_idx] = arrays.back(); - auto parent = static_cast &>(*parent_ptr).value(); - - if (child_idx >= parent->size()) - { - arrays.pop_back(); - - if (dimension_idx == 0) - break; - - dimensions[dimension_idx].emplace_back(Array(dimensions[dimension_idx + 1].begin(), dimensions[dimension_idx + 1].end())); - dimensions[dimension_idx + 1].clear(); - - continue; - } - - Poco::MongoDB::Element::Ptr child = parent->get(static_cast(child_idx)); - arrays.back().second += 1; - - if (child->type() == Poco::MongoDB::ElementTraits::TypeId) - { - arrays.emplace_back(child.get(), 0); - } - else if (child->type() == Poco::MongoDB::ElementTraits::TypeId) - { - if (dimension_idx + 1 == expected_dimensions) - dimensions[dimension_idx + 1].emplace_back(array_info[idx].default_value); - else - dimensions[dimension_idx + 1].emplace_back(Array()); - } - else if (dimension_idx + 1 == expected_dimensions) - { - dimensions[dimension_idx + 1].emplace_back(parse_value(*child, name)); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Got less dimensions than expected. ({} instead of {})", dimension_idx + 1, expected_dimensions); - } - } - - assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); - break; - - } - default: - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Value of unsupported type: {}", column.getName()); - } - } - - void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } -} - - -bool isMongoDBWireProtocolOld(Poco::MongoDB::Connection & connection_, const std::string & database_name_) -{ - Poco::MongoDB::Database db(database_name_); - Poco::MongoDB::Document::Ptr doc = db.queryServerHello(connection_, false); - - if (doc->exists("maxWireVersion")) - { - auto wire_version = doc->getInteger("maxWireVersion"); - return wire_version < Poco::MongoDB::Database::WireVersion::VER_36; - } - - doc = db.queryServerHello(connection_, true); - if (doc->exists("maxWireVersion")) - { - auto wire_version = doc->getInteger("maxWireVersion"); - return wire_version < Poco::MongoDB::Database::WireVersion::VER_36; - } - - return true; -} - - -MongoDBPocoLegacyCursor::MongoDBPocoLegacyCursor( - const std::string & database, - const std::string & collection, - const Block & sample_block_to_select, - const Poco::MongoDB::Document & query, - Poco::MongoDB::Connection & connection) - : is_wire_protocol_old(isMongoDBWireProtocolOld(connection, database)) -{ - Poco::MongoDB::Document projection; - - /// Looks like selecting _id column is implicit by default. - if (!sample_block_to_select.has("_id")) - projection.add("_id", 0); - - for (const auto & column : sample_block_to_select) - projection.add(column.name, 1); - - if (is_wire_protocol_old) - { - old_cursor = std::make_unique(database, collection); - old_cursor->query().selector() = query; - old_cursor->query().returnFieldSelector() = projection; - } - else - { - new_cursor = std::make_unique(database, collection); - new_cursor->query().setCommandName(Poco::MongoDB::OpMsgMessage::CMD_FIND); - new_cursor->query().body().addNewDocument("filter") = query; - new_cursor->query().body().addNewDocument("projection") = projection; - } -} - -Poco::MongoDB::Document::Vector MongoDBPocoLegacyCursor::nextDocuments(Poco::MongoDB::Connection & connection) -{ - if (is_wire_protocol_old) - { - auto response = old_cursor->next(connection); - cursor_id = response.cursorID(); - return std::move(response.documents()); - } - - auto response = new_cursor->next(connection); - cursor_id = new_cursor->cursorID(); - return std::move(response.documents()); -} - -Int64 MongoDBPocoLegacyCursor::cursorID() const -{ - return cursor_id; -} - - -MongoDBPocoLegacySource::MongoDBPocoLegacySource( - std::shared_ptr & connection_, - const String & database_name_, - const String & collection_name_, - const Poco::MongoDB::Document & query_, - const Block & sample_block, - UInt64 max_block_size_) - : ISource(sample_block.cloneEmpty()) - , connection(connection_) - , cursor(database_name_, collection_name_, sample_block, query_, *connection_) - , max_block_size{max_block_size_} -{ - description.init(sample_block); - - for (const auto idx : collections::range(0, description.sample_block.columns())) - if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray) - prepareMongoDBArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type); -} - - -MongoDBPocoLegacySource::~MongoDBPocoLegacySource() = default; - -Chunk MongoDBPocoLegacySource::generate() -{ - if (all_read) - return {}; - - MutableColumns columns(description.sample_block.columns()); - const size_t size = columns.size(); - - for (const auto i : collections::range(0, size)) - columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty(); - - size_t num_rows = 0; - while (num_rows < max_block_size) - { - auto documents = cursor.nextDocuments(*connection); - - for (auto & document : documents) - { - if (document->exists("ok") && document->exists("$err") - && document->exists("code") && document->getInteger("ok") == 0) - { - auto code = document->getInteger("code"); - const Poco::MongoDB::Element::Ptr value = document->get("$err"); - auto message = static_cast &>(*value).value(); - throw Exception(ErrorCodes::MONGODB_ERROR, "Got error from MongoDB: {}, code: {}", message, code); - } - ++num_rows; - - for (const auto idx : collections::range(0, size)) - { - const auto & name = description.sample_block.getByPosition(idx).name; - - bool exists_in_current_document = document->exists(name); - if (!exists_in_current_document) - { - insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); - continue; - } - - const Poco::MongoDB::Element::Ptr value = document->get(name); - - if (value.isNull() || value->type() == Poco::MongoDB::ElementTraits::TypeId) - { - insertDefaultValue(*columns[idx], *description.sample_block.getByPosition(idx).column); - } - else - { - bool is_nullable = description.types[idx].second; - if (is_nullable) - { - ColumnNullable & column_nullable = assert_cast(*columns[idx]); - insertValue(column_nullable.getNestedColumn(), description.types[idx].first, *value, name, array_info, idx); - column_nullable.getNullMapData().emplace_back(0); - } - else - insertValue(*columns[idx], description.types[idx].first, *value, name, array_info, idx); - } - } - } - - if (cursor.cursorID() == 0) - { - all_read = true; - break; - } - } - - if (num_rows == 0) - return {}; - - return Chunk(std::move(columns), num_rows); -} - -} -#endif diff --git a/src/Processors/Sources/MongoDBPocoLegacySource.h b/src/Processors/Sources/MongoDBPocoLegacySource.h deleted file mode 100644 index 0c9f2c7cc9f..00000000000 --- a/src/Processors/Sources/MongoDBPocoLegacySource.h +++ /dev/null @@ -1,92 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_MONGODB -#include -#include - -#include -#include -#include - -#include - - -namespace Poco -{ -namespace MongoDB -{ - class Connection; - class Document; - class Cursor; - class OpMsgCursor; -} -} - -namespace DB -{ - -struct MongoDBPocoLegacyArrayInfo -{ - size_t num_dimensions; - Field default_value; - std::function parser; -}; - -void authenticate(Poco::MongoDB::Connection & connection, const std::string & database, const std::string & user, const std::string & password); - -bool isMongoDBWireProtocolOld(Poco::MongoDB::Connection & connection_, const std::string & database_name_); - -/// Deprecated, will be removed soon. -class MongoDBPocoLegacyCursor -{ -public: - MongoDBPocoLegacyCursor( - const std::string & database, - const std::string & collection, - const Block & sample_block_to_select, - const Poco::MongoDB::Document & query, - Poco::MongoDB::Connection & connection); - - Poco::MongoDB::Document::Vector nextDocuments(Poco::MongoDB::Connection & connection); - - Int64 cursorID() const; - -private: - const bool is_wire_protocol_old; - std::unique_ptr old_cursor; - std::unique_ptr new_cursor; - Int64 cursor_id = 0; -}; - -/// Converts MongoDB Cursor to a stream of Blocks. Deprecated, will be removed soon. -class MongoDBPocoLegacySource final : public ISource -{ -public: - MongoDBPocoLegacySource( - std::shared_ptr & connection_, - const String & database_name_, - const String & collection_name_, - const Poco::MongoDB::Document & query_, - const Block & sample_block, - UInt64 max_block_size_); - - ~MongoDBPocoLegacySource() override; - - String getName() const override { return "MongoDB"; } - -private: - Chunk generate() override; - - std::shared_ptr connection; - MongoDBPocoLegacyCursor cursor; - const UInt64 max_block_size; - ExternalResultDescription description; - bool all_read = false; - - std::unordered_map array_info; -}; - -} -#endif diff --git a/src/Storages/StorageMongoDBPocoLegacy.cpp b/src/Storages/StorageMongoDBPocoLegacy.cpp deleted file mode 100644 index 04f73cb0510..00000000000 --- a/src/Storages/StorageMongoDBPocoLegacy.cpp +++ /dev/null @@ -1,327 +0,0 @@ -#include "config.h" - -#if USE_MONGODB -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int MONGODB_CANNOT_AUTHENTICATE; -} - -StorageMongoDBPocoLegacy::StorageMongoDBPocoLegacy( - const StorageID & table_id_, - const std::string & host_, - uint16_t port_, - const std::string & database_name_, - const std::string & collection_name_, - const std::string & username_, - const std::string & password_, - const std::string & options_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment) - : IStorage(table_id_) - , database_name(database_name_) - , collection_name(collection_name_) - , username(username_) - , password(password_) - , uri("mongodb://" + host_ + ":" + std::to_string(port_) + "/" + database_name_ + "?" + options_) -{ - LOG_WARNING(getLogger("StorageMongoDB (" + table_id_.table_name + ")"), "The deprecated MongoDB integartion implementation is used, this will be removed in next releases."); - - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); -} - - -void StorageMongoDBPocoLegacy::connectIfNotConnected() -{ - std::lock_guard lock{connection_mutex}; - if (!connection) - { - StorageMongoDBPocoLegacySocketFactory factory; - connection = std::make_shared(uri, factory); - } - - if (!authenticated) - { - Poco::URI poco_uri(uri); - auto query_params = poco_uri.getQueryParameters(); - auto auth_source = std::find_if(query_params.begin(), query_params.end(), - [&](const std::pair & param) { return param.first == "authSource"; }); - auto auth_db = database_name; - if (auth_source != query_params.end()) - auth_db = auth_source->second; - - if (!username.empty() && !password.empty()) - { - Poco::MongoDB::Database poco_db(auth_db); - if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1)) - throw Exception(ErrorCodes::MONGODB_CANNOT_AUTHENTICATE, "Cannot authenticate in MongoDB, incorrect user or password"); - } - - authenticated = true; - } -} - -class StorageMongoDBLegacySink : public SinkToStorage -{ -public: - explicit StorageMongoDBLegacySink( - const std::string & collection_name_, - const std::string & db_name_, - const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr connection_) - : SinkToStorage(metadata_snapshot_->getSampleBlock()) - , collection_name(collection_name_) - , db_name(db_name_) - , metadata_snapshot{metadata_snapshot_} - , connection(connection_) - , is_wire_protocol_old(isMongoDBWireProtocolOld(*connection_, db_name)) - { - } - - String getName() const override { return "StorageMongoDBLegacySink"; } - - void consume(Chunk & chunk) override - { - Poco::MongoDB::Database db(db_name); - Poco::MongoDB::Document::Vector documents; - - auto block = getHeader().cloneWithColumns(chunk.getColumns()); - - size_t num_rows = block.rows(); - size_t num_cols = block.columns(); - - const auto columns = block.getColumns(); - const auto data_types = block.getDataTypes(); - const auto data_names = block.getNames(); - - documents.reserve(num_rows); - - for (const auto i : collections::range(0, num_rows)) - { - Poco::MongoDB::Document::Ptr document = new Poco::MongoDB::Document(); - - for (const auto j : collections::range(0, num_cols)) - { - insertValueIntoMongoDB(*document, data_names[j], *data_types[j], *columns[j], i); - } - - documents.push_back(std::move(document)); - } - - if (is_wire_protocol_old) - { - Poco::SharedPtr insert_request = db.createInsertRequest(collection_name); - insert_request->documents() = std::move(documents); - connection->sendRequest(*insert_request); - } - else - { - Poco::SharedPtr insert_request = db.createOpMsgMessage(collection_name); - insert_request->setCommandName(Poco::MongoDB::OpMsgMessage::CMD_INSERT); - insert_request->documents() = std::move(documents); - connection->sendRequest(*insert_request); - } - } - -private: - - void insertValueIntoMongoDB( - Poco::MongoDB::Document & document, - const std::string & name, - const IDataType & data_type, - const IColumn & column, - size_t idx) - { - WhichDataType which(data_type); - - if (which.isArray()) - { - const ColumnArray & column_array = assert_cast(column); - const ColumnArray::Offsets & offsets = column_array.getOffsets(); - - size_t offset = offsets[idx - 1]; - size_t next_offset = offsets[idx]; - - const IColumn & nested_column = column_array.getData(); - - const auto * array_type = assert_cast(&data_type); - const DataTypePtr & nested_type = array_type->getNestedType(); - - Poco::MongoDB::Array::Ptr array = new Poco::MongoDB::Array(); - for (size_t i = 0; i + offset < next_offset; ++i) - { - insertValueIntoMongoDB(*array, Poco::NumberFormatter::format(i), *nested_type, nested_column, i + offset); - } - - document.add(name, array); - return; - } - - /// MongoDB does not support UInt64 type, so just cast it to Int64 - if (which.isNativeUInt()) - document.add(name, static_cast(column.getUInt(idx))); - else if (which.isNativeInt()) - document.add(name, static_cast(column.getInt(idx))); - else if (which.isFloat32()) - document.add(name, static_cast(column.getFloat32(idx))); - else if (which.isFloat64()) - document.add(name, column.getFloat64(idx)); - else if (which.isDate()) - document.add(name, Poco::Timestamp(DateLUT::instance().fromDayNum(DayNum(column.getUInt(idx))) * 1000000)); - else if (which.isDateTime()) - document.add(name, Poco::Timestamp(column.getUInt(idx) * 1000000)); - else - { - WriteBufferFromOwnString ostr; - data_type.getDefaultSerialization()->serializeText(column, idx, ostr, FormatSettings{}); - document.add(name, ostr.str()); - } - } - - String collection_name; - String db_name; - StorageMetadataPtr metadata_snapshot; - std::shared_ptr connection; - - const bool is_wire_protocol_old; -}; - -Pipe StorageMongoDBPocoLegacy::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /*query_info*/, - ContextPtr /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t /*num_streams*/) -{ - connectIfNotConnected(); - - storage_snapshot->check(column_names); - - Block sample_block; - for (const String & column_name : column_names) - { - auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); - sample_block.insert({ column_data.type, column_data.name }); - } - - return Pipe(std::make_shared(connection, database_name, collection_name, Poco::MongoDB::Document{}, sample_block, max_block_size)); -} - - -SinkToStoragePtr StorageMongoDBPocoLegacy::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */, bool /*async_insert*/) -{ - connectIfNotConnected(); - return std::make_shared(collection_name, database_name, metadata_snapshot, connection); -} - -StorageMongoDBPocoLegacy::Configuration StorageMongoDBPocoLegacy::getConfiguration(ASTs engine_args, ContextPtr context) -{ - Configuration configuration; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context)) - { - validateNamedCollection( - *named_collection, - ValidateKeysMultiset{"host", "port", "user", "username", "password", "database", "db", "collection", "table"}, - {"options"}); - - configuration.host = named_collection->getAny({"host", "hostname"}); - configuration.port = static_cast(named_collection->get("port")); - configuration.username = named_collection->getAny({"user", "username"}); - configuration.password = named_collection->get("password"); - configuration.database = named_collection->getAny({"database", "db"}); - configuration.table = named_collection->getAny({"collection", "table"}); - configuration.options = named_collection->getOrDefault("options", ""); - } - else - { - if (engine_args.size() < 5 || engine_args.size() > 6) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage MongoDB requires from 5 to 6 parameters: " - "MongoDB('host:port', database, collection, 'user', 'password' [, 'options'])."); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); - - /// 27017 is the default MongoDB port. - auto parsed_host_port = parseAddress(checkAndGetLiteralArgument(engine_args[0], "host:port"), 27017); - - configuration.host = parsed_host_port.first; - configuration.port = parsed_host_port.second; - configuration.database = checkAndGetLiteralArgument(engine_args[1], "database"); - configuration.table = checkAndGetLiteralArgument(engine_args[2], "table"); - configuration.username = checkAndGetLiteralArgument(engine_args[3], "username"); - configuration.password = checkAndGetLiteralArgument(engine_args[4], "password"); - - if (engine_args.size() >= 6) - configuration.options = checkAndGetLiteralArgument(engine_args[5], "database"); - } - - context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); - - return configuration; -} - - -void registerStorageMongoDBPocoLegacy(StorageFactory & factory) -{ - factory.registerStorage("MongoDB", [](const StorageFactory::Arguments & args) - { - auto configuration = StorageMongoDBPocoLegacy::getConfiguration(args.engine_args, args.getLocalContext()); - - return std::make_shared( - args.table_id, - configuration.host, - configuration.port, - configuration.database, - configuration.table, - configuration.username, - configuration.password, - configuration.options, - args.columns, - args.constraints, - args.comment); - }, - { - .source_access_type = AccessType::MONGO, - }); -} - -} -#endif diff --git a/src/Storages/StorageMongoDBPocoLegacy.h b/src/Storages/StorageMongoDBPocoLegacy.h deleted file mode 100644 index a5814ccd5dd..00000000000 --- a/src/Storages/StorageMongoDBPocoLegacy.h +++ /dev/null @@ -1,79 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_MONGODB -#include - -#include - -namespace DB -{ -/* Implements storage in the MongoDB database. - * Use ENGINE = MongoDB(host:port, database, collection, user, password [, options]); - * Read only. - */ - -/// Deprecated, will be removed soon. -class StorageMongoDBPocoLegacy final : public IStorage -{ -public: - StorageMongoDBPocoLegacy( - const StorageID & table_id_, - const std::string & host_, - uint16_t port_, - const std::string & database_name_, - const std::string & collection_name_, - const std::string & username_, - const std::string & password_, - const std::string & options_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment); - - std::string getName() const override { return "MongoDB"; } - - Pipe read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - size_t num_streams) override; - - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context, - bool async_insert) override; - - struct Configuration - { - std::string host; - UInt16 port; - std::string username; - std::string password; - std::string database; - std::string table; - std::string options; - }; - - static Configuration getConfiguration(ASTs engine_args, ContextPtr context); - -private: - void connectIfNotConnected(); - - const std::string database_name; - const std::string collection_name; - const std::string username; - const std::string password; - const std::string uri; - - std::shared_ptr connection; - bool authenticated = false; - std::mutex connection_mutex; /// Protects the variables `connection` and `authenticated`. -}; - -} -#endif diff --git a/src/Storages/StorageMongoDBPocoLegacySocketFactory.cpp b/src/Storages/StorageMongoDBPocoLegacySocketFactory.cpp deleted file mode 100644 index bcfe995dcc6..00000000000 --- a/src/Storages/StorageMongoDBPocoLegacySocketFactory.cpp +++ /dev/null @@ -1,57 +0,0 @@ -#include "config.h" - -#if USE_MONGODB -#include "StorageMongoDBPocoLegacySocketFactory.h" - -#include - -#include -#include - -#if USE_SSL -# include -#endif - - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME; -} - -Poco::Net::StreamSocket StorageMongoDBPocoLegacySocketFactory::createSocket(const std::string & host, int port, Poco::Timespan connectTimeout, bool secure) -{ - return secure ? createSecureSocket(host, port, connectTimeout) : createPlainSocket(host, port, connectTimeout); -} - -Poco::Net::StreamSocket StorageMongoDBPocoLegacySocketFactory::createPlainSocket(const std::string & host, int port, Poco::Timespan connectTimeout) -{ - Poco::Net::SocketAddress address(host, port); - Poco::Net::StreamSocket socket; - - socket.connect(address, connectTimeout); - - return socket; -} - - -Poco::Net::StreamSocket StorageMongoDBPocoLegacySocketFactory::createSecureSocket(const std::string & host [[maybe_unused]], int port [[maybe_unused]], Poco::Timespan connectTimeout [[maybe_unused]]) -{ -#if USE_SSL - Poco::Net::SocketAddress address(host, port); - Poco::Net::SecureStreamSocket socket; - - socket.setPeerHostName(host); - - socket.connect(address, connectTimeout); - - return socket; -#else - throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "SSL is not enabled at build time."); -#endif -} - -} -#endif diff --git a/src/Storages/StorageMongoDBPocoLegacySocketFactory.h b/src/Storages/StorageMongoDBPocoLegacySocketFactory.h deleted file mode 100644 index ee6ee8faa29..00000000000 --- a/src/Storages/StorageMongoDBPocoLegacySocketFactory.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_MONGODB -#include - - -namespace DB -{ - -/// Deprecated, will be removed soon. -class StorageMongoDBPocoLegacySocketFactory : public Poco::MongoDB::Connection::SocketFactory -{ -public: - Poco::Net::StreamSocket createSocket(const std::string & host, int port, Poco::Timespan connectTimeout, bool secure) override; - -private: - static Poco::Net::StreamSocket createPlainSocket(const std::string & host, int port, Poco::Timespan connectTimeout); - static Poco::Net::StreamSocket createSecureSocket(const std::string & host, int port, Poco::Timespan connectTimeout); -}; - -} -#endif diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 458b151a400..d95464c206f 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -106,7 +106,7 @@ void registerStorageKeeperMap(StorageFactory & factory); void registerStorageObjectStorage(StorageFactory & factory); -void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]]) +void registerStorages() { auto & factory = StorageFactory::instance(); @@ -169,10 +169,7 @@ void registerStorages(bool use_legacy_mongodb_integration [[maybe_unused]]) #endif #if USE_MONGODB - if (use_legacy_mongodb_integration) - registerStorageMongoDBPocoLegacy(factory); - else - registerStorageMongoDB(factory); + registerStorageMongoDB(factory); #endif registerStorageRedis(factory); diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index 330855a49d0..d44b934ff9f 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -2,5 +2,5 @@ namespace DB { -void registerStorages(bool use_legacy_mongodb_integration); +void registerStorages(); } diff --git a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp b/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp deleted file mode 100644 index 4e27fd35e12..00000000000 --- a/src/TableFunctions/TableFunctionMongoDBPocoLegacy.cpp +++ /dev/null @@ -1,128 +0,0 @@ -#include "config.h" - -#if USE_MONGODB -#include - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -namespace -{ - -/// Deprecated, will be removed soon. -class TableFunctionMongoDBPocoLegacy : public ITableFunction -{ -public: - static constexpr auto name = "mongodb"; - - std::string getName() const override { return name; } - -private: - StoragePtr executeImpl( - const ASTPtr & ast_function, ContextPtr context, - const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "MongoDB"; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - std::optional configuration; - String structure; -}; - -StoragePtr TableFunctionMongoDBPocoLegacy::executeImpl(const ASTPtr & /*ast_function*/, - ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const -{ - auto columns = getActualTableStructure(context, is_insert_query); - auto storage = std::make_shared( - StorageID(configuration->database, table_name), - configuration->host, - configuration->port, - configuration->database, - configuration->table, - configuration->username, - configuration->password, - configuration->options, - columns, - ConstraintsDescription(), - String{}); - storage->startup(); - return storage; -} - -ColumnsDescription TableFunctionMongoDBPocoLegacy::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const -{ - return parseColumnsListFromString(structure, context); -} - -void TableFunctionMongoDBPocoLegacy::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - const auto & func_args = ast_function->as(); - if (!func_args.arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function 'mongodb' must have arguments."); - - ASTs & args = func_args.arguments->children; - - if (args.size() < 6 || args.size() > 7) - { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Table function 'mongodb' requires from 6 to 7 parameters: " - "mongodb('host:port', database, collection, 'user', 'password', structure, [, 'options'])"); - } - - ASTs main_arguments(args.begin(), args.begin() + 5); - - for (size_t i = 5; i < args.size(); ++i) - { - if (const auto * ast_func = typeid_cast(args[i].get())) - { - const auto & [arg_name, arg_value] = getKeyValueMongoDBArgument(ast_func); - if (arg_name == "structure") - structure = checkAndGetLiteralArgument(arg_value, "structure"); - else if (arg_name == "options") - main_arguments.push_back(arg_value); - } - else if (i == 5) - { - structure = checkAndGetLiteralArgument(args[i], "structure"); - } - else if (i == 6) - { - main_arguments.push_back(args[i]); - } - } - - configuration = StorageMongoDBPocoLegacy::getConfiguration(main_arguments, context); -} - -} - -void registerTableFunctionMongoDBPocoLegacy(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} -#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index fbe2c7c59ed..156a0fd2c92 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -3,7 +3,7 @@ namespace DB { -void registerTableFunctions(bool use_legacy_mongodb_integration [[maybe_unused]]) +void registerTableFunctions() { auto & factory = TableFunctionFactory::instance(); @@ -23,10 +23,7 @@ void registerTableFunctions(bool use_legacy_mongodb_integration [[maybe_unused]] registerTableFunctionInput(factory); registerTableFunctionGenerate(factory); #if USE_MONGODB - if (use_legacy_mongodb_integration) - registerTableFunctionMongoDBPocoLegacy(factory); - else - registerTableFunctionMongoDB(factory); + registerTableFunctionMongoDB(factory); #endif registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index e22ba7346fa..64d1584c5db 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -73,6 +73,6 @@ void registerDataLakeTableFunctions(TableFunctionFactory & factory); void registerTableFunctionTimeSeries(TableFunctionFactory & factory); -void registerTableFunctions(bool use_legacy_mongodb_integration [[maybe_unused]]); +void registerTableFunctions(); } diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 1b2c73142f3..6c1fb4d0d13 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -170,7 +170,6 @@ class SourceMongo(ExternalSource): user, password, secure=False, - legacy=False, ): ExternalSource.__init__( self, @@ -183,13 +182,10 @@ class SourceMongo(ExternalSource): password, ) self.secure = secure - self.legacy = legacy def get_source_str(self, table_name): options = "" - if self.secure and self.legacy: - options = "ssl=true" - if self.secure and not self.legacy: + if self.secure: options = "tls=true&tlsAllowInvalidCertificates=true" return """ @@ -267,9 +263,7 @@ class SourceMongoURI(SourceMongo): def get_source_str(self, table_name): options = "" - if self.secure and self.legacy: - options = "ssl=true" - if self.secure and not self.legacy: + if self.secure: options = "tls=true&tlsAllowInvalidCertificates=true" return """ diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/legacy.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/legacy.xml deleted file mode 100644 index 4ee05db9d1e..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/legacy.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/new.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/new.xml deleted file mode 100644 index eb2d328060a..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/mongo/new.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 0 - diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index d188705cf6c..2cf6250b106 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -19,20 +19,14 @@ def secure_connection(request): @pytest.fixture(scope="module") -def legacy(request): - return request.param - - -@pytest.fixture(scope="module") -def cluster(secure_connection, legacy): +def cluster(secure_connection): cluster_name = __file__.removeprefix("test_").removesuffix(".py") - cluster_name += "_legacy" if legacy else "_new" cluster_name += "_secure" if secure_connection else "_insecure" return ClickHouseCluster(cluster_name) @pytest.fixture(scope="module") -def source(secure_connection, legacy, cluster): +def source(secure_connection, cluster): return SourceMongo( "MongoDB", "localhost", @@ -42,7 +36,6 @@ def source(secure_connection, legacy, cluster): "root", "clickhouse", secure=secure_connection, - legacy=legacy, ) @@ -69,18 +62,10 @@ def ranged_tester(source): @pytest.fixture(scope="module") -def main_config(secure_connection, legacy): - if legacy: - main_config = [os.path.join("configs", "mongo", "legacy.xml")] - else: - main_config = [os.path.join("configs", "mongo", "new.xml")] - +def main_config(secure_connection): if secure_connection: - main_config.append(os.path.join("configs", "disable_ssl_verification.xml")) - else: - main_config.append(os.path.join("configs", "ssl_verification.xml")) - - return main_config + return [os.path.join("configs", "disable_ssl_verification.xml")] + return [os.path.join("configs", "ssl_verification.xml")] @pytest.fixture(scope="module") @@ -113,32 +98,28 @@ def started_cluster( @pytest.mark.parametrize("secure_connection", [False], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) -def test_simple(secure_connection, legacy, started_cluster, layout_name, simple_tester): +def test_simple(secure_connection, started_cluster, layout_name, simple_tester): simple_tester.execute(layout_name, started_cluster.instances["node1"]) @pytest.mark.parametrize("secure_connection", [False], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX)) def test_complex( - secure_connection, legacy, started_cluster, layout_name, complex_tester + secure_connection, started_cluster, layout_name, complex_tester ): complex_tester.execute(layout_name, started_cluster.instances["node1"]) @pytest.mark.parametrize("secure_connection", [False], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_RANGED)) -def test_ranged(secure_connection, legacy, started_cluster, layout_name, ranged_tester): +def test_ranged(secure_connection, started_cluster, layout_name, ranged_tester): ranged_tester.execute(layout_name, started_cluster.instances["node1"]) @pytest.mark.parametrize("secure_connection", [True], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) def test_simple_ssl( - secure_connection, legacy, started_cluster, layout_name, simple_tester + secure_connection, started_cluster, layout_name, simple_tester ): simple_tester.execute(layout_name, started_cluster.instances["node1"]) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index bdaebab1149..407031ed7d3 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -17,18 +17,13 @@ def secure_connection(request): return request.param -@pytest.fixture(scope="module") -def legacy(request): - return request.param - - @pytest.fixture(scope="module") def cluster(secure_connection): return ClickHouseCluster(__file__) @pytest.fixture(scope="module") -def source(secure_connection, legacy, cluster): +def source(secure_connection, cluster): return SourceMongoURI( "MongoDB", "localhost", @@ -38,7 +33,6 @@ def source(secure_connection, legacy, cluster): "root", "clickhouse", secure=secure_connection, - legacy=legacy, ) @@ -51,22 +45,14 @@ def simple_tester(source): @pytest.fixture(scope="module") -def main_config(secure_connection, legacy): - if legacy: - main_config = [os.path.join("configs", "mongo", "legacy.xml")] - else: - main_config = [os.path.join("configs", "mongo", "new.xml")] - +def main_config(secure_connection): if secure_connection: - main_config.append(os.path.join("configs", "disable_ssl_verification.xml")) - else: - main_config.append(os.path.join("configs", "ssl_verification.xml")) - - return main_config + return [os.path.join("configs", "disable_ssl_verification.xml")] + return [os.path.join("configs", "ssl_verification.xml")] @pytest.fixture(scope="module") -def started_cluster(secure_connection, legacy, cluster, main_config, simple_tester): +def started_cluster(secure_connection, cluster, main_config, simple_tester): dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance( @@ -85,16 +71,14 @@ def started_cluster(secure_connection, legacy, cluster, main_config, simple_test # See comment in SourceMongoURI @pytest.mark.parametrize("secure_connection", [False], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", ["flat"]) -def test_simple(secure_connection, legacy, started_cluster, simple_tester, layout_name): +def test_simple(secure_connection, started_cluster, simple_tester, layout_name): simple_tester.execute(layout_name, started_cluster.instances["uri_node"]) @pytest.mark.parametrize("secure_connection", [True], indirect=["secure_connection"]) -@pytest.mark.parametrize("legacy", [False, True], indirect=["legacy"]) @pytest.mark.parametrize("layout_name", ["flat"]) def test_simple_ssl( - secure_connection, legacy, started_cluster, simple_tester, layout_name + secure_connection, started_cluster, simple_tester, layout_name ): simple_tester.execute(layout_name, started_cluster.instances["uri_node"]) diff --git a/tests/integration/test_storage_mongodb/configs/feature_flag.xml b/tests/integration/test_storage_mongodb/configs/feature_flag.xml deleted file mode 100644 index eb2d328060a..00000000000 --- a/tests/integration/test_storage_mongodb/configs/feature_flag.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 0 - diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index e810b613290..d69b9763b13 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -17,7 +17,6 @@ def started_cluster(request): "node", main_configs=[ "configs/named_collections.xml", - "configs/feature_flag.xml", ], user_configs=["configs/users.xml"], with_mongo=True, diff --git a/tests/integration/test_storage_mongodb_legacy/__init__.py b/tests/integration/test_storage_mongodb_legacy/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_mongodb_legacy/configs/feature_flag.xml b/tests/integration/test_storage_mongodb_legacy/configs/feature_flag.xml deleted file mode 100644 index 4ee05db9d1e..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/configs/feature_flag.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_storage_mongodb_legacy/configs/named_collections.xml b/tests/integration/test_storage_mongodb_legacy/configs/named_collections.xml deleted file mode 100644 index 5f7db390982..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/configs/named_collections.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - root - clickhouse - mongo1 - 27017 - test - simple_table - - - diff --git a/tests/integration/test_storage_mongodb_legacy/configs/users.xml b/tests/integration/test_storage_mongodb_legacy/configs/users.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/configs/users.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/cert.crt b/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/cert.crt deleted file mode 100644 index 94249109d41..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/cert.crt +++ /dev/null @@ -1,24 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIEGzCCAwOgAwIBAgIUaoGlyuJAyvs6yowFXymfu7seEiUwDQYJKoZIhvcNAQEL -BQAwgZwxCzAJBgNVBAYTAlVTMRMwEQYDVQQIDApDbGlja0hvdXNlMRMwEQYDVQQH -DApDbGlja0hvdXNlMREwDwYDVQQKDAhQZXJzb25hbDETMBEGA1UECwwKQ2xpY2tI -b3VzZTEkMCIGCSqGSIb3DQEJARYVY2xpY2tob3VzZUBjbGlja2hvdXNlMRUwEwYD -VQQDDAxtb25nb19zZWN1cmUwHhcNMjQwNTI2MTYwMDMxWhcNMzQwNTI0MTYwMDMx -WjCBnDELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNsaWNrSG91c2UxEzARBgNVBAcM -CkNsaWNrSG91c2UxETAPBgNVBAoMCFBlcnNvbmFsMRMwEQYDVQQLDApDbGlja0hv -dXNlMSQwIgYJKoZIhvcNAQkBFhVjbGlja2hvdXNlQGNsaWNraG91c2UxFTATBgNV -BAMMDG1vbmdvX3NlY3VyZTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB -AJSeQfMG7xd0+kPehYlEsEw0Sm1DB05SXVCEzIX3DFD6XJrd8eeWwlzYaBatkcwj -+8yvqske02X/3YwpzJyFizAqJIoKql5c5Yii2xH1S9PFP0y+LoJre+eQziHyO33t -eeedeGNJ05Sm2ZAzjfMQ7Rdh6S+gdIO4Y102iQR5yr2aTrh7tu7XkNCjwKTqMMvz -SikP1Rft2J6ECim+MjYCCtH/4yXGeEJ5epU4t3y6Q23B2ZEhY+sqUdwgK9pu8oe4 -mkZ1Qvwakc9Qg12owRSDjBBYrPvghXVpkJ2JkgKTrIAIz9tZ53eDVHNXbWMAotov -jEmRSoGIS1yzwmQ9PdxUwYcCAwEAAaNTMFEwHQYDVR0OBBYEFJyz3Kt5XBDg5cvI -0v1ioqejqX+CMB8GA1UdIwQYMBaAFJyz3Kt5XBDg5cvI0v1ioqejqX+CMA8GA1Ud -EwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAHAQFA5VMYvaQFnKtKfHg9TF -qfJ4uM3YsGdgsgmGWgflD1S4Z290H6Q2QvyZAEceTrlJxqArlWlVp5DAU6EeXjEh -QMAgdkJHF1Hg2jsZKPtdkb88UtuzwAME357T8NtEJSHzNE5QqYwlVM71JkWpdqvA -UUdOJbWhhJfowIf4tMmL1DUuIy2qYpoP/tEBXEw9uwpmZqb7KELwT3lRyOMaGFN7 -RHVwbvJWlHiu83QDNaWz6ijQkWl3tCN6TWcFD1qc1x8GpMzjbsAAYbCx7fbHM2LD -9kGSCiyv5K0MLNK5u67RtUFfPHtyD8RA0TtxIZ4PEN/eFANKS2/5NEi1ZuZ5/Pk= ------END CERTIFICATE----- diff --git a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml b/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml deleted file mode 100644 index 3efe98e7045..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - none - - - diff --git a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/key.pem b/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/key.pem deleted file mode 100644 index 9444d19a3d2..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/key.pem +++ /dev/null @@ -1,52 +0,0 @@ ------BEGIN PRIVATE KEY----- -MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCUnkHzBu8XdPpD -3oWJRLBMNEptQwdOUl1QhMyF9wxQ+lya3fHnlsJc2GgWrZHMI/vMr6rJHtNl/92M -KcychYswKiSKCqpeXOWIotsR9UvTxT9Mvi6Ca3vnkM4h8jt97XnnnXhjSdOUptmQ -M43zEO0XYekvoHSDuGNdNokEecq9mk64e7bu15DQo8Ck6jDL80opD9UX7diehAop -vjI2AgrR/+MlxnhCeXqVOLd8ukNtwdmRIWPrKlHcICvabvKHuJpGdUL8GpHPUINd -qMEUg4wQWKz74IV1aZCdiZICk6yACM/bWed3g1RzV21jAKLaL4xJkUqBiEtcs8Jk -PT3cVMGHAgMBAAECggEAAul6qiHchB+uQMCWyC5xTeRqAXR3tAv4Tj4fGJjkXY4Z -OrAjr9Kp38EvX1amgvUWV3FT3NMevDf5xd9OdzAA0g0uJIF+mAhYFW48i1FnQcHQ -mOf0zmiZR7l8o7ROb3JvooXHxW+ba/qjGPVwC801gJvruehgbOCRxh9DTRp7sH5K -BmcddhULhKBEQjWUmYNEM3A2axpdi3g1aYKERRLn8J0DXcItTwbxuxbNcs3erl8W -3yyv/JKmqnWF5sNyX3wEWuQcDEZZy+W7Hn4KPMxyU+WA5el5nJ8kFlxhpInmajwu -8Ytn6IEyThyXutVomosVBuP16QORl2Nad0hnQO9toQKBgQDDgiehXr3k2wfVaVOD -PocW4leXausIU2XcCn6FxTG9vLUDMPANw0MxgenC2nrjaUU9J9UjdRYgMcFGWrl4 -E27wEn5e0nZ/Y7F2cfhuOc9vNmZ+eHm2KQRyfAjIVL5Hpldqk2jXyCnLBNeWGHSw -kPQMU+FLqmrOFUvXlD2my+OSHwKBgQDCmgS9r+xFh4BCB9dY6eyQJF/jYmAQHs26 -80WJ6gAhbUw1O71uDtS9/3PZVXwwNCOHrcc49BPrpJdxGPHGvd2Q5y+j5LDDbQSZ -aLTiCZ2B0RM5Bd2dXD8gEHN4WCX7pJ/o4kDi4zONBmp5mg/tFfer5z5IU/1P7Wak -1Mu0JIHzmQKBgDNaNoqeVgaMuYwGtFbez6DlJtiwzrdLIJAheYYte5k4vdruub8D -sNyKIRp7RJgDCJq9obBEiuE98GRIZDrz78nDMco6QcHIL87KtNRO/vtZMKa7gkyk -jXR8u9nS2H/9YyytN3amLsQSq4XTOqM+D7xFNAIp6w/ibB9d4quzFj1FAoGBAKTE -x/LcO897NWuzO/D6z+QUCGR87R15F3SNenmVedrTskz4ciH3yMW+v5ZrPSWLX/IH -f8GHWD6TM+780eoW5L1GIh5BCjHN4rEJ6O3iekxqfD4x6zzL2F8Lztk8uZxh/Uuw -FoSFHybvIcQoYAe8K+KPfzq6cqb0OY6i5n920dkxAoGAJkw6ADqsJfH3NR+bQfgF -oEA1KqriMxyEJm44Y7E80C+iF4iNALF+Er9TSnr4mDxX5e/dW9d1YeS9o0nOfkpF -MaBmJfxqo4QQJLPRaxYQ2Jhfn7irir4BroxeNXQgNNhgSuKIvkfRyGYwl7P0AT4v -8H8rkZGneMD3gLB5MfnRhGk= ------END PRIVATE KEY----- ------BEGIN CERTIFICATE----- -MIIEGzCCAwOgAwIBAgIUaoGlyuJAyvs6yowFXymfu7seEiUwDQYJKoZIhvcNAQEL -BQAwgZwxCzAJBgNVBAYTAlVTMRMwEQYDVQQIDApDbGlja0hvdXNlMRMwEQYDVQQH -DApDbGlja0hvdXNlMREwDwYDVQQKDAhQZXJzb25hbDETMBEGA1UECwwKQ2xpY2tI -b3VzZTEkMCIGCSqGSIb3DQEJARYVY2xpY2tob3VzZUBjbGlja2hvdXNlMRUwEwYD -VQQDDAxtb25nb19zZWN1cmUwHhcNMjQwNTI2MTYwMDMxWhcNMzQwNTI0MTYwMDMx -WjCBnDELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNsaWNrSG91c2UxEzARBgNVBAcM -CkNsaWNrSG91c2UxETAPBgNVBAoMCFBlcnNvbmFsMRMwEQYDVQQLDApDbGlja0hv -dXNlMSQwIgYJKoZIhvcNAQkBFhVjbGlja2hvdXNlQGNsaWNraG91c2UxFTATBgNV -BAMMDG1vbmdvX3NlY3VyZTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB -AJSeQfMG7xd0+kPehYlEsEw0Sm1DB05SXVCEzIX3DFD6XJrd8eeWwlzYaBatkcwj -+8yvqske02X/3YwpzJyFizAqJIoKql5c5Yii2xH1S9PFP0y+LoJre+eQziHyO33t -eeedeGNJ05Sm2ZAzjfMQ7Rdh6S+gdIO4Y102iQR5yr2aTrh7tu7XkNCjwKTqMMvz -SikP1Rft2J6ECim+MjYCCtH/4yXGeEJ5epU4t3y6Q23B2ZEhY+sqUdwgK9pu8oe4 -mkZ1Qvwakc9Qg12owRSDjBBYrPvghXVpkJ2JkgKTrIAIz9tZ53eDVHNXbWMAotov -jEmRSoGIS1yzwmQ9PdxUwYcCAwEAAaNTMFEwHQYDVR0OBBYEFJyz3Kt5XBDg5cvI -0v1ioqejqX+CMB8GA1UdIwQYMBaAFJyz3Kt5XBDg5cvI0v1ioqejqX+CMA8GA1Ud -EwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAHAQFA5VMYvaQFnKtKfHg9TF -qfJ4uM3YsGdgsgmGWgflD1S4Z290H6Q2QvyZAEceTrlJxqArlWlVp5DAU6EeXjEh -QMAgdkJHF1Hg2jsZKPtdkb88UtuzwAME357T8NtEJSHzNE5QqYwlVM71JkWpdqvA -UUdOJbWhhJfowIf4tMmL1DUuIy2qYpoP/tEBXEw9uwpmZqb7KELwT3lRyOMaGFN7 -RHVwbvJWlHiu83QDNaWz6ijQkWl3tCN6TWcFD1qc1x8GpMzjbsAAYbCx7fbHM2LD -9kGSCiyv5K0MLNK5u67RtUFfPHtyD8RA0TtxIZ4PEN/eFANKS2/5NEi1ZuZ5/Pk= ------END CERTIFICATE----- diff --git a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_cert.pem b/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_cert.pem deleted file mode 100644 index 9444d19a3d2..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_cert.pem +++ /dev/null @@ -1,52 +0,0 @@ ------BEGIN PRIVATE KEY----- -MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCUnkHzBu8XdPpD -3oWJRLBMNEptQwdOUl1QhMyF9wxQ+lya3fHnlsJc2GgWrZHMI/vMr6rJHtNl/92M -KcychYswKiSKCqpeXOWIotsR9UvTxT9Mvi6Ca3vnkM4h8jt97XnnnXhjSdOUptmQ -M43zEO0XYekvoHSDuGNdNokEecq9mk64e7bu15DQo8Ck6jDL80opD9UX7diehAop -vjI2AgrR/+MlxnhCeXqVOLd8ukNtwdmRIWPrKlHcICvabvKHuJpGdUL8GpHPUINd -qMEUg4wQWKz74IV1aZCdiZICk6yACM/bWed3g1RzV21jAKLaL4xJkUqBiEtcs8Jk -PT3cVMGHAgMBAAECggEAAul6qiHchB+uQMCWyC5xTeRqAXR3tAv4Tj4fGJjkXY4Z -OrAjr9Kp38EvX1amgvUWV3FT3NMevDf5xd9OdzAA0g0uJIF+mAhYFW48i1FnQcHQ -mOf0zmiZR7l8o7ROb3JvooXHxW+ba/qjGPVwC801gJvruehgbOCRxh9DTRp7sH5K -BmcddhULhKBEQjWUmYNEM3A2axpdi3g1aYKERRLn8J0DXcItTwbxuxbNcs3erl8W -3yyv/JKmqnWF5sNyX3wEWuQcDEZZy+W7Hn4KPMxyU+WA5el5nJ8kFlxhpInmajwu -8Ytn6IEyThyXutVomosVBuP16QORl2Nad0hnQO9toQKBgQDDgiehXr3k2wfVaVOD -PocW4leXausIU2XcCn6FxTG9vLUDMPANw0MxgenC2nrjaUU9J9UjdRYgMcFGWrl4 -E27wEn5e0nZ/Y7F2cfhuOc9vNmZ+eHm2KQRyfAjIVL5Hpldqk2jXyCnLBNeWGHSw -kPQMU+FLqmrOFUvXlD2my+OSHwKBgQDCmgS9r+xFh4BCB9dY6eyQJF/jYmAQHs26 -80WJ6gAhbUw1O71uDtS9/3PZVXwwNCOHrcc49BPrpJdxGPHGvd2Q5y+j5LDDbQSZ -aLTiCZ2B0RM5Bd2dXD8gEHN4WCX7pJ/o4kDi4zONBmp5mg/tFfer5z5IU/1P7Wak -1Mu0JIHzmQKBgDNaNoqeVgaMuYwGtFbez6DlJtiwzrdLIJAheYYte5k4vdruub8D -sNyKIRp7RJgDCJq9obBEiuE98GRIZDrz78nDMco6QcHIL87KtNRO/vtZMKa7gkyk -jXR8u9nS2H/9YyytN3amLsQSq4XTOqM+D7xFNAIp6w/ibB9d4quzFj1FAoGBAKTE -x/LcO897NWuzO/D6z+QUCGR87R15F3SNenmVedrTskz4ciH3yMW+v5ZrPSWLX/IH -f8GHWD6TM+780eoW5L1GIh5BCjHN4rEJ6O3iekxqfD4x6zzL2F8Lztk8uZxh/Uuw -FoSFHybvIcQoYAe8K+KPfzq6cqb0OY6i5n920dkxAoGAJkw6ADqsJfH3NR+bQfgF -oEA1KqriMxyEJm44Y7E80C+iF4iNALF+Er9TSnr4mDxX5e/dW9d1YeS9o0nOfkpF -MaBmJfxqo4QQJLPRaxYQ2Jhfn7irir4BroxeNXQgNNhgSuKIvkfRyGYwl7P0AT4v -8H8rkZGneMD3gLB5MfnRhGk= ------END PRIVATE KEY----- ------BEGIN CERTIFICATE----- -MIIEGzCCAwOgAwIBAgIUaoGlyuJAyvs6yowFXymfu7seEiUwDQYJKoZIhvcNAQEL -BQAwgZwxCzAJBgNVBAYTAlVTMRMwEQYDVQQIDApDbGlja0hvdXNlMRMwEQYDVQQH -DApDbGlja0hvdXNlMREwDwYDVQQKDAhQZXJzb25hbDETMBEGA1UECwwKQ2xpY2tI -b3VzZTEkMCIGCSqGSIb3DQEJARYVY2xpY2tob3VzZUBjbGlja2hvdXNlMRUwEwYD -VQQDDAxtb25nb19zZWN1cmUwHhcNMjQwNTI2MTYwMDMxWhcNMzQwNTI0MTYwMDMx -WjCBnDELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNsaWNrSG91c2UxEzARBgNVBAcM -CkNsaWNrSG91c2UxETAPBgNVBAoMCFBlcnNvbmFsMRMwEQYDVQQLDApDbGlja0hv -dXNlMSQwIgYJKoZIhvcNAQkBFhVjbGlja2hvdXNlQGNsaWNraG91c2UxFTATBgNV -BAMMDG1vbmdvX3NlY3VyZTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB -AJSeQfMG7xd0+kPehYlEsEw0Sm1DB05SXVCEzIX3DFD6XJrd8eeWwlzYaBatkcwj -+8yvqske02X/3YwpzJyFizAqJIoKql5c5Yii2xH1S9PFP0y+LoJre+eQziHyO33t -eeedeGNJ05Sm2ZAzjfMQ7Rdh6S+gdIO4Y102iQR5yr2aTrh7tu7XkNCjwKTqMMvz -SikP1Rft2J6ECim+MjYCCtH/4yXGeEJ5epU4t3y6Q23B2ZEhY+sqUdwgK9pu8oe4 -mkZ1Qvwakc9Qg12owRSDjBBYrPvghXVpkJ2JkgKTrIAIz9tZ53eDVHNXbWMAotov -jEmRSoGIS1yzwmQ9PdxUwYcCAwEAAaNTMFEwHQYDVR0OBBYEFJyz3Kt5XBDg5cvI -0v1ioqejqX+CMB8GA1UdIwQYMBaAFJyz3Kt5XBDg5cvI0v1ioqejqX+CMA8GA1Ud -EwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAHAQFA5VMYvaQFnKtKfHg9TF -qfJ4uM3YsGdgsgmGWgflD1S4Z290H6Q2QvyZAEceTrlJxqArlWlVp5DAU6EeXjEh -QMAgdkJHF1Hg2jsZKPtdkb88UtuzwAME357T8NtEJSHzNE5QqYwlVM71JkWpdqvA -UUdOJbWhhJfowIf4tMmL1DUuIy2qYpoP/tEBXEw9uwpmZqb7KELwT3lRyOMaGFN7 -RHVwbvJWlHiu83QDNaWz6ijQkWl3tCN6TWcFD1qc1x8GpMzjbsAAYbCx7fbHM2LD -9kGSCiyv5K0MLNK5u67RtUFfPHtyD8RA0TtxIZ4PEN/eFANKS2/5NEi1ZuZ5/Pk= ------END CERTIFICATE----- diff --git a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_secure.conf b/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_secure.conf deleted file mode 100644 index 42d9853c6eb..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/mongo_secure_config/mongo_secure.conf +++ /dev/null @@ -1,6 +0,0 @@ -net: - ssl: - mode: requireSSL - PEMKeyFile: /mongo/key.pem - CAFile: /mongo/cert.crt - allowConnectionsWithoutCertificates: true diff --git a/tests/integration/test_storage_mongodb_legacy/test.py b/tests/integration/test_storage_mongodb_legacy/test.py deleted file mode 100644 index ddfd21279c9..00000000000 --- a/tests/integration/test_storage_mongodb_legacy/test.py +++ /dev/null @@ -1,509 +0,0 @@ -import datetime -from uuid import UUID - -import pymongo -import pytest - -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - - -@pytest.fixture(scope="module") -def started_cluster(request): - try: - cluster = ClickHouseCluster(__file__) - node = cluster.add_instance( - "node", - main_configs=[ - "mongo_secure_config/config.d/ssl_conf.xml", - "configs/named_collections.xml", - "configs/feature_flag.xml", - ], - user_configs=["configs/users.xml"], - with_mongo=True, - ) - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def get_mongo_connection(started_cluster, secure=False, with_credentials=True): - connection_str = "" - if with_credentials: - connection_str = "mongodb://root:clickhouse@localhost:{}".format( - started_cluster.mongo_secure_port if secure else started_cluster.mongo_port - ) - else: - connection_str = "mongodb://localhost:{}".format( - started_cluster.mongo_no_cred_port - ) - if secure: - connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" - return pymongo.MongoClient(connection_str) - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_uuid(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - mongo_table = db["uuid_table"] - mongo_table.insert({"key": 0, "data": UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed")}) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE uuid_mongo_table(key UInt64, data UUID) ENGINE = MongoDB('mongo1:27017', 'test', 'uuid_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM uuid_mongo_table") == "1\n" - assert ( - node.query("SELECT data from uuid_mongo_table where key = 0") - == "f0e77736-91d1-48ce-8f01-15123ca1c7ed\n" - ) - node.query("DROP TABLE uuid_mongo_table") - mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM simple_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from simple_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_select_from_view(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - simple_mongo_table_view = db.create_collection( - "simple_table_view", viewOn="simple_table" - ) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table_view', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM simple_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from simple_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table_view.drop() - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_arrays(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - arrays_mongo_table = db["arrays_table"] - data = [] - for i in range(0, 100): - data.append( - { - "key": i, - "arr_int64": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int32": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int16": [-(i + 1), -(i + 2), -(i + 3)], - "arr_int8": [-(i + 1), -(i + 2), -(i + 3)], - "arr_uint64": [i + 1, i + 2, i + 3], - "arr_uint32": [i + 1, i + 2, i + 3], - "arr_uint16": [i + 1, i + 2, i + 3], - "arr_uint8": [i + 1, i + 2, i + 3], - "arr_float32": [i + 1.125, i + 2.5, i + 3.750], - "arr_float64": [i + 1.125, i + 2.5, i + 3.750], - "arr_date": [ - datetime.datetime(2002, 10, 27), - datetime.datetime(2024, 1, 8), - ], - "arr_datetime": [ - datetime.datetime(2023, 3, 31, 6, 3, 12), - datetime.datetime(1999, 2, 28, 12, 46, 34), - ], - "arr_string": [str(i + 1), str(i + 2), str(i + 3)], - "arr_uuid": [ - "f0e77736-91d1-48ce-8f01-15123ca1c7ed", - "93376a07-c044-4281-a76e-ad27cf6973c5", - ], - "arr_mongo_uuid": [ - UUID("f0e77736-91d1-48ce-8f01-15123ca1c7ed"), - UUID("93376a07-c044-4281-a76e-ad27cf6973c5"), - ], - "arr_arr_bool": [ - [True, False, True], - [True], - [], - None, - [False], - [None], - ], - "arr_empty": [], - "arr_null": None, - "arr_nullable": None, - } - ) - - arrays_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE arrays_mongo_table(" - "key UInt64," - "arr_int64 Array(Int64)," - "arr_int32 Array(Int32)," - "arr_int16 Array(Int16)," - "arr_int8 Array(Int8)," - "arr_uint64 Array(UInt64)," - "arr_uint32 Array(UInt32)," - "arr_uint16 Array(UInt16)," - "arr_uint8 Array(UInt8)," - "arr_float32 Array(Float32)," - "arr_float64 Array(Float64)," - "arr_date Array(Date)," - "arr_datetime Array(DateTime)," - "arr_string Array(String)," - "arr_uuid Array(UUID)," - "arr_mongo_uuid Array(UUID)," - "arr_arr_bool Array(Array(Bool))," - "arr_empty Array(UInt64)," - "arr_null Array(UInt64)," - "arr_arr_null Array(Array(UInt64))," - "arr_nullable Array(Nullable(UInt64))" - ") ENGINE = MongoDB('mongo1:27017', 'test', 'arrays_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "100\n" - - for column_name in ["arr_int64", "arr_int32", "arr_int16", "arr_int8"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[-43,-44,-45]\n" - ) - - for column_name in ["arr_uint64", "arr_uint32", "arr_uint16", "arr_uint8"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[43,44,45]\n" - ) - - for column_name in ["arr_float32", "arr_float64"]: - assert ( - node.query(f"SELECT {column_name} FROM arrays_mongo_table WHERE key = 42") - == "[43.125,44.5,45.75]\n" - ) - - assert ( - node.query(f"SELECT arr_date FROM arrays_mongo_table WHERE key = 42") - == "['2002-10-27','2024-01-08']\n" - ) - - assert ( - node.query(f"SELECT arr_datetime FROM arrays_mongo_table WHERE key = 42") - == "['2023-03-31 06:03:12','1999-02-28 12:46:34']\n" - ) - - assert ( - node.query(f"SELECT arr_string FROM arrays_mongo_table WHERE key = 42") - == "['43','44','45']\n" - ) - - assert ( - node.query(f"SELECT arr_uuid FROM arrays_mongo_table WHERE key = 42") - == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" - ) - - assert ( - node.query(f"SELECT arr_mongo_uuid FROM arrays_mongo_table WHERE key = 42") - == "['f0e77736-91d1-48ce-8f01-15123ca1c7ed','93376a07-c044-4281-a76e-ad27cf6973c5']\n" - ) - - assert ( - node.query(f"SELECT arr_arr_bool FROM arrays_mongo_table WHERE key = 42") - == "[[true,false,true],[true],[],[],[false],[false]]\n" - ) - - assert ( - node.query(f"SELECT arr_empty FROM arrays_mongo_table WHERE key = 42") == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_null FROM arrays_mongo_table WHERE key = 42") == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_arr_null FROM arrays_mongo_table WHERE key = 42") - == "[]\n" - ) - - assert ( - node.query(f"SELECT arr_nullable FROM arrays_mongo_table WHERE key = 42") - == "[]\n" - ) - - # Test INSERT SELECT - node.query("INSERT INTO arrays_mongo_table SELECT * FROM arrays_mongo_table") - - assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "200\n" - assert node.query("SELECT COUNT(DISTINCT *) FROM arrays_mongo_table") == "100\n" - - node.query("DROP TABLE arrays_mongo_table") - arrays_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - incomplete_mongo_table = db["complex_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) - incomplete_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE incomplete_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse')" - ) - - assert node.query("SELECT COUNT() FROM incomplete_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM incomplete_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from incomplete_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE incomplete_mongo_table") - incomplete_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - strange_mongo_table = db["strange_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) - strange_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE strange_mongo_table(key String, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" - ) - - with pytest.raises(QueryRuntimeException): - node.query("SELECT COUNT() FROM strange_mongo_table") - - with pytest.raises(QueryRuntimeException): - node.query("SELECT uniq(key) FROM strange_mongo_table") - - node.query( - "CREATE TABLE strange_mongo_table2(key UInt64, data String, bbbb String) ENGINE = MongoDB('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse')" - ) - - node.query("DROP TABLE strange_mongo_table") - node.query("DROP TABLE strange_mongo_table2") - strange_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) -def test_secure_connection(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, secure=True) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo_secure:27017', 'test', 'simple_table', 'root', 'clickhouse', 'ssl=true')" - ) - - assert node.query("SELECT COUNT() FROM simple_mongo_table") == "100\n" - assert ( - node.query("SELECT sum(key) FROM simple_mongo_table") - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query("SELECT data from simple_mongo_table where key = 42") - == hex(42 * 42) + "\n" - ) - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_predefined_connection_configuration(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query("drop table if exists simple_mongo_table") - node.query( - "create table simple_mongo_table(key UInt64, data String) engine = MongoDB(mongo1)" - ) - assert node.query("SELECT count() FROM simple_mongo_table") == "100\n" - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_no_credentials(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - f"create table simple_mongo_table_2(key UInt64, data String) engine = MongoDB('mongo_no_cred:27017', 'test', 'simple_table', '', '')" - ) - assert node.query("SELECT count() FROM simple_mongo_table_2") == "100\n" - simple_mongo_table.drop() - node.query("DROP TABLE IF EXISTS simple_mongo_table_2") - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_auth_source(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - admin_db = mongo_connection["admin"] - admin_db.add_user( - "root", - "clickhouse", - roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], - ) - simple_mongo_table_admin = admin_db["simple_table"] - data = [] - for i in range(0, 50): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table_admin.insert_many(data) - - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query( - "create table simple_mongo_table_fail(key UInt64, data String) engine = MongoDB('mongo_no_cred:27017', 'test', 'simple_table', 'root', 'clickhouse')" - ) - node.query_and_get_error("SELECT count() FROM simple_mongo_table_fail") - node.query( - "create table simple_mongo_table_ok(key UInt64, data String) engine = MongoDB('mongo_no_cred:27017', 'test', 'simple_table', 'root', 'clickhouse', 'authSource=admin')" - ) - assert node.query("SELECT count() FROM simple_mongo_table_ok") == "100\n" - simple_mongo_table.drop() - simple_mongo_table_admin.drop() - node.query("DROP TABLE IF EXISTS simple_mongo_table_ok") - node.query("DROP TABLE IF EXISTS simple_mongo_table_fail") - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_missing_columns(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 10): - data.append({"key": i, "data": hex(i * i)}) - for i in range(0, 10): - data.append({"key": i}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - node.query("drop table if exists simple_mongo_table") - node.query( - "create table simple_mongo_table(key UInt64, data Nullable(String)) engine = MongoDB(mongo1)" - ) - result = node.query("SELECT count() FROM simple_mongo_table WHERE isNull(data)") - assert result == "10\n" - simple_mongo_table.drop() - node.query("DROP TABLE IF EXISTS simple_mongo_table") - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_insert_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - - node = started_cluster.instances["node"] - node.query("DROP TABLE IF EXISTS simple_mongo_table") - node.query( - "CREATE TABLE simple_mongo_table(key UInt64, data String) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')" - ) - node.query( - "INSERT INTO simple_mongo_table SELECT number, 'kek' || toString(number) FROM numbers(10)" - ) - - assert ( - node.query("SELECT data from simple_mongo_table where key = 7").strip() - == "kek7" - ) - node.query("INSERT INTO simple_mongo_table(key) SELECT 12") - assert int(node.query("SELECT count() from simple_mongo_table")) == 11 - assert ( - node.query("SELECT data from simple_mongo_table where key = 12").strip() == "" - ) - - node.query("DROP TABLE simple_mongo_table") - simple_mongo_table.drop() diff --git a/tests/integration/test_table_function_mongodb/configs/feature_flag.xml b/tests/integration/test_table_function_mongodb/configs/feature_flag.xml deleted file mode 100644 index eb2d328060a..00000000000 --- a/tests/integration/test_table_function_mongodb/configs/feature_flag.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 0 - diff --git a/tests/integration/test_table_function_mongodb/test.py b/tests/integration/test_table_function_mongodb/test.py index b3e990cf73b..c4c27b079f2 100644 --- a/tests/integration/test_table_function_mongodb/test.py +++ b/tests/integration/test_table_function_mongodb/test.py @@ -14,7 +14,6 @@ def started_cluster(request): with_mongo=True, main_configs=[ "configs/named_collections.xml", - "configs/feature_flag.xml", ], user_configs=["configs/users.xml"], ) diff --git a/tests/integration/test_table_function_mongodb_legacy/__init__.py b/tests/integration/test_table_function_mongodb_legacy/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_table_function_mongodb_legacy/configs/feature_flag.xml b/tests/integration/test_table_function_mongodb_legacy/configs/feature_flag.xml deleted file mode 100644 index 4ee05db9d1e..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/configs/feature_flag.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_table_function_mongodb_legacy/configs/users.xml b/tests/integration/test_table_function_mongodb_legacy/configs/users.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/configs/users.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/cert.crt b/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/cert.crt deleted file mode 100644 index 94249109d41..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/cert.crt +++ /dev/null @@ -1,24 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIEGzCCAwOgAwIBAgIUaoGlyuJAyvs6yowFXymfu7seEiUwDQYJKoZIhvcNAQEL -BQAwgZwxCzAJBgNVBAYTAlVTMRMwEQYDVQQIDApDbGlja0hvdXNlMRMwEQYDVQQH -DApDbGlja0hvdXNlMREwDwYDVQQKDAhQZXJzb25hbDETMBEGA1UECwwKQ2xpY2tI -b3VzZTEkMCIGCSqGSIb3DQEJARYVY2xpY2tob3VzZUBjbGlja2hvdXNlMRUwEwYD -VQQDDAxtb25nb19zZWN1cmUwHhcNMjQwNTI2MTYwMDMxWhcNMzQwNTI0MTYwMDMx -WjCBnDELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNsaWNrSG91c2UxEzARBgNVBAcM -CkNsaWNrSG91c2UxETAPBgNVBAoMCFBlcnNvbmFsMRMwEQYDVQQLDApDbGlja0hv -dXNlMSQwIgYJKoZIhvcNAQkBFhVjbGlja2hvdXNlQGNsaWNraG91c2UxFTATBgNV -BAMMDG1vbmdvX3NlY3VyZTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB -AJSeQfMG7xd0+kPehYlEsEw0Sm1DB05SXVCEzIX3DFD6XJrd8eeWwlzYaBatkcwj -+8yvqske02X/3YwpzJyFizAqJIoKql5c5Yii2xH1S9PFP0y+LoJre+eQziHyO33t -eeedeGNJ05Sm2ZAzjfMQ7Rdh6S+gdIO4Y102iQR5yr2aTrh7tu7XkNCjwKTqMMvz -SikP1Rft2J6ECim+MjYCCtH/4yXGeEJ5epU4t3y6Q23B2ZEhY+sqUdwgK9pu8oe4 -mkZ1Qvwakc9Qg12owRSDjBBYrPvghXVpkJ2JkgKTrIAIz9tZ53eDVHNXbWMAotov -jEmRSoGIS1yzwmQ9PdxUwYcCAwEAAaNTMFEwHQYDVR0OBBYEFJyz3Kt5XBDg5cvI -0v1ioqejqX+CMB8GA1UdIwQYMBaAFJyz3Kt5XBDg5cvI0v1ioqejqX+CMA8GA1Ud -EwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAHAQFA5VMYvaQFnKtKfHg9TF -qfJ4uM3YsGdgsgmGWgflD1S4Z290H6Q2QvyZAEceTrlJxqArlWlVp5DAU6EeXjEh -QMAgdkJHF1Hg2jsZKPtdkb88UtuzwAME357T8NtEJSHzNE5QqYwlVM71JkWpdqvA -UUdOJbWhhJfowIf4tMmL1DUuIy2qYpoP/tEBXEw9uwpmZqb7KELwT3lRyOMaGFN7 -RHVwbvJWlHiu83QDNaWz6ijQkWl3tCN6TWcFD1qc1x8GpMzjbsAAYbCx7fbHM2LD -9kGSCiyv5K0MLNK5u67RtUFfPHtyD8RA0TtxIZ4PEN/eFANKS2/5NEi1ZuZ5/Pk= ------END CERTIFICATE----- diff --git a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml b/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml deleted file mode 100644 index 3efe98e7045..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/config.d/ssl_conf.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - none - - - diff --git a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/key.pem b/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/key.pem deleted file mode 100644 index 9444d19a3d2..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/key.pem +++ /dev/null @@ -1,52 +0,0 @@ ------BEGIN PRIVATE KEY----- -MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCUnkHzBu8XdPpD -3oWJRLBMNEptQwdOUl1QhMyF9wxQ+lya3fHnlsJc2GgWrZHMI/vMr6rJHtNl/92M -KcychYswKiSKCqpeXOWIotsR9UvTxT9Mvi6Ca3vnkM4h8jt97XnnnXhjSdOUptmQ -M43zEO0XYekvoHSDuGNdNokEecq9mk64e7bu15DQo8Ck6jDL80opD9UX7diehAop -vjI2AgrR/+MlxnhCeXqVOLd8ukNtwdmRIWPrKlHcICvabvKHuJpGdUL8GpHPUINd -qMEUg4wQWKz74IV1aZCdiZICk6yACM/bWed3g1RzV21jAKLaL4xJkUqBiEtcs8Jk -PT3cVMGHAgMBAAECggEAAul6qiHchB+uQMCWyC5xTeRqAXR3tAv4Tj4fGJjkXY4Z -OrAjr9Kp38EvX1amgvUWV3FT3NMevDf5xd9OdzAA0g0uJIF+mAhYFW48i1FnQcHQ -mOf0zmiZR7l8o7ROb3JvooXHxW+ba/qjGPVwC801gJvruehgbOCRxh9DTRp7sH5K -BmcddhULhKBEQjWUmYNEM3A2axpdi3g1aYKERRLn8J0DXcItTwbxuxbNcs3erl8W -3yyv/JKmqnWF5sNyX3wEWuQcDEZZy+W7Hn4KPMxyU+WA5el5nJ8kFlxhpInmajwu -8Ytn6IEyThyXutVomosVBuP16QORl2Nad0hnQO9toQKBgQDDgiehXr3k2wfVaVOD -PocW4leXausIU2XcCn6FxTG9vLUDMPANw0MxgenC2nrjaUU9J9UjdRYgMcFGWrl4 -E27wEn5e0nZ/Y7F2cfhuOc9vNmZ+eHm2KQRyfAjIVL5Hpldqk2jXyCnLBNeWGHSw -kPQMU+FLqmrOFUvXlD2my+OSHwKBgQDCmgS9r+xFh4BCB9dY6eyQJF/jYmAQHs26 -80WJ6gAhbUw1O71uDtS9/3PZVXwwNCOHrcc49BPrpJdxGPHGvd2Q5y+j5LDDbQSZ -aLTiCZ2B0RM5Bd2dXD8gEHN4WCX7pJ/o4kDi4zONBmp5mg/tFfer5z5IU/1P7Wak -1Mu0JIHzmQKBgDNaNoqeVgaMuYwGtFbez6DlJtiwzrdLIJAheYYte5k4vdruub8D -sNyKIRp7RJgDCJq9obBEiuE98GRIZDrz78nDMco6QcHIL87KtNRO/vtZMKa7gkyk -jXR8u9nS2H/9YyytN3amLsQSq4XTOqM+D7xFNAIp6w/ibB9d4quzFj1FAoGBAKTE -x/LcO897NWuzO/D6z+QUCGR87R15F3SNenmVedrTskz4ciH3yMW+v5ZrPSWLX/IH -f8GHWD6TM+780eoW5L1GIh5BCjHN4rEJ6O3iekxqfD4x6zzL2F8Lztk8uZxh/Uuw -FoSFHybvIcQoYAe8K+KPfzq6cqb0OY6i5n920dkxAoGAJkw6ADqsJfH3NR+bQfgF -oEA1KqriMxyEJm44Y7E80C+iF4iNALF+Er9TSnr4mDxX5e/dW9d1YeS9o0nOfkpF -MaBmJfxqo4QQJLPRaxYQ2Jhfn7irir4BroxeNXQgNNhgSuKIvkfRyGYwl7P0AT4v -8H8rkZGneMD3gLB5MfnRhGk= ------END PRIVATE KEY----- ------BEGIN CERTIFICATE----- -MIIEGzCCAwOgAwIBAgIUaoGlyuJAyvs6yowFXymfu7seEiUwDQYJKoZIhvcNAQEL -BQAwgZwxCzAJBgNVBAYTAlVTMRMwEQYDVQQIDApDbGlja0hvdXNlMRMwEQYDVQQH -DApDbGlja0hvdXNlMREwDwYDVQQKDAhQZXJzb25hbDETMBEGA1UECwwKQ2xpY2tI -b3VzZTEkMCIGCSqGSIb3DQEJARYVY2xpY2tob3VzZUBjbGlja2hvdXNlMRUwEwYD -VQQDDAxtb25nb19zZWN1cmUwHhcNMjQwNTI2MTYwMDMxWhcNMzQwNTI0MTYwMDMx -WjCBnDELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNsaWNrSG91c2UxEzARBgNVBAcM -CkNsaWNrSG91c2UxETAPBgNVBAoMCFBlcnNvbmFsMRMwEQYDVQQLDApDbGlja0hv -dXNlMSQwIgYJKoZIhvcNAQkBFhVjbGlja2hvdXNlQGNsaWNraG91c2UxFTATBgNV -BAMMDG1vbmdvX3NlY3VyZTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEB -AJSeQfMG7xd0+kPehYlEsEw0Sm1DB05SXVCEzIX3DFD6XJrd8eeWwlzYaBatkcwj -+8yvqske02X/3YwpzJyFizAqJIoKql5c5Yii2xH1S9PFP0y+LoJre+eQziHyO33t -eeedeGNJ05Sm2ZAzjfMQ7Rdh6S+gdIO4Y102iQR5yr2aTrh7tu7XkNCjwKTqMMvz -SikP1Rft2J6ECim+MjYCCtH/4yXGeEJ5epU4t3y6Q23B2ZEhY+sqUdwgK9pu8oe4 -mkZ1Qvwakc9Qg12owRSDjBBYrPvghXVpkJ2JkgKTrIAIz9tZ53eDVHNXbWMAotov -jEmRSoGIS1yzwmQ9PdxUwYcCAwEAAaNTMFEwHQYDVR0OBBYEFJyz3Kt5XBDg5cvI -0v1ioqejqX+CMB8GA1UdIwQYMBaAFJyz3Kt5XBDg5cvI0v1ioqejqX+CMA8GA1Ud -EwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAHAQFA5VMYvaQFnKtKfHg9TF -qfJ4uM3YsGdgsgmGWgflD1S4Z290H6Q2QvyZAEceTrlJxqArlWlVp5DAU6EeXjEh -QMAgdkJHF1Hg2jsZKPtdkb88UtuzwAME357T8NtEJSHzNE5QqYwlVM71JkWpdqvA -UUdOJbWhhJfowIf4tMmL1DUuIy2qYpoP/tEBXEw9uwpmZqb7KELwT3lRyOMaGFN7 -RHVwbvJWlHiu83QDNaWz6ijQkWl3tCN6TWcFD1qc1x8GpMzjbsAAYbCx7fbHM2LD -9kGSCiyv5K0MLNK5u67RtUFfPHtyD8RA0TtxIZ4PEN/eFANKS2/5NEi1ZuZ5/Pk= ------END CERTIFICATE----- diff --git a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/mongo_secure.conf b/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/mongo_secure.conf deleted file mode 100644 index 42d9853c6eb..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/mongo_secure_config/mongo_secure.conf +++ /dev/null @@ -1,6 +0,0 @@ -net: - ssl: - mode: requireSSL - PEMKeyFile: /mongo/key.pem - CAFile: /mongo/cert.crt - allowConnectionsWithoutCertificates: true diff --git a/tests/integration/test_table_function_mongodb_legacy/test.py b/tests/integration/test_table_function_mongodb_legacy/test.py deleted file mode 100644 index 353ef4d1dff..00000000000 --- a/tests/integration/test_table_function_mongodb_legacy/test.py +++ /dev/null @@ -1,276 +0,0 @@ -import pymongo -import pytest - -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - - -@pytest.fixture(scope="module") -def started_cluster(request): - try: - cluster = ClickHouseCluster(__file__) - node = cluster.add_instance( - "node", - with_mongo=True, - main_configs=[ - "mongo_secure_config/config.d/ssl_conf.xml", - "configs/feature_flag.xml", - ], - user_configs=["configs/users.xml"], - ) - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def get_mongo_connection(started_cluster, secure=False, with_credentials=True): - connection_str = "" - if with_credentials: - connection_str = "mongodb://root:clickhouse@localhost:{}".format( - started_cluster.mongo_secure_port if secure else started_cluster.mongo_port - ) - else: - connection_str = "mongodb://localhost:{}".format( - started_cluster.mongo_no_cred_port - ) - if secure: - connection_str += "/?tls=true&tlsAllowInvalidCertificates=true" - return pymongo.MongoClient(connection_str) - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - - node = started_cluster.instances["node"] - for i in range(0, 100): - node.query( - "INSERT INTO FUNCTION mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') (key, data) VALUES ({}, '{}')".format( - i, hex(i * i) - ) - ) - assert ( - node.query( - "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == "100\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == str(sum(range(0, 100))) + "\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - incomplete_mongo_table = db["complex_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "dict": {"a": i, "b": str(i)}}) - incomplete_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - assert ( - node.query( - "SELECT COUNT() FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" - ) - == "100\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo1:27017', 'test', 'complex_table', 'root', 'clickhouse', structure='key UInt64, data String, dict Map(UInt64, String)') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - incomplete_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - strange_mongo_table = db["strange_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i), "aaaa": "Hello"}) - strange_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - with pytest.raises(QueryRuntimeException): - node.query( - "SELECT aaaa FROM mongodb('mongo1:27017', 'test', 'strange_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - - strange_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [True], indirect=["started_cluster"]) -def test_secure_connection(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, secure=True) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - assert ( - node.query( - "SELECT COUNT() FROM mongodb('mongo_secure:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" - ) - == "100\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo_secure:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true')" - ) - == str(sum(range(0, 100))) + "\n" - ) - assert ( - node.query( - "SELECT sum(key) FROM mongodb('mongo_secure:27017', 'test', 'simple_table', 'root', 'clickhouse', 'key UInt64, data String', 'ssl=true')" - ) - == str(sum(range(0, 100))) + "\n" - ) - - assert ( - node.query( - "SELECT data from mongodb('mongo_secure:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='ssl=true') where key = 42" - ) - == hex(42 * 42) + "\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_predefined_connection_configuration(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - assert ( - node.query( - "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_no_credentials(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - assert ( - node.query( - "SELECT count() FROM mongodb('mongo_no_cred:27017', 'test', 'simple_table', '', '', structure='key UInt64, data String')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_auth_source(started_cluster): - mongo_connection = get_mongo_connection(started_cluster, with_credentials=False) - admin_db = mongo_connection["admin"] - admin_db.add_user( - "root", - "clickhouse", - roles=[{"role": "userAdminAnyDatabase", "db": "admin"}, "readWriteAnyDatabase"], - ) - simple_mongo_table = admin_db["simple_table"] - data = [] - for i in range(0, 50): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - db = mongo_connection["test"] - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 100): - data.append({"key": i, "data": hex(i * i)}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - - node.query_and_get_error( - "SELECT count() FROM mongodb('mongo_no_cred:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String')" - ) - - assert ( - node.query( - "SELECT count() FROM mongodb('mongo_no_cred:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data String', options='authSource=admin')" - ) - == "100\n" - ) - simple_mongo_table.drop() - - -@pytest.mark.parametrize("started_cluster", [False], indirect=["started_cluster"]) -def test_missing_columns(started_cluster): - mongo_connection = get_mongo_connection(started_cluster) - db = mongo_connection["test"] - db.add_user("root", "clickhouse") - simple_mongo_table = db["simple_table"] - data = [] - for i in range(0, 10): - data.append({"key": i, "data": hex(i * i)}) - for i in range(0, 10): - data.append({"key": i}) - simple_mongo_table.insert_many(data) - - node = started_cluster.instances["node"] - result = node.query( - "SELECT count() FROM mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse', structure='key UInt64, data Nullable(String)') WHERE isNull(data)" - ) - assert result == "10\n" - simple_mongo_table.drop() diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql index 7e4adbb7b5b..0e01bd7afbb 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql @@ -3,5 +3,4 @@ SELECT name FROM system.table_functions WHERE length(description) < 10 AND name NOT IN ( 'cosn', 'oss', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite', 'urlCluster', 'mergeTreeParts' -- these functions are not enabled in fast test - , 'mongodb' -- will be removed when `use_legacy_mongodb_integration` setting will be purged will with the old implementation ) ORDER BY name; From 1c5b362f170e16a934f80f26f93238a070de22e1 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Sat, 16 Nov 2024 16:12:12 +0400 Subject: [PATCH 111/433] fix style --- .../test_mongo.py | 8 ++------ .../test_mongo_uri.py | 4 +--- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index 2cf6250b106..fa271ae5cc0 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -105,9 +105,7 @@ def test_simple(secure_connection, started_cluster, layout_name, simple_tester): @pytest.mark.parametrize("secure_connection", [False], indirect=["secure_connection"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX)) -def test_complex( - secure_connection, started_cluster, layout_name, complex_tester -): +def test_complex(secure_connection, started_cluster, layout_name, complex_tester): complex_tester.execute(layout_name, started_cluster.instances["node1"]) @@ -119,7 +117,5 @@ def test_ranged(secure_connection, started_cluster, layout_name, ranged_tester): @pytest.mark.parametrize("secure_connection", [True], indirect=["secure_connection"]) @pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) -def test_simple_ssl( - secure_connection, started_cluster, layout_name, simple_tester -): +def test_simple_ssl(secure_connection, started_cluster, layout_name, simple_tester): simple_tester.execute(layout_name, started_cluster.instances["node1"]) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 407031ed7d3..5d632ba6996 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -78,7 +78,5 @@ def test_simple(secure_connection, started_cluster, simple_tester, layout_name): @pytest.mark.parametrize("secure_connection", [True], indirect=["secure_connection"]) @pytest.mark.parametrize("layout_name", ["flat"]) -def test_simple_ssl( - secure_connection, started_cluster, simple_tester, layout_name -): +def test_simple_ssl(secure_connection, started_cluster, simple_tester, layout_name): simple_tester.execute(layout_name, started_cluster.instances["uri_node"]) From 2906524f0dbd1d182d848c143f951b5cc4b85b85 Mon Sep 17 00:00:00 2001 From: Kirill Nikiforov Date: Sat, 16 Nov 2024 16:56:05 +0400 Subject: [PATCH 112/433] fix --- src/Dictionaries/tests/gtest_dictionary_configuration.cpp | 8 ++++---- src/Interpreters/fuzzers/execute_query_fuzzer.cpp | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 1f803567cf8..08aad663a8c 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -30,7 +30,7 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration) { if (!registered) { - registerDictionaries(false); + registerDictionaries(); registered = true; } @@ -103,7 +103,7 @@ TEST(ConvertDictionaryAST, TrickyAttributes) { if (!registered) { - registerDictionaries(false); + registerDictionaries(); registered = true; } @@ -147,7 +147,7 @@ TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams) { if (!registered) { - registerDictionaries(false); + registerDictionaries(); registered = true; } @@ -198,7 +198,7 @@ TEST(ConvertDictionaryAST, ComplexSource) { if (!registered) { - registerDictionaries(false); + registerDictionaries(); registered = true; } diff --git a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp index 908ac6322d0..c29efae1e7d 100644 --- a/src/Interpreters/fuzzers/execute_query_fuzzer.cpp +++ b/src/Interpreters/fuzzers/execute_query_fuzzer.cpp @@ -29,10 +29,10 @@ extern "C" int LLVMFuzzerInitialize(int *, char ***) registerInterpreters(); registerFunctions(); registerAggregateFunctions(); - registerTableFunctions(false); + registerTableFunctions(); registerDatabases(); - registerStorages(false); - registerDictionaries(false); + registerStorages(); + registerDictionaries(); registerDisks(/* global_skip_access_check= */ true); registerFormats(); From e29fc6ee63365098313998cba6bfbf80b7978da0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 18 Nov 2024 10:34:37 +0000 Subject: [PATCH 113/433] fix build --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 9 --------- src/Processors/QueryPlan/JoinStep.cpp | 6 ++++-- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index de8d38be70d..014ff1d6277 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -81,15 +81,6 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. - * Because if they are constants, then in the "not joined" rows, they may have different values - * - default values, which can differ from the values of these constants. - */ - if constexpr (join_features.right || join_features.full) - { - materializeBlockInplace(block); - } - auto & source_block = block.getSourceBlock(); /** For LEFT/INNER JOIN, the saved blocks do not contain keys. diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 9527b08373b..33b85e8af54 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -83,6 +83,7 @@ JoinStep::JoinStep( bool use_new_analyzer_) : join(std::move(join_)) , max_block_size(max_block_size_) + , min_block_size_bytes(min_block_size_bytes_) , max_streams(max_streams_) , required_output(std::move(required_output_)) , keep_left_read_in_order(keep_left_read_in_order_) @@ -117,6 +118,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines join, join_algorithm_header, max_block_size, + min_block_size_bytes, max_streams, keep_left_read_in_order, &processors); @@ -136,11 +138,11 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines if (join->supportParallelJoin()) { - pipeline->addSimpleTransform([&](const Block & header) + joined_pipeline->addSimpleTransform([&](const Block & header) { return std::make_shared(header, 0, min_block_size_bytes); }); } - return pipeline; + return joined_pipeline; } bool JoinStep::allowPushDownToRight() const From b8c70bf059e65771ac39adf4ec3dc5d55cc7b6f9 Mon Sep 17 00:00:00 2001 From: zhangwanyun1 Date: Tue, 19 Nov 2024 11:19:23 +0800 Subject: [PATCH 114/433] Re run pipeline From 82a7c33796afa5a9713a57050cdea701504c50d8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Nov 2024 13:18:37 +0000 Subject: [PATCH 115/433] fix --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 014ff1d6277..105b6e5d5bf 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -82,6 +82,7 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( } auto & source_block = block.getSourceBlock(); + size_t existing_columns = source_block.columns(); /** For LEFT/INNER JOIN, the saved blocks do not contain keys. * For FULL/RIGHT JOIN, the saved blocks contain keys; @@ -184,7 +185,7 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( chassert(offsets.size() == block.rows()); auto && columns = block.getSourceBlock().getColumns(); - for (size_t i = 0; i < columns.size(); ++i) + for (size_t i = 0; i < existing_columns; ++i) columns[i] = columns[i]->replicate(offsets); for (size_t pos : right_keys_to_replicate) columns[pos] = columns[pos]->replicate(offsets); From a74d615aafde161caa2ec2cb9e88932d6803d34a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Nov 2024 16:15:12 +0000 Subject: [PATCH 116/433] added primary index cache --- programs/local/LocalServer.cpp | 13 ++ programs/server/Server.cpp | 14 ++ src/Access/Common/AccessType.h | 2 + src/Common/ProfileEvents.cpp | 5 + src/Core/ServerSettings.cpp | 4 + src/Interpreters/Context.cpp | 41 ++++ src/Interpreters/Context.h | 6 + src/Interpreters/InterpreterSystemQuery.cpp | 36 ++- src/Interpreters/InterpreterSystemQuery.h | 4 +- .../ServerAsynchronousMetrics.cpp | 6 + src/Parsers/ASTSystemQuery.cpp | 2 + src/Parsers/ASTSystemQuery.h | 2 + src/Parsers/ParserSystemQuery.cpp | 1 + src/Processors/QueryPlan/PartsSplitter.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 210 +++++++++++------- src/Storages/MergeTree/IMergeTreeDataPart.h | 21 +- .../MergeTree/IMergeTreeDataPartWriter.cpp | 6 +- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- .../MergeTree/IMergedBlockOutputStream.h | 1 + .../MergeTree/MergeFromLogEntryTask.cpp | 9 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 11 +- src/Storages/MergeTree/MergeTask.cpp | 7 - src/Storages/MergeTree/MergeTask.h | 4 + src/Storages/MergeTree/MergeTreeData.cpp | 62 ++++-- src/Storages/MergeTree/MergeTreeData.h | 12 +- .../MergeTreeDataPartWriterOnDisk.cpp | 15 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 5 - .../MergeTree/MergeTreeIOSettings.cpp | 2 + src/Storages/MergeTree/MergeTreeIOSettings.h | 2 + src/Storages/MergeTree/MergeTreeSettings.cpp | 2 + src/Storages/MergeTree/MergeTreeSink.cpp | 11 +- .../MergeTree/MergedBlockOutputStream.cpp | 15 +- .../MergeTree/MergedBlockOutputStream.h | 1 - .../MergedColumnOnlyOutputStream.cpp | 14 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 - src/Storages/MergeTree/MutateTask.cpp | 2 - src/Storages/MergeTree/PrimaryIndexCache.cpp | 8 + src/Storages/MergeTree/PrimaryIndexCache.h | 73 ++++++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 42 ++-- .../MergeTree/ReplicatedMergeTreeSink.h | 2 + src/Storages/MergeTree/new_index | 0 src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 +- .../03273_primary_index_cache.reference | 16 ++ .../0_stateless/03273_primary_index_cache.sql | 45 ++++ ...3274_prewarm_primary_index_cache.reference | 16 ++ .../03274_prewarm_primary_index_cache.sql | 46 ++++ 48 files changed, 646 insertions(+), 174 deletions(-) create mode 100644 src/Storages/MergeTree/PrimaryIndexCache.cpp create mode 100644 src/Storages/MergeTree/PrimaryIndexCache.h create mode 100644 src/Storages/MergeTree/new_index create mode 100644 tests/queries/0_stateless/03273_primary_index_cache.reference create mode 100644 tests/queries/0_stateless/03273_primary_index_cache.sql create mode 100644 tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference create mode 100644 tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e6f8ecef097..adfd3adcb2a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -110,6 +110,9 @@ namespace ServerSetting extern const ServerSettingsString uncompressed_cache_policy; extern const ServerSettingsUInt64 uncompressed_cache_size; extern const ServerSettingsDouble uncompressed_cache_size_ratio; + extern const ServerSettingsString primary_index_cache_policy; + extern const ServerSettingsUInt64 primary_index_cache_size; + extern const ServerSettingsDouble primary_index_cache_size_ratio; extern const ServerSettingsBool use_legacy_mongodb_integration; } @@ -778,6 +781,16 @@ void LocalServer::processConfig() } global_context->setIndexMarkCache(index_mark_cache_policy, index_mark_cache_size, index_mark_cache_size_ratio); + String primary_index_cache_policy = server_settings[ServerSetting::primary_index_cache_policy]; + size_t primary_index_cache_size = server_settings[ServerSetting::primary_index_cache_size]; + double primary_index_cache_size_ratio = server_settings[ServerSetting::primary_index_cache_size_ratio]; + if (primary_index_cache_size > max_cache_size) + { + primary_index_cache_size = max_cache_size; + LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + } + global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); + size_t mmap_cache_size = server_settings[ServerSetting::mmap_cache_size]; if (mmap_cache_size > max_cache_size) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 68f262079ff..0b76adebb41 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -279,6 +279,9 @@ namespace ServerSetting extern const ServerSettingsString uncompressed_cache_policy; extern const ServerSettingsUInt64 uncompressed_cache_size; extern const ServerSettingsDouble uncompressed_cache_size_ratio; + extern const ServerSettingsString primary_index_cache_policy; + extern const ServerSettingsUInt64 primary_index_cache_size; + extern const ServerSettingsDouble primary_index_cache_size_ratio; extern const ServerSettingsBool use_legacy_mongodb_integration; } @@ -1562,6 +1565,16 @@ try } global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio); + String primary_index_cache_policy = server_settings[ServerSetting::primary_index_cache_policy]; + size_t primary_index_cache_size = server_settings[ServerSetting::primary_index_cache_size]; + double primary_index_cache_size_ratio = server_settings[ServerSetting::primary_index_cache_size_ratio]; + if (primary_index_cache_size > max_cache_size) + { + primary_index_cache_size = max_cache_size; + LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + } + global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); + size_t page_cache_size = server_settings[ServerSetting::page_cache_size]; if (page_cache_size != 0) global_context->setPageCache( @@ -1895,6 +1908,7 @@ try global_context->updateUncompressedCacheConfiguration(*config); global_context->updateMarkCacheConfiguration(*config); + global_context->updatePrimaryIndexCacheConfiguration(*config); global_context->updateIndexUncompressedCacheConfiguration(*config); global_context->updateIndexMarkCacheConfiguration(*config); global_context->updateMMappedFileCacheConfiguration(*config); diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index ec543104167..1027567a753 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -165,6 +165,8 @@ enum class AccessType : uint8_t M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_PREWARM_MARK_CACHE, "SYSTEM PREWARM MARK, PREWARM MARK CACHE, PREWARM MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MARK_CACHE, "SYSTEM DROP MARK, DROP MARK CACHE, DROP MARKS", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_PREWARM_PRIMARY_INDEX_CACHE, "SYSTEM PREWARM PRIMARY INDEX, PREWARM PRIMARY INDEX CACHE, PREWARM PRIMARY INDEX", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_PRIMARY_INDEX_CACHE, "SYSTEM DROP PRIMARY INDEX, DROP PRIMARY INDEX CACHE, DROP PRIMARY INDEX", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_UNCOMPRESSED_CACHE, "SYSTEM DROP UNCOMPRESSED, DROP UNCOMPRESSED CACHE, DROP UNCOMPRESSED", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_MMAP_CACHE, "SYSTEM DROP MMAP, DROP MMAP CACHE, DROP MMAP", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_QUERY_CACHE, "SYSTEM DROP QUERY, DROP QUERY CACHE, DROP QUERY", GLOBAL, SYSTEM_DROP_CACHE) \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 7b9f670d340..2eeb7bb6e3b 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -63,6 +63,8 @@ M(TableFunctionExecute, "Number of table function calls.", ValueType::Number) \ M(MarkCacheHits, "Number of times an entry has been found in the mark cache, so we didn't have to load a mark file.", ValueType::Number) \ M(MarkCacheMisses, "Number of times an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.", ValueType::Number) \ + M(PrimaryIndexCacheHits, "Number of times an entry has been found in the primary index cache, so we didn't have to load a index file.", ValueType::Number) \ + M(PrimaryIndexCacheMisses, "Number of times an entry has not been found in the primary index cache, so we had to load a index file in memory, which is a costly operation, adding to query latency.", ValueType::Number) \ M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided). Only updated for SELECT queries with SETTING use_query_cache = 1.", ValueType::Number) \ M(QueryCacheMisses, "Number of times a query result has not been found in the query cache (and required query computation). Only updated for SELECT queries with SETTING use_query_cache = 1.", ValueType::Number) \ /* Each page cache chunk access increments exactly one of the following 5 PageCacheChunk* counters. */ \ @@ -229,6 +231,9 @@ M(BackgroundLoadingMarksTasks, "Number of background tasks for loading marks", ValueType::Number) \ M(LoadedMarksCount, "Number of marks loaded (total across columns).", ValueType::Number) \ M(LoadedMarksMemoryBytes, "Size of in-memory representations of loaded marks.", ValueType::Bytes) \ + M(LoadedPrimaryIndexFiles, "Number of primary index files loaded.", ValueType::Number) \ + M(LoadedPrimaryIndexRows, "Number of rows of primary key loaded.", ValueType::Number) \ + M(LoadedPrimaryIndexBytes, "Number of rows of primary key loaded.", ValueType::Bytes) \ \ M(Merge, "Number of launched background merges.", ValueType::Number) \ M(MergeSourceParts, "Number of source parts scheduled for merges.", ValueType::Number) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 2f8e7b6843a..78a3ad3c419 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -101,6 +101,10 @@ namespace DB DECLARE(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ DECLARE(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \ DECLARE(Double, mark_cache_prewarm_ratio, 0.95, "The ratio of total size of mark cache to fill during prewarm.", 0) \ + DECLARE(String, primary_index_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Primary index cache policy name.", 0) \ + DECLARE(UInt64, primary_index_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for primary index (index of MergeTree family of tables).", 0) \ + DECLARE(Double, primary_index_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the primary index cache relative to the cache's total size.", 0) \ + DECLARE(Double, primary_index_cache_prewarm_ratio, 0.95, "The ratio of total size of mark cache to fill during prewarm.", 0) \ DECLARE(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \ DECLARE(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \ DECLARE(Double, index_uncompressed_cache_size_ratio, DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index uncompressed cache relative to the cache's total size.", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d2aad0a52d8..b923c1aaf82 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -406,6 +407,7 @@ struct ContextSharedPart : boost::noncopyable mutable ResourceManagerPtr resource_manager; mutable UncompressedCachePtr uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files. + mutable PrimaryIndexCachePtr primary_index_cache TSA_GUARDED_BY(mutex); mutable OnceFlag load_marks_threadpool_initialized; mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable OnceFlag prefetch_threadpool_initialized; @@ -3234,6 +3236,41 @@ ThreadPool & Context::getLoadMarksThreadpool() const return *shared->load_marks_threadpool; } +void Context::setPrimaryIndexCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio) +{ + std::lock_guard lock(shared->mutex); + + if (shared->primary_index_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Primary index cache has been already created."); + + shared->primary_index_cache = std::make_shared(cache_policy, max_cache_size_in_bytes, size_ratio); +} + +void Context::updatePrimaryIndexCacheConfiguration(const Poco::Util::AbstractConfiguration & config) +{ + std::lock_guard lock(shared->mutex); + + if (!shared->primary_index_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache was not created yet."); + + size_t max_size_in_bytes = config.getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + shared->primary_index_cache->setMaxSizeInBytes(max_size_in_bytes); +} + +PrimaryIndexCachePtr Context::getPrimaryIndexCache() const +{ + SharedLockGuard lock(shared->mutex); + return shared->primary_index_cache; +} + +void Context::clearPrimaryIndexCache() const +{ + std::lock_guard lock(shared->mutex); + + if (shared->primary_index_cache) + shared->primary_index_cache->clear(); +} + void Context::setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) { std::lock_guard lock(shared->mutex); @@ -3389,6 +3426,10 @@ void Context::clearCaches() const throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache was not created yet."); shared->mark_cache->clear(); + if (!shared->primary_index_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Primary index cache was not created yet."); + shared->primary_index_cache->clear(); + if (!shared->index_uncompressed_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index uncompressed cache was not created yet."); shared->index_uncompressed_cache->clear(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e8ccc31f597..e4db880da6a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -22,6 +22,7 @@ #include #include +#include "Storages/MergeTree/PrimaryIndexCache.h" #include "config.h" #include @@ -1075,6 +1076,11 @@ public: void clearMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; + void setPrimaryIndexCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio); + void updatePrimaryIndexCacheConfiguration(const Poco::Util::AbstractConfiguration & config); + std::shared_ptr getPrimaryIndexCache() const; + void clearPrimaryIndexCache() const; + void setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio); void updateIndexUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getIndexUncompressedCache() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b651bfb245e..41b677efe45 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -365,13 +365,22 @@ BlockIO InterpreterSystemQuery::execute() } case Type::PREWARM_MARK_CACHE: { - prewarmMarkCache(); + prewarmCaches(getContext()->getMarkCache(), nullptr); + break; + } + case Type::PREWARM_PRIMARY_INDEX_CACHE: + { + prewarmCaches(nullptr, getContext()->getPrimaryIndexCache()); break; } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); system_context->clearMarkCache(); break; + case Type::DROP_PRIMARY_INDEX_CACHE: + getContext()->checkAccess(AccessType::SYSTEM_DROP_PRIMARY_INDEX_CACHE); + system_context->clearPrimaryIndexCache(); + break; case Type::DROP_UNCOMPRESSED_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); system_context->clearUncompressedCache(); @@ -1307,18 +1316,25 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks() return tasks; } -void InterpreterSystemQuery::prewarmMarkCache() +void InterpreterSystemQuery::prewarmCaches(MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache) { - if (table_id.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM MARK CACHE command"); + if (!mark_cache && !index_cache) + return; - getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); + if (table_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM CACHE command"); + + if (mark_cache) + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); + + if (index_cache) + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_PRIMARY_INDEX_CACHE, table_id); auto table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); auto * merge_tree = dynamic_cast(table_ptr.get()); if (!merge_tree) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM MARK CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); ThreadPool pool( CurrentMetrics::MergeTreePartsLoaderThreads, @@ -1326,7 +1342,7 @@ void InterpreterSystemQuery::prewarmMarkCache() CurrentMetrics::MergeTreePartsLoaderThreadsScheduled, getContext()->getSettingsRef()[Setting::max_threads]); - merge_tree->prewarmMarkCache(pool); + merge_tree->prewarmCaches(pool, std::move(mark_cache), std::move(index_cache)); } @@ -1348,6 +1364,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_DNS_CACHE: case Type::DROP_CONNECTIONS_CACHE: case Type::DROP_MARK_CACHE: + case Type::DROP_PRIMARY_INDEX_CACHE: case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: case Type::DROP_COMPILED_EXPRESSION_CACHE: @@ -1535,6 +1552,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_PREWARM_MARK_CACHE, query.getDatabase(), query.getTable()); break; } + case Type::PREWARM_PRIMARY_INDEX_CACHE: + { + required_access.emplace_back(AccessType::SYSTEM_PREWARM_MARK_CACHE, query.getDatabase(), query.getTable()); + break; + } case Type::SYNC_DATABASE_REPLICA: { required_access.emplace_back(AccessType::SYSTEM_SYNC_DATABASE_REPLICA, query.getDatabase()); diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index e31c6cd739b..6ae84fed672 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -6,6 +6,8 @@ #include #include #include +#include "Storages/MarkCache.h" +#include "Storages/MergeTree/PrimaryIndexCache.h" #include @@ -82,7 +84,7 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); - void prewarmMarkCache(); + void prewarmCaches(MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache); void stopReplicatedDDLQueries(); void startReplicatedDDLQueries(); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 46a811822c2..dfaebec03a8 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -83,6 +83,12 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" }; } + if (auto primary_index_cache = getContext()->getPrimaryIndexCache()) + { + new_values["PrimaryIndexCacheBytes"] = { primary_index_cache->sizeInBytes(), "Total size of primary index cache in bytes" }; + new_values["PrimaryIndexCacheFiles"] = { primary_index_cache->count(), "Total number of index files cached in the primary index cache" }; + } + if (auto page_cache = getContext()->getPageCache()) { auto rss = page_cache->getResidentSetSize(); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index d76d33ce708..ec908b02e7b 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -192,6 +192,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::WAIT_LOADING_PARTS: case Type::FLUSH_DISTRIBUTED: case Type::PREWARM_MARK_CACHE: + case Type::PREWARM_PRIMARY_INDEX_CACHE: { if (table) { @@ -408,6 +409,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::DROP_MMAP_CACHE: case Type::DROP_QUERY_CACHE: case Type::DROP_MARK_CACHE: + case Type::DROP_PRIMARY_INDEX_CACHE: case Type::DROP_INDEX_MARK_CACHE: case Type::DROP_UNCOMPRESSED_CACHE: case Type::DROP_INDEX_UNCOMPRESSED_CACHE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index d9ee4d8aa22..78852cd9fac 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -24,7 +24,9 @@ public: DROP_DNS_CACHE, DROP_CONNECTIONS_CACHE, PREWARM_MARK_CACHE, + PREWARM_PRIMARY_INDEX_CACHE, DROP_MARK_CACHE, + DROP_PRIMARY_INDEX_CACHE, DROP_UNCOMPRESSED_CACHE, DROP_INDEX_MARK_CACHE, DROP_INDEX_UNCOMPRESSED_CACHE, diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 453ae0b5032..c1f33dc74a3 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -277,6 +277,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::SYNC_REPLICA: case Type::WAIT_LOADING_PARTS: case Type::PREWARM_MARK_CACHE: + case Type::PREWARM_PRIMARY_INDEX_CACHE: { if (!parseQueryWithOnCluster(res, pos, expected)) return false; diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 57fd41e2a32..92b2a0b05a9 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -205,7 +205,7 @@ public: } private: const RangesInDataParts & parts; - std::vector indices; + std::vector indices; size_t loaded_columns = std::numeric_limits::max(); }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ea01a0ed0f9..f955d649f6d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -31,6 +32,7 @@ #include #include #include +#include "Common/Logger.h" #include #include #include @@ -57,6 +59,13 @@ namespace CurrentMetrics extern const Metric PartsCompact; } +namespace ProfileEvents +{ + extern const Event LoadedPrimaryIndexFiles; + extern const Event LoadedPrimaryIndexRows; + extern const Event LoadedPrimaryIndexBytes; +} + namespace DB { @@ -351,7 +360,6 @@ IMergeTreeDataPart::IMergeTreeDataPart( incrementStateMetric(state); incrementTypeMetric(part_type); - index = std::make_shared(); minmax_idx = std::make_shared(); initializeIndexGranularityInfo(); @@ -364,46 +372,62 @@ IMergeTreeDataPart::~IMergeTreeDataPart() decrementTypeMetric(part_type); } - -IMergeTreeDataPart::Index IMergeTreeDataPart::getIndex() const +IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::getIndex() const { std::scoped_lock lock(index_mutex); - if (!index_loaded) - loadIndex(); - index_loaded = true; + + if (index) + return index; + + if (auto index_cache = storage.getPrimaryIndexCache()) + return loadIndexToCache(*index_cache); + + index = loadIndex(); return index; } - -void IMergeTreeDataPart::setIndex(const Columns & cols_) +IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::loadIndexToCache(PrimaryIndexCache & index_cache) const { - std::scoped_lock lock(index_mutex); - if (!index->empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "The index of data part can be set only once"); - index = std::make_shared(cols_); - index_loaded = true; + LOG_DEBUG(getLogger("KEK"), "part name: {}, load index path: {}", name, getDataPartStorage().getFullPath()); + auto key = PrimaryIndexCache::hash(getDataPartStorage().getFullPath()); + auto callback = [this] { return loadIndex(); }; + return index_cache.getOrSet(key, callback); } -void IMergeTreeDataPart::setIndex(Columns && cols_) +void IMergeTreeDataPart::moveIndexToCache(PrimaryIndexCache & index_cache) { std::scoped_lock lock(index_mutex); - if (!index->empty()) + if (!index) + return; + + auto key = PrimaryIndexCache::hash(getDataPartStorage().getFullPath()); + index_cache.set(key, std::const_pointer_cast(index)); + index.reset(); + + for (const auto & [_, projection] : projection_parts) + projection->moveIndexToCache(index_cache); +} + +void IMergeTreeDataPart::setIndex(Columns index_columns) +{ + std::scoped_lock lock(index_mutex); + if (index) throw Exception(ErrorCodes::LOGICAL_ERROR, "The index of data part can be set only once"); - index = std::make_shared(std::move(cols_)); - index_loaded = true; + + optimizeIndexColumns(index_granularity.getMarksCount(), index_columns); + index = std::make_shared(std::move(index_columns)); } void IMergeTreeDataPart::unloadIndex() { std::scoped_lock lock(index_mutex); - index = std::make_shared(); - index_loaded = false; + index.reset(); } bool IMergeTreeDataPart::isIndexLoaded() const { std::scoped_lock lock(index_mutex); - return index_loaded; + return index != nullptr; } void IMergeTreeDataPart::setName(const String & new_name) @@ -609,8 +633,11 @@ void IMergeTreeDataPart::removeIfNeeded() UInt64 IMergeTreeDataPart::getIndexSizeInBytes() const { std::scoped_lock lock(index_mutex); + if (!index) + return 0; + UInt64 res = 0; - for (const ColumnPtr & column : *index) + for (const auto & column : *index) res += column->byteSize(); return res; } @@ -618,8 +645,11 @@ UInt64 IMergeTreeDataPart::getIndexSizeInBytes() const UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const { std::scoped_lock lock(index_mutex); + if (!index) + return 0; + UInt64 res = 0; - for (const ColumnPtr & column : *index) + for (const auto & column : *index) res += column->allocatedBytes(); return res; } @@ -923,7 +953,36 @@ void IMergeTreeDataPart::appendFilesOfIndexGranularity(Strings & /* files */) co { } -void IMergeTreeDataPart::loadIndex() const +template +void IMergeTreeDataPart::optimizeIndexColumns(size_t marks_count, Columns & index_columns) const +{ + size_t key_size = index_columns.size(); + Float64 ratio_to_drop_suffix_columns = (*storage.getSettings())[MergeTreeSetting::primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns]; + + /// Cut useless suffix columns, if necessary. + if (key_size > 1 && ratio_to_drop_suffix_columns > 0 && ratio_to_drop_suffix_columns < 1) + { + chassert(marks_count > 0); + for (size_t j = 0; j < key_size - 1; ++j) + { + size_t num_changes = 0; + for (size_t i = 1; i < marks_count; ++i) + { + if (0 != index_columns[j]->compareAt(i, i - 1, *index_columns[j], 0)) + ++num_changes; + } + + if (static_cast(num_changes) / marks_count >= ratio_to_drop_suffix_columns) + { + key_size = j + 1; + index_columns.resize(key_size); + break; + } + } + } +} + +std::shared_ptr IMergeTreeDataPart::loadIndex() const { /// Memory for index must not be accounted as memory usage for query, because it belongs to a table. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; @@ -931,68 +990,59 @@ void IMergeTreeDataPart::loadIndex() const auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (parent_part) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; + const auto & primary_key = metadata_snapshot->getPrimaryKey(); size_t key_size = primary_key.column_names.size(); - if (key_size) + if (!key_size) + return std::make_shared(); + + MutableColumns loaded_index; + loaded_index.resize(key_size); + + for (size_t i = 0; i < key_size; ++i) { - MutableColumns loaded_index; - loaded_index.resize(key_size); - - for (size_t i = 0; i < key_size; ++i) - { - loaded_index[i] = primary_key.data_types[i]->createColumn(); - loaded_index[i]->reserve(index_granularity.getMarksCount()); - } - - String index_name = "primary" + getIndexExtensionFromFilesystem(getDataPartStorage()); - String index_path = fs::path(getDataPartStorage().getRelativePath()) / index_name; - auto index_file = metadata_manager->read(index_name); - size_t marks_count = index_granularity.getMarksCount(); - - Serializations key_serializations(key_size); - for (size_t j = 0; j < key_size; ++j) - key_serializations[j] = primary_key.data_types[j]->getDefaultSerialization(); - - for (size_t i = 0; i < marks_count; ++i) - for (size_t j = 0; j < key_size; ++j) - key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file, {}); - - /// Cut useless suffix columns, if necessary. - Float64 ratio_to_drop_suffix_columns = (*storage.getSettings())[MergeTreeSetting::primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns]; - if (key_size > 1 && ratio_to_drop_suffix_columns > 0 && ratio_to_drop_suffix_columns < 1) - { - chassert(marks_count > 0); - for (size_t j = 0; j < key_size - 1; ++j) - { - size_t num_changes = 0; - for (size_t i = 1; i < marks_count; ++i) - if (0 != loaded_index[j]->compareAt(i, i - 1, *loaded_index[j], 0)) - ++num_changes; - - if (static_cast(num_changes) / marks_count >= ratio_to_drop_suffix_columns) - { - key_size = j + 1; - loaded_index.resize(key_size); - break; - } - } - } - - for (size_t i = 0; i < key_size; ++i) - { - loaded_index[i]->shrinkToFit(); - loaded_index[i]->protect(); - if (loaded_index[i]->size() != marks_count) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data from index file {}(expected size: " - "{}, read: {})", index_path, marks_count, loaded_index[i]->size()); - } - - if (!index_file->eof()) - throw Exception(ErrorCodes::EXPECTED_END_OF_FILE, "Index file {} is unexpectedly long", index_path); - - index = std::make_shared(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); + loaded_index[i] = primary_key.data_types[i]->createColumn(); + loaded_index[i]->reserve(index_granularity.getMarksCount()); } + + String index_name = "primary" + getIndexExtensionFromFilesystem(getDataPartStorage()); + String index_path = fs::path(getDataPartStorage().getRelativePath()) / index_name; + auto index_file = metadata_manager->read(index_name); + size_t marks_count = index_granularity.getMarksCount(); + + Serializations key_serializations(key_size); + for (size_t j = 0; j < key_size; ++j) + key_serializations[j] = primary_key.data_types[j]->getDefaultSerialization(); + + for (size_t i = 0; i < marks_count; ++i) + { + for (size_t j = 0; j < key_size; ++j) + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file, {}); + } + + optimizeIndexColumns(marks_count, loaded_index); + size_t total_bytes = 0; + + for (size_t i = 0; i < key_size; ++i) + { + loaded_index[i]->shrinkToFit(); + loaded_index[i]->protect(); + total_bytes += loaded_index[i]->byteSize(); + + if (loaded_index[i]->size() != marks_count) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data from index file {}(expected size: " + "{}, read: {})", index_path, marks_count, loaded_index[i]->size()); + } + + if (!index_file->eof()) + throw Exception(ErrorCodes::EXPECTED_END_OF_FILE, "Index file {} is unexpectedly long", index_path); + + ProfileEvents::increment(ProfileEvents::LoadedPrimaryIndexFiles); + ProfileEvents::increment(ProfileEvents::LoadedPrimaryIndexRows, marks_count); + ProfileEvents::increment(ProfileEvents::LoadedPrimaryIndexBytes, total_bytes); + + return std::make_shared(std::make_move_iterator(loaded_index.begin()), std::make_move_iterator(loaded_index.end())); } void IMergeTreeDataPart::appendFilesOfIndex(Strings & files) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 24625edf154..3e70f9940f8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -25,6 +25,7 @@ #include #include #include +#include "Storages/MergeTree/PrimaryIndexCache.h" namespace zkutil @@ -77,7 +78,8 @@ public: using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; - using Index = std::shared_ptr; + using Index = Columns; + using IndexPtr = std::shared_ptr; using IndexSizeByName = std::unordered_map; using Type = MergeTreeDataPartType; @@ -371,9 +373,11 @@ public: /// Version of part metadata (columns, pk and so on). Managed properly only for replicated merge tree. int32_t metadata_version; - Index getIndex() const; - void setIndex(const Columns & cols_); - void setIndex(Columns && cols_); + IndexPtr getIndex() const; + IndexPtr loadIndexToCache(PrimaryIndexCache & index_cache) const; + void moveIndexToCache(PrimaryIndexCache & index_cache); + + void setIndex(Columns index_columns); void unloadIndex(); bool isIndexLoaded() const; @@ -598,8 +602,7 @@ protected: /// Lazily loaded in RAM. Contains each index_granularity-th value of primary key tuple. /// Note that marks (also correspond to primary key) are not always in RAM, but cached. See MarkCache.h. mutable std::mutex index_mutex; - mutable Index index TSA_GUARDED_BY(index_mutex); - mutable bool index_loaded TSA_GUARDED_BY(index_mutex) = false; + mutable IndexPtr index; /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; @@ -694,7 +697,11 @@ private: virtual void appendFilesOfIndexGranularity(Strings & files) const; /// Loads the index file. - void loadIndex() const TSA_REQUIRES(index_mutex); + std::shared_ptr loadIndex() const; + + /// Optimize index. Drop useless columns from suffix of primary key. + template + void optimizeIndexColumns(size_t marks_count, Columns & index_columns) const; void appendFilesOfIndex(Strings & files) const; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index dbfdbbdea88..e66b44aa2d7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -72,8 +73,11 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( { } -Columns IMergeTreeDataPartWriter::releaseIndexColumns() +std::optional IMergeTreeDataPartWriter::releaseIndexColumns() { + if (!settings.save_primary_index_in_memory) + return {}; + /// The memory for index was allocated without thread memory tracker. /// We need to deallocate it in shrinkToFit without memory tracker as well. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 8923f6a59ca..3bf488f2c04 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -48,7 +48,7 @@ public: virtual size_t getNumberOfOpenStreams() const = 0; - Columns releaseIndexColumns(); + std::optional releaseIndexColumns(); PlainMarksByName releaseCachedMarks(); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 7dd6d720170..84aec1ea30a 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index d7e807c689f..fa03b3f63fb 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -445,8 +445,13 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); - if (auto * mark_cache = storage.getContext()->getMarkCache().get()) - addMarksToCache(*part, cached_marks, mark_cache); + if (auto mark_cache = storage.getMarkCacheToPrewarm()) + addMarksToCache(*part, cached_marks, mark_cache.get()); + + /// Move index to cache and reset it here because we need + /// a correct part name after rename for a key of cache entry. + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + part->moveIndexToCache(*index_cache); write_part_log({}); StorageReplicatedMergeTree::incrementMergedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 6aca58faf47..d8b84fbf2e5 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -152,10 +152,17 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); - if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + if (auto mark_cache = storage.getMarkCacheToPrewarm()) { auto marks = merge_task->releaseCachedMarks(); - addMarksToCache(*new_part, marks, mark_cache); + addMarksToCache(*new_part, marks, mark_cache.get()); + } + + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + { + /// Move index to cache and reset it here because we need + /// a correct part name after rename for a key of cache entry. + new_part->moveIndexToCache(*index_cache); } write_part_log({}); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 08066113375..59169589730 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -93,7 +93,6 @@ namespace MergeTreeSetting extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_columns_to_activate; extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_rows_to_activate; extern const MergeTreeSettingsBool vertical_merge_remote_filesystem_prefetch; - extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -547,8 +546,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const } } - bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); - global_ctx->to = std::make_shared( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -558,7 +555,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const ctx->compression_codec, global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, - save_marks_in_cache, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); @@ -1089,8 +1085,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->executor = std::make_unique(ctx->column_parts_pipeline); NamesAndTypesList columns_list = {*ctx->it_name_and_type}; - bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); - ctx->column_to = std::make_unique( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -1099,7 +1093,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, - save_marks_in_cache, global_ctx->to->getIndexGranularity()); ctx->column_elems_written = 0; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 53792165987..b863c5c65cc 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -5,6 +5,8 @@ #include #include +#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include "Storages/MergeTree/PrimaryIndexCache.h" #include #include @@ -217,7 +219,9 @@ private: std::promise promise{}; IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; + PlainMarksByName cached_marks; + std::unique_ptr cached_index; MergeTreeTransactionPtr txn; bool need_prefix; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b2f35d0a309..8738eea1100 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -231,12 +231,15 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString storage_policy; extern const MergeTreeSettingsFloat zero_copy_concurrent_part_removal_max_postpone_ratio; extern const MergeTreeSettingsUInt64 zero_copy_concurrent_part_removal_max_split_times; + extern const MergeTreeSettingsBool use_primary_index_cache; + extern const MergeTreeSettingsBool prewarm_primary_index_cache; extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ServerSetting { extern const ServerSettingsDouble mark_cache_prewarm_ratio; + extern const ServerSettingsDouble primary_index_cache_prewarm_ratio; } namespace ErrorCodes @@ -2343,32 +2346,49 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() } } -void MergeTreeData::prewarmMarkCacheIfNeeded(ThreadPool & pool) +PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCache() const { - if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) - return; + if (!(*getSettings())[MergeTreeSetting::use_primary_index_cache]) + return nullptr; - prewarmMarkCache(pool); + return getContext()->getPrimaryIndexCache(); } -void MergeTreeData::prewarmMarkCache(ThreadPool & pool) +PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCacheToPrewarm() const { - auto * mark_cache = getContext()->getMarkCache().get(); - if (!mark_cache) + if (!(*getSettings())[MergeTreeSetting::prewarm_primary_index_cache]) + return nullptr; + + return getPrimaryIndexCache(); +} + +MarkCachePtr MergeTreeData::getMarkCacheToPrewarm() const +{ + if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) + return nullptr; + + return getContext()->getMarkCache(); +} + +void MergeTreeData::prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache) +{ + if (!mark_cache && !index_cache) return; - auto metadata_snaphost = getInMemoryMetadataPtr(); - auto column_names = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); + Names columns_to_prewarm_marks; - if (column_names.empty()) - return; + if (mark_cache) + { + auto metadata_snaphost = getInMemoryMetadataPtr(); + columns_to_prewarm_marks = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); + } Stopwatch watch; - LOG_TRACE(log, "Prewarming mark cache"); + LOG_TRACE(log, "Prewarming mark and/or primary index caches"); auto data_parts = getDataPartsVectorForInternalUsage(); - /// Prewarm mark cache firstly for the most fresh parts according + /// Prewarm caches firstly for the most fresh parts according /// to time columns in partition key (if exists) and by modification time. auto to_tuple = [](const auto & part) @@ -2381,20 +2401,22 @@ void MergeTreeData::prewarmMarkCache(ThreadPool & pool) return to_tuple(lhs) > to_tuple(rhs); }); - ThreadPoolCallbackRunnerLocal runner(pool, "PrewarmMarks"); - double ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio]; + ThreadPoolCallbackRunnerLocal runner(pool, "PrewarmCaches"); + + double marks_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio]; + double index_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::primary_index_cache_prewarm_ratio]; for (const auto & part : data_parts) { - if (mark_cache->sizeInBytes() >= mark_cache->maxSizeInBytes() * ratio_to_prewarm) - break; + if (index_cache && index_cache->sizeInBytes() < index_cache->maxSizeInBytes() * index_ratio_to_prewarm) + runner([&] { part->loadIndexToCache(*index_cache); }); - runner([&] { part->loadMarksToCache(column_names, mark_cache); }); + if (mark_cache && mark_cache->sizeInBytes() < mark_cache->maxSizeInBytes() * marks_ratio_to_prewarm) + runner([&] { part->loadMarksToCache(columns_to_prewarm_marks, mark_cache.get()); }); } runner.waitForAllToFinishAndRethrowFirstError(); - watch.stop(); - LOG_TRACE(log, "Prewarmed mark cache in {} seconds", watch.elapsedSeconds()); + LOG_TRACE(log, "Prewarmed mark and/or primary index caches in {} seconds", watch.elapsedSeconds()); } /// Is the part directory old. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fe360907875..58a909e6a2d 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -31,6 +31,8 @@ #include #include #include +#include +#include #include #include #include @@ -506,9 +508,15 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks, std::optional> expected_parts); + /// Returns a pointer to primary index cache if it is enabled. + PrimaryIndexCachePtr getPrimaryIndexCache() const; + /// Returns a pointer to primary index cache if it is enabled and required to be prewarmed. + PrimaryIndexCachePtr getPrimaryIndexCacheToPrewarm() const; + /// Returns a pointer to primary mark cache if it is required to be prewarmed. + MarkCachePtr getMarkCacheToPrewarm() const; + /// Prewarm mark cache for the most recent data parts. - void prewarmMarkCache(ThreadPool & pool); - void prewarmMarkCacheIfNeeded(ThreadPool & pool); + void prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache); String getLogName() const { return log.loadName(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index c483d47fed7..893afcc7b0e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -327,9 +327,10 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndexRow(const B for (size_t i = 0; i < index_block.columns(); ++i) { const auto & column = index_block.getByPosition(i).column; - - index_columns[i]->insertFrom(*column, row); index_serializations[i]->serializeBinary(*column, row, index_stream, {}); + + if (settings.save_primary_index_in_memory) + index_columns[i]->insertFrom(*column, row); } } @@ -347,8 +348,14 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc */ MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; - if (index_columns.empty()) - index_columns = primary_index_block.cloneEmptyColumns(); + if (settings.save_primary_index_in_memory) + { + if (index_columns.empty()) + index_columns = primary_index_block.cloneEmptyColumns(); + + for (const auto & column : index_columns) + column->reserve(column->size() + granules_to_write.size()); + } /// Write index. The index contains Primary Key value for each `index_granularity` row. for (const auto & granule : granules_to_write) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c4ca545ca90..ebc73a2e90f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1074,7 +1074,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( DataTypes key_types; if (!key_indices.empty()) { - const auto & index = part->getIndex(); + const auto index = part->getIndex(); for (size_t i : key_indices) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 6d19f45e2c4..f9a6c7de034 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -73,7 +73,6 @@ namespace MergeTreeSetting extern const MergeTreeSettingsFloat min_free_disk_ratio_to_perform_insert; extern const MergeTreeSettingsBool optimize_row_order; extern const MergeTreeSettingsFloat ratio_of_defaults_for_sparse_serialization; - extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -685,7 +684,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - bool save_marks_in_cache = (*data_settings)[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -696,7 +694,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( compression_codec, context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, /*reset_columns=*/ false, - save_marks_in_cache, /*blocks_are_granules_size=*/ false, context->getWriteSettings()); @@ -832,7 +829,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - bool save_marks_in_cache = (*data.getSettings())[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -844,7 +840,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( compression_codec, Tx::PrehistoricTID, /*reset_columns=*/ false, - save_marks_in_cache, /*blocks_are_granules_size=*/ false, data.getContext()->getWriteSettings()); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.cpp b/src/Storages/MergeTree/MergeTreeIOSettings.cpp index bacfbbd5720..67f2ca31be7 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeIOSettings.cpp @@ -35,6 +35,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( bool can_use_adaptive_granularity_, bool rewrite_primary_key_, bool save_marks_in_cache_, + bool save_primary_index_in_memory_, bool blocks_are_granules_size_) : min_compress_block_size( (*storage_settings)[MergeTreeSetting::min_compress_block_size] ? (*storage_settings)[MergeTreeSetting::min_compress_block_size] : global_settings[Setting::min_compress_block_size]) @@ -48,6 +49,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , can_use_adaptive_granularity(can_use_adaptive_granularity_) , rewrite_primary_key(rewrite_primary_key_) , save_marks_in_cache(save_marks_in_cache_) + , save_primary_index_in_memory(save_primary_index_in_memory_) , blocks_are_granules_size(blocks_are_granules_size_) , query_write_settings(query_write_settings_) , low_cardinality_max_dictionary_size(global_settings[Setting::low_cardinality_max_dictionary_size]) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 7506c726bc4..fdcab59724d 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -64,6 +64,7 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity_, bool rewrite_primary_key_, bool save_marks_in_cache_, + bool save_primary_index_in_memory_, bool blocks_are_granules_size_); size_t min_compress_block_size; @@ -79,6 +80,7 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity; bool rewrite_primary_key; bool save_marks_in_cache; + bool save_primary_index_in_memory; bool blocks_are_granules_size; WriteSettings query_write_settings; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index fcd4e05cf00..eea03be20dc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -236,6 +236,8 @@ namespace ErrorCodes DECLARE(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ DECLARE(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ DECLARE(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ + DECLARE(Bool, use_primary_index_cache, false, "Use cache for primary index instead of saving all indexes in memory. Can be useful for very large tables", 0) \ + DECLARE(Bool, prewarm_primary_index_cache, false, "If true primary index cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(Bool, prewarm_mark_cache, false, "If true mark cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(String, columns_to_prewarm_mark_cache, "", "List of columns to prewarm mark cache for (if enabled). Empty means all columns", 0) \ /** Projection settings. */ \ diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 99852309c77..6de4fa4feef 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -247,15 +247,22 @@ void MergeTreeSink::finishDelayedChunk() /// Part can be deduplicated, so increment counters and add to part log only if it's really added if (added) { - if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + if (auto mark_cache = storage.getMarkCacheToPrewarm()) { for (const auto & stream : partition.temp_part.streams) { auto marks = stream.stream->releaseCachedMarks(); - addMarksToCache(*part, marks, mark_cache); + addMarksToCache(*part, marks, mark_cache.get()); } } + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + { + /// Move index to cache and reset it here because we need + /// a correct part name after rename for a key of cache entry. + part->moveIndexToCache(*index_cache); + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot)); StorageMergeTree::incrementInsertedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 14a521ce429..52bc40e7555 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -15,6 +16,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace MergeTreeSetting +{ + extern const MergeTreeSettingsBool use_primary_index_cache; +} MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeMutableDataPartPtr & data_part, @@ -25,7 +30,6 @@ MergedBlockOutputStream::MergedBlockOutputStream( CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_, - bool save_marks_in_cache, bool blocks_are_granules_size, const WriteSettings & write_settings_, const MergeTreeIndexGranularity & computed_index_granularity) @@ -34,6 +38,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( , default_codec(default_codec_) , write_settings(write_settings_) { + bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); + MergeTreeWriterSettings writer_settings( data_part->storage.getContext()->getSettingsRef(), write_settings, @@ -41,6 +48,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( data_part->index_granularity_info.mark_type.adaptive, /* rewrite_primary_key = */ true, save_marks_in_cache, + save_primary_index_in_memory, blocks_are_granules_size); /// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart @@ -202,7 +210,10 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); - new_part->setIndex(writer->releaseIndexColumns()); + + if (auto computed_index = writer->releaseIndexColumns()) + new_part->setIndex(std::move(*computed_index)); + new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk()); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 060778866e0..e212fe5bb5a 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -24,7 +24,6 @@ public: CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_ = false, - bool save_marks_in_cache = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}, const MergeTreeIndexGranularity & computed_index_granularity = {}); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index bed539dfe02..ef10c9c492a 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -1,16 +1,23 @@ #include #include +#include #include #include #include namespace DB { + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; } +namespace MergeTreeSetting +{ + extern const MergeTreeSettingsBool use_primary_index_cache; +} + MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeMutableDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, @@ -19,20 +26,21 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeIndices & indices_to_recalc, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_, - bool save_marks_in_cache, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { - const auto & global_settings = data_part->storage.getContext()->getSettingsRef(); + bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); MergeTreeWriterSettings writer_settings( - global_settings, + data_part->storage.getContext()->getSettingsRef(), data_part->storage.getContext()->getWriteSettings(), storage_settings, index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), /* rewrite_primary_key = */ false, save_marks_in_cache, + save_primary_index_in_memory, /* blocks_are_granules_size = */ false); writer = createMergeTreeDataPartWriter( diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index f6bf9e37a58..e837a62743e 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -22,7 +22,6 @@ public: const MergeTreeIndices & indices_to_recalc_, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, - bool save_marks_in_cache = false, const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 7f6588fc632..aea6d3d1505 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1625,7 +1625,6 @@ private: ctx->compression_codec, ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, - /*save_marks_in_cache=*/ false, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings(), computed_granularity); @@ -1854,7 +1853,6 @@ private: std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), nullptr, - /*save_marks_in_cache=*/ false, ctx->source_part->index_granularity, &ctx->source_part->index_granularity_info ); diff --git a/src/Storages/MergeTree/PrimaryIndexCache.cpp b/src/Storages/MergeTree/PrimaryIndexCache.cpp new file mode 100644 index 00000000000..aeb9969f578 --- /dev/null +++ b/src/Storages/MergeTree/PrimaryIndexCache.cpp @@ -0,0 +1,8 @@ +#include + +namespace DB +{ + +template class CacheBase; + +} diff --git a/src/Storages/MergeTree/PrimaryIndexCache.h b/src/Storages/MergeTree/PrimaryIndexCache.h new file mode 100644 index 00000000000..758f18dbed8 --- /dev/null +++ b/src/Storages/MergeTree/PrimaryIndexCache.h @@ -0,0 +1,73 @@ +#pragma once +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event PrimaryIndexCacheHits; + extern const Event PrimaryIndexCacheMisses; +} + +namespace DB +{ + +using PrimaryIndex = std::vector; + +/// Estimate of number of bytes in cache for primaryindexs. +struct PrimaryIndexWeightFunction +{ + /// We spent additional bytes on key in hashmap, linked lists, shared pointers, etc ... + static constexpr size_t PRIMARY_INDEX_CACHE_OVERHEAD = 128; + + size_t operator()(const PrimaryIndex & index) const + { + size_t res = 0; + for (const auto & column : index) + res += column->byteSize(); + return res; + } +}; + +extern template class CacheBase; + +/** Cache of primary index for StorageMergeTree. + * PrimaryIndex is an index structure that addresses ranges in column file, corresponding to ranges of primary key. + */ +class PrimaryIndexCache : public CacheBase +{ +private: + using Base = CacheBase; + +public: + PrimaryIndexCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) + : Base(cache_policy, max_size_in_bytes, 0, size_ratio) + { + } + + /// Calculate key from path to file and offset. + static UInt128 hash(const String & part_path) + { + SipHash hash; + hash.update(part_path.data(), part_path.size() + 1); + return hash.get128(); + } + + template + MappedPtr getOrSet(const Key & key, LoadFunc && load) + { + auto result = Base::getOrSet(key, load); + if (result.second) + ProfileEvents::increment(ProfileEvents::PrimaryIndexCacheMisses); + else + ProfileEvents::increment(ProfileEvents::PrimaryIndexCacheHits); + + return result.first; + } +}; + +using PrimaryIndexCachePtr = std::shared_ptr; + +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c0e25a54bf3..d884b3aca7e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -466,6 +466,28 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) ++num_blocks_processed; } +template +void ReplicatedMergeTreeSinkImpl::prewarmCaches(const MergeTreeDataWriter::TemporaryPart & temp_part) const +{ + const auto & part = temp_part.part; + + if (auto mark_cache = storage.getMarkCacheToPrewarm()) + { + for (const auto & stream : temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache.get()); + } + } + + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + { + /// Move index to cache and reset it here because we need + /// a correct part name after rename for a key of cache entry. + part->moveIndexToCache(*index_cache); + } +} + template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { @@ -486,16 +508,9 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; - auto * mark_cache = storage.getContext()->getMarkCache().get(); - if (!error && mark_cache) - { - for (const auto & stream : partition.temp_part.streams) - { - auto marks = stream.stream->releaseCachedMarks(); - addMarksToCache(*part, marks, mark_cache); - } - } + if (!error) + prewarmCaches(partition.temp_part); auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); @@ -540,14 +555,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa if (conflict_block_ids.empty()) { - if (auto * mark_cache = storage.getContext()->getMarkCache().get()) - { - for (const auto & stream : partition.temp_part.streams) - { - auto marks = stream.stream->releaseCachedMarks(); - addMarksToCache(*partition.temp_part.part, marks, mark_cache); - } - } + prewarmCaches(partition.temp_part); auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 7d025361717..b467cc167f8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -129,6 +130,7 @@ private: std::unique_ptr delayed_chunk; void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); + void prewarmCaches(const MergeTreeDataWriter::TemporaryPart & temp_part) const; }; using ReplicatedMergeTreeSinkWithAsyncDeduplicate = ReplicatedMergeTreeSinkImpl; diff --git a/src/Storages/MergeTree/new_index b/src/Storages/MergeTree/new_index new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1ba0617d8ae..81e6a22ba58 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -155,7 +155,11 @@ StorageMergeTree::StorageMergeTree( loadMutations(); loadDeduplicationLog(); - prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get()); + + prewarmCaches( + getActivePartsLoadingThreadPool().get(), + getMarkCacheToPrewarm(), + getPrimaryIndexCacheToPrewarm()); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 793fd02c656..c49e7fba7e5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -208,7 +208,6 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool use_minimalistic_checksums_in_zookeeper; extern const MergeTreeSettingsBool use_minimalistic_part_header_in_zookeeper; extern const MergeTreeSettingsMilliseconds wait_for_unique_parts_send_before_shutdown_ms; - extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace FailPoints @@ -509,7 +508,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } loadDataParts(skip_sanity_checks, expected_parts_on_this_replica); - prewarmMarkCacheIfNeeded(getActivePartsLoadingThreadPool().get()); + + prewarmCaches( + getActivePartsLoadingThreadPool().get(), + getMarkCacheToPrewarm(), + getPrimaryIndexCacheToPrewarm()); if (LoadingStrictnessLevel::ATTACH <= mode) { @@ -5082,10 +5085,10 @@ bool StorageReplicatedMergeTree::fetchPart( ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } - if ((*getSettings())[MergeTreeSetting::prewarm_mark_cache] && getContext()->getMarkCache()) + if (auto mark_cache = getMarkCacheToPrewarm()) { auto column_names = getColumnsToPrewarmMarks(*getSettings(), part->getColumns()); - part->loadMarksToCache(column_names, getContext()->getMarkCache().get()); + part->loadMarksToCache(column_names, mark_cache.get()); } write_part_log({}); diff --git a/tests/queries/0_stateless/03273_primary_index_cache.reference b/tests/queries/0_stateless/03273_primary_index_cache.reference new file mode 100644 index 00000000000..611787366ee --- /dev/null +++ b/tests/queries/0_stateless/03273_primary_index_cache.reference @@ -0,0 +1,16 @@ +0 +PrimaryIndexCacheBytes 0 +PrimaryIndexCacheFiles 0 +99 +0 +PrimaryIndexCacheBytes 1280 +PrimaryIndexCacheFiles 2 +0 +PrimaryIndexCacheBytes 0 +PrimaryIndexCacheFiles 0 +49 +0 +PrimaryIndexCacheBytes 640 +PrimaryIndexCacheFiles 1 +2 160 1280 +1 80 640 diff --git a/tests/queries/0_stateless/03273_primary_index_cache.sql b/tests/queries/0_stateless/03273_primary_index_cache.sql new file mode 100644 index 00000000000..18c2703689b --- /dev/null +++ b/tests/queries/0_stateless/03273_primary_index_cache.sql @@ -0,0 +1,45 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS t_primary_index_cache; + +CREATE TABLE t_primary_index_cache (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY a PARTITION BY a % 2 +SETTINGS use_primary_index_cache = 1, prewarm_primary_index_cache = 0, index_granularity = 64, index_granularity_bytes = '10M', min_bytes_for_wide_part = 0; + +SYSTEM DROP PRIMARY INDEX CACHE; + +INSERT INTO t_primary_index_cache SELECT number, number FROM numbers(10000); + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SELECT count() FROM t_primary_index_cache WHERE a > 100 AND a < 200; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SYSTEM DROP PRIMARY INDEX CACHE; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SELECT count() FROM t_primary_index_cache WHERE a > 100 AND a < 200 AND a % 2 = 0; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SYSTEM FLUSH LOGS; + +SELECT + ProfileEvents['LoadedPrimaryIndexFiles'], + ProfileEvents['LoadedPrimaryIndexRows'], + ProfileEvents['LoadedPrimaryIndexBytes'] +FROM system.query_log +WHERE query LIKE 'SELECT count() FROM t_primary_index_cache%' AND current_database = currentDatabase() AND type = 'QueryFinish' +ORDER BY event_time_microseconds; + +DROP TABLE t_primary_index_cache; diff --git a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference new file mode 100644 index 00000000000..ed46312d77d --- /dev/null +++ b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference @@ -0,0 +1,16 @@ +0 +PrimaryIndexCacheBytes 1280 +PrimaryIndexCacheFiles 2 +99 +0 +PrimaryIndexCacheBytes 1280 +PrimaryIndexCacheFiles 2 +0 +PrimaryIndexCacheBytes 1280 +PrimaryIndexCacheFiles 2 +49 +0 +PrimaryIndexCacheBytes 1280 +PrimaryIndexCacheFiles 2 +0 0 0 +0 0 0 diff --git a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql new file mode 100644 index 00000000000..e9e04cff6ec --- /dev/null +++ b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS t_primary_index_cache_2; + +CREATE TABLE t_primary_index_cache_2 (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY a PARTITION BY a % 2 +SETTINGS use_primary_index_cache = 1, prewarm_primary_index_cache = 1, index_granularity = 64, index_granularity_bytes = '10M', min_bytes_for_wide_part = 0; + +SYSTEM DROP PRIMARY INDEX CACHE; + +INSERT INTO t_primary_index_cache_2 SELECT number, number FROM numbers(10000); + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SELECT count() FROM t_primary_index_cache_2 WHERE a > 100 AND a < 200; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SYSTEM DROP PRIMARY INDEX CACHE; +SYSTEM PREWARM PRIMARY INDEX CACHE t_primary_index_cache_2; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SELECT count() FROM t_primary_index_cache_2 WHERE a > 100 AND a < 200 AND a % 2 = 0; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; + +SYSTEM FLUSH LOGS; + +SELECT + ProfileEvents['LoadedPrimaryIndexFiles'], + ProfileEvents['LoadedPrimaryIndexRows'], + ProfileEvents['LoadedPrimaryIndexBytes'] +FROM system.query_log +WHERE query LIKE 'SELECT count() FROM t_primary_index_cache_2%' AND current_database = currentDatabase() AND type = 'QueryFinish' +ORDER BY event_time_microseconds; + +DROP TABLE t_primary_index_cache_2; \ No newline at end of file From 574f26f21fbf5673fcac056d4205c33bacefb893 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 19 Nov 2024 23:16:47 +0000 Subject: [PATCH 117/433] PR: skip index analysis on followers --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 52ea6db787d..c47736c7c31 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -80,6 +80,7 @@ namespace Setting extern const SettingsUInt64 parallel_replica_offset; extern const SettingsUInt64 parallel_replicas_count; extern const SettingsParallelReplicasMode parallel_replicas_mode; + extern const SettingsBool parallel_replicas_local_plan; } namespace MergeTreeSetting @@ -631,10 +632,26 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd bool use_skip_indexes, bool find_exact_ranges) { - RangesInDataParts parts_with_ranges; - parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); + if (context->canUseParallelReplicasOnFollower() && settings[Setting::parallel_replicas_local_plan]) + { + RangesInDataParts parts_with_ranges; + parts_with_ranges.reserve(parts.size()); + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + { + const auto & part = parts[part_index]; + // LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "part {}", part->getNameWithState()); + + MarkRanges ranges; + ranges.emplace_back(0, part->getMarksCount()); + parts_with_ranges.emplace_back(part, part_index, std::move(ranges)); + } + return parts_with_ranges; + } + + RangesInDataParts parts_with_ranges; + parts_with_ranges.resize(parts.size()); if (use_skip_indexes && settings[Setting::force_data_skipping_indices].changed) { const auto & indices_str = settings[Setting::force_data_skipping_indices].toString(); From 4fc32ea96fb59bcc95ef6a9a471f47f3c7ace1b6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 19 Nov 2024 23:52:54 +0000 Subject: [PATCH 118/433] Polish --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c47736c7c31..36bc7588542 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -650,8 +650,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd return parts_with_ranges; } - RangesInDataParts parts_with_ranges; - parts_with_ranges.resize(parts.size()); if (use_skip_indexes && settings[Setting::force_data_skipping_indices].changed) { const auto & indices_str = settings[Setting::force_data_skipping_indices].toString(); @@ -690,6 +688,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd std::atomic sum_marks_pk = 0; std::atomic sum_parts_pk = 0; + RangesInDataParts parts_with_ranges(parts.size()); + /// Let's find what range to read from each part. { auto mark_cache = context->getIndexMarkCache(); From dc0aa941105e49c3911257802126e0651b3f3e31 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Nov 2024 11:28:35 +0000 Subject: [PATCH 119/433] fix --- src/Interpreters/HashJoin/HashJoinMethodsImpl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 105b6e5d5bf..9b92e44d64a 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -122,9 +122,6 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( block.filterBySelector(); - for (size_t i = 0; i < added_columns.size(); ++i) - source_block.insert(added_columns.moveColumn(i)); - const auto & table_join = join.table_join; std::set block_columns_to_erase; if (join.canRemoveColumnsFromLeftBlock()) @@ -139,6 +136,9 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( } } + for (size_t i = 0; i < added_columns.size(); ++i) + source_block.insert(added_columns.moveColumn(i)); + std::vector right_keys_to_replicate [[maybe_unused]]; if constexpr (join_features.need_filter) From 93e10644b740e5d0b356ae0321979e45a3a4eb51 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Nov 2024 11:37:30 +0000 Subject: [PATCH 120/433] update query_plan_join_swap_table default value to "auto" --- src/Core/Settings.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 2351e4b7b0d..17b65edd6f8 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1921,7 +1921,7 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ -DECLARE(BoolAuto, query_plan_join_swap_table, {}, R"( +DECLARE(BoolAuto, query_plan_join_swap_table, Field("auto"), R"( Determine which side of the join should be the build table (also called inner, the one inserted into the hash table for a hash join) in the query plan. This setting is supported only for `ALL` join strictness with the `JOIN ON` clause. Possible values are: - 'auto': Let the planner decide which table to use as the build table. - 'false': Never swap tables (the right table is the build table). From c6df1b09c6e806fd577a339d263acde1ce15a758 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Nov 2024 15:55:42 +0000 Subject: [PATCH 121/433] fx --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index b7355f72161..db6dd58f282 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -438,7 +438,7 @@ bool HashJoin::isUsedByAnotherAlgorithm() const bool HashJoin::canRemoveColumnsFromLeftBlock() const { - return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm(); + return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm() && strictness != JoinStrictness::RightAny; } void HashJoin::initRightBlockStructure(Block & saved_block_sample) From 27fb90bb58d0cfc5e63ed1538109742a521bee07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Nov 2024 17:37:32 +0100 Subject: [PATCH 122/433] Fix bugs when using UDF in join on expression with the old analyzer --- .../UserDefinedSQLFunctionVisitor.cpp | 16 +++++------ src/Interpreters/CrossToInnerJoinVisitor.cpp | 2 +- src/Parsers/ASTColumnDeclaration.cpp | 27 ++++++++++++++----- src/Parsers/ASTTablesInSelectQuery.cpp | 23 ++++++++++++++++ src/Parsers/ASTTablesInSelectQuery.h | 3 +++ .../0_stateless/03274_udf_in_join.reference | 7 +++++ .../queries/0_stateless/03274_udf_in_join.sh | 21 +++++++++++++++ 7 files changed, 83 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03274_udf_in_join.reference create mode 100755 tests/queries/0_stateless/03274_udf_in_join.sh diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index a04b8d7b998..57867eeb5cf 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -26,14 +26,6 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) { chassert(ast); - if (const auto * function = ast->template as()) - { - std::unordered_set udf_in_replace_process; - auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process); - if (replace_result) - ast = replace_result; - } - for (auto & child : ast->children) { if (!child) @@ -48,6 +40,14 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast) if (new_ptr != old_ptr) ast->updatePointerToChild(old_ptr, new_ptr); } + + if (const auto * function = ast->template as()) + { + std::unordered_set udf_in_replace_process; + auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process); + if (replace_result) + ast = replace_result; + } } void UserDefinedSQLFunctionVisitor::visit(IAST * ast) diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index e3e8b80e437..dbe10045c98 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -70,7 +70,7 @@ struct JoinedElement join->strictness = JoinStrictness::All; join->on_expression = on_expression; - join->children.push_back(join->on_expression); + join->children = {join->on_expression}; return true; } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 1c7d72bafcc..d96a52da61c 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -130,12 +130,25 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo void ASTColumnDeclaration::forEachPointerToChild(std::function f) { - f(reinterpret_cast(&default_expression)); - f(reinterpret_cast(&comment)); - f(reinterpret_cast(&codec)); - f(reinterpret_cast(&statistics_desc)); - f(reinterpret_cast(&ttl)); - f(reinterpret_cast(&collation)); - f(reinterpret_cast(&settings)); + auto visit_child = [&f](ASTPtr & member) + { + IAST * new_member_ptr = member.get(); + f(reinterpret_cast(&new_member_ptr)); + if (new_member_ptr != member.get()) + { + if (new_member_ptr) + member = new_member_ptr->ptr(); + else + member.reset(); + } + }; + + visit_child(default_expression); + visit_child(comment); + visit_child(codec); + visit_child(statistics_desc); + visit_child(ttl); + visit_child(collation); + visit_child(settings); } } diff --git a/src/Parsers/ASTTablesInSelectQuery.cpp b/src/Parsers/ASTTablesInSelectQuery.cpp index b6d42513aa7..0046f603c7d 100644 --- a/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/src/Parsers/ASTTablesInSelectQuery.cpp @@ -61,6 +61,29 @@ ASTPtr ASTTableJoin::clone() const return res; } +void ASTTableJoin::forEachPointerToChild(std::function f) +{ + IAST * new_using_expression_list = using_expression_list.get(); + f(reinterpret_cast(&new_using_expression_list)); + if (new_using_expression_list != using_expression_list.get()) + { + if (new_using_expression_list) + using_expression_list = new_using_expression_list->ptr(); + else + using_expression_list.reset(); + } + + IAST * new_on_expression = on_expression.get(); + f(reinterpret_cast(&new_on_expression)); + if (new_on_expression != on_expression.get()) + { + if (new_on_expression) + on_expression = new_on_expression->ptr(); + else + on_expression.reset(); + } +} + void ASTArrayJoin::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const { hash_state.update(kind); diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index f3f329ca2b6..ab32e51a1b4 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -80,6 +80,9 @@ struct ASTTableJoin : public IAST void formatImplAfterTable(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override; + +protected: + void forEachPointerToChild(std::function f) override; }; /// Specification of ARRAY JOIN. diff --git a/tests/queries/0_stateless/03274_udf_in_join.reference b/tests/queries/0_stateless/03274_udf_in_join.reference new file mode 100644 index 00000000000..c05f6ac7c8a --- /dev/null +++ b/tests/queries/0_stateless/03274_udf_in_join.reference @@ -0,0 +1,7 @@ +SELECT 1 +FROM +( + SELECT 1 AS c0 +) AS v0 +ALL INNER JOIN v0 AS vx ON c0 = vx.c0 +1 diff --git a/tests/queries/0_stateless/03274_udf_in_join.sh b/tests/queries/0_stateless/03274_udf_in_join.sh new file mode 100755 index 00000000000..052534d7a3c --- /dev/null +++ b/tests/queries/0_stateless/03274_udf_in_join.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + CREATE VIEW v0 AS SELECT 1 AS c0; + CREATE FUNCTION ${CLICKHOUSE_DATABASE}_second AS (x, y) -> y; + CREATE FUNCTION ${CLICKHOUSE_DATABASE}_equals AS (x, y) -> x = y; + -- SET optimize_rewrite_array_exists_to_has = 1; + + EXPLAIN SYNTAX SELECT 1 FROM v0 JOIN v0 vx ON ${CLICKHOUSE_DATABASE}_second(v0.c0, vx.c0); -- { serverError INVALID_JOIN_ON_EXPRESSION } + EXPLAIN SYNTAX SELECT 1 FROM v0 JOIN v0 vx ON ${CLICKHOUSE_DATABASE}_equals(v0.c0, vx.c0); + + SELECT 1 FROM v0 JOIN v0 vx ON ${CLICKHOUSE_DATABASE}_equals(v0.c0, vx.c0); + + DROP view v0; + DROP FUNCTION ${CLICKHOUSE_DATABASE}_second; + DROP FUNCTION ${CLICKHOUSE_DATABASE}_equals; +" From f6610790a84ca2ff13cb2dc69d8b6cd9f0ff1954 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 20 Nov 2024 17:44:01 +0100 Subject: [PATCH 123/433] Fix broken check --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- tests/ci/run_check.py | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 976c69d3c34..b4ca4cb8d98 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -12,7 +12,7 @@ tests/ci/cancel_and_rerun_workflow_lambda/app.py - Backward Incompatible Change - Build/Testing/Packaging Improvement - Documentation (changelog entry is not required) -- Critical Bug Fix (crash, data loss, RBAC) +- Critical Bug Fix (crash, data loss, RBAC) or LOGICAL_ERROR - Bug Fix (user-visible misbehavior in an official stable release) - CI Fix or Improvement (changelog entry is not required) - Not for changelog (changelog entry is not required) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 7f665165c59..61c3c0c4ec4 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -56,7 +56,9 @@ LABEL_CATEGORIES = { "Bug Fix (user-visible misbehaviour in official stable or prestable release)", "Bug Fix (user-visible misbehavior in official stable or prestable release)", ], - "pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"], + "pr-critical-bugfix": [ + "Critical Bug Fix (crash, data loss, RBAC) or LOGICAL_ERROR" + ], "pr-build": [ "Build/Testing/Packaging Improvement", "Build Improvement", From 7881ae22866b14862c09f1ae4bb9414c012d56d5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Nov 2024 17:18:39 +0000 Subject: [PATCH 124/433] better primary index cache --- src/Interpreters/InterpreterSystemQuery.cpp | 51 +++++++--- src/Interpreters/InterpreterSystemQuery.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/MergeTreeData.cpp | 35 +++++-- .../MergeTree/MergeTreeDataWriter.cpp | 23 ++++- src/Storages/MergeTree/MergeTreeDataWriter.h | 1 + src/Storages/MergeTree/MergeTreeSink.cpp | 16 +--- .../MergeTree/MergedBlockOutputStream.cpp | 3 +- .../MergedColumnOnlyOutputStream.cpp | 7 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 26 +---- src/Storages/StorageReplicatedMergeTree.cpp | 5 + .../01271_show_privileges.reference | 2 + ...3274_prewarm_primary_index_cache.reference | 30 +++--- .../03274_prewarm_primary_index_cache.sql | 96 ++++++++++++------- 14 files changed, 185 insertions(+), 124 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 41b677efe45..d1b6b9e1ace 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -365,12 +365,12 @@ BlockIO InterpreterSystemQuery::execute() } case Type::PREWARM_MARK_CACHE: { - prewarmCaches(getContext()->getMarkCache(), nullptr); + prewarmMarkCache(); break; } case Type::PREWARM_PRIMARY_INDEX_CACHE: { - prewarmCaches(nullptr, getContext()->getPrimaryIndexCache()); + prewarmPrimaryIndexCache(); break; } case Type::DROP_MARK_CACHE: @@ -1316,25 +1316,21 @@ RefreshTaskList InterpreterSystemQuery::getRefreshTasks() return tasks; } -void InterpreterSystemQuery::prewarmCaches(MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache) +void InterpreterSystemQuery::prewarmMarkCache() { - if (!mark_cache && !index_cache) - return; - if (table_id.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM CACHE command"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM MARK CACHE command"); - if (mark_cache) - getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); - - if (index_cache) - getContext()->checkAccess(AccessType::SYSTEM_PREWARM_PRIMARY_INDEX_CACHE, table_id); + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_MARK_CACHE, table_id); auto table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); auto * merge_tree = dynamic_cast(table_ptr.get()); - if (!merge_tree) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM MARK CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); + + auto mark_cache = getContext()->getMarkCache(); + if (!mark_cache) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mark cache is not configured"); ThreadPool pool( CurrentMetrics::MergeTreePartsLoaderThreads, @@ -1342,7 +1338,32 @@ void InterpreterSystemQuery::prewarmCaches(MarkCachePtr mark_cache, PrimaryIndex CurrentMetrics::MergeTreePartsLoaderThreadsScheduled, getContext()->getSettingsRef()[Setting::max_threads]); - merge_tree->prewarmCaches(pool, std::move(mark_cache), std::move(index_cache)); + merge_tree->prewarmCaches(pool, std::move(mark_cache), nullptr); +} + +void InterpreterSystemQuery::prewarmPrimaryIndexCache() +{ + if (table_id.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table is not specified for PREWARM PRIMARY INDEX CACHE command"); + + getContext()->checkAccess(AccessType::SYSTEM_PREWARM_PRIMARY_INDEX_CACHE, table_id); + + auto table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto * merge_tree = dynamic_cast(table_ptr.get()); + if (!merge_tree) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Command PREWARM PRIMARY INDEX CACHE is supported only for MergeTree table, but got: {}", table_ptr->getName()); + + auto index_cache = merge_tree->getPrimaryIndexCache(); + if (!index_cache) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary index cache is not configured or is not enabled for table {}", table_id.getFullTableName()); + + ThreadPool pool( + CurrentMetrics::MergeTreePartsLoaderThreads, + CurrentMetrics::MergeTreePartsLoaderThreadsActive, + CurrentMetrics::MergeTreePartsLoaderThreadsScheduled, + getContext()->getSettingsRef()[Setting::max_threads]); + + merge_tree->prewarmCaches(pool, nullptr, std::move(index_cache)); } diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 6ae84fed672..47fac8330a4 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -84,7 +84,9 @@ private: AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); - void prewarmCaches(MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache); + + void prewarmMarkCache(); + void prewarmPrimaryIndexCache(); void stopReplicatedDDLQueries(); void startReplicatedDDLQueries(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a591289c68f..2aa9ac6a17c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -33,7 +33,6 @@ #include #include #include -#include "Common/Logger.h" #include #include #include @@ -389,7 +388,6 @@ IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::getIndex() const IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::loadIndexToCache(PrimaryIndexCache & index_cache) const { - LOG_DEBUG(getLogger("KEK"), "part name: {}, load index path: {}", name, getDataPartStorage().getFullPath()); auto key = PrimaryIndexCache::hash(getDataPartStorage().getFullPath()); auto callback = [this] { return loadIndex(); }; return index_cache.getOrSet(key, callback); @@ -415,7 +413,7 @@ void IMergeTreeDataPart::setIndex(Columns index_columns) if (index) throw Exception(ErrorCodes::LOGICAL_ERROR, "The index of data part can be set only once"); - optimizeIndexColumns(index_granularity.getMarksCount(), index_columns); + optimizeIndexColumns(index_granularity->getMarksCount(), index_columns); index = std::make_shared(std::move(index_columns)); } @@ -780,7 +778,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadIndexGranularity(); if (!(*storage.getSettings())[MergeTreeSetting::primary_key_lazy_load]) - getIndex(); + index = loadIndex(); calculateColumnsAndSecondaryIndicesSizesOnDisk(); loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`. @@ -1006,13 +1004,13 @@ std::shared_ptr IMergeTreeDataPart::loadIndex() const for (size_t i = 0; i < key_size; ++i) { loaded_index[i] = primary_key.data_types[i]->createColumn(); - loaded_index[i]->reserve(index_granularity.getMarksCount()); + loaded_index[i]->reserve(index_granularity->getMarksCount()); } String index_name = "primary" + getIndexExtensionFromFilesystem(getDataPartStorage()); String index_path = fs::path(getDataPartStorage().getRelativePath()) / index_name; auto index_file = metadata_manager->read(index_name); - size_t marks_count = index_granularity.getMarksCount(); + size_t marks_count = index_granularity->getMarksCount(); Serializations key_serializations(key_size); for (size_t j = 0; j < key_size; ++j) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 81bafae209c..10b518822ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -235,6 +235,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool use_primary_index_cache; extern const MergeTreeSettingsBool prewarm_primary_index_cache; extern const MergeTreeSettingsBool prewarm_mark_cache; + extern const MergeTreeSettingsBool primary_key_lazy_load; } namespace ServerSetting @@ -2349,7 +2350,10 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCache() const { - if (!(*getSettings())[MergeTreeSetting::use_primary_index_cache]) + bool use_primary_index_cache = (*getSettings())[MergeTreeSetting::use_primary_index_cache]; + bool primary_key_lazy_load = (*getSettings())[MergeTreeSetting::primary_key_lazy_load]; + + if (!use_primary_index_cache || !primary_key_lazy_load) return nullptr; return getContext()->getPrimaryIndexCache(); @@ -2376,14 +2380,6 @@ void MergeTreeData::prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, Pr if (!mark_cache && !index_cache) return; - Names columns_to_prewarm_marks; - - if (mark_cache) - { - auto metadata_snaphost = getInMemoryMetadataPtr(); - columns_to_prewarm_marks = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); - } - Stopwatch watch; LOG_TRACE(log, "Prewarming mark and/or primary index caches"); @@ -2407,13 +2403,32 @@ void MergeTreeData::prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, Pr double marks_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio]; double index_ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::primary_index_cache_prewarm_ratio]; + Names columns_to_prewarm_marks; + + if (mark_cache) + { + auto metadata_snaphost = getInMemoryMetadataPtr(); + columns_to_prewarm_marks = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); + } + for (const auto & part : data_parts) { - if (index_cache && index_cache->sizeInBytes() < index_cache->maxSizeInBytes() * index_ratio_to_prewarm) + bool added_task = false; + + if (index_cache && !part->isIndexLoaded() && index_cache->sizeInBytes() < index_cache->maxSizeInBytes() * index_ratio_to_prewarm) + { + added_task = true; runner([&] { part->loadIndexToCache(*index_cache); }); + } if (mark_cache && mark_cache->sizeInBytes() < mark_cache->maxSizeInBytes() * marks_ratio_to_prewarm) + { + added_task = true; runner([&] { part->loadMarksToCache(columns_to_prewarm_marks, mark_cache.get()); }); + } + + if (!added_task) + break; } runner.waitForAllToFinishAndRethrowFirstError(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 588390f012b..a0e1de24cf1 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -14,7 +14,7 @@ #include #include #include -#include "Common/logger_useful.h" +#include #include #include #include @@ -225,6 +225,27 @@ void MergeTreeDataWriter::TemporaryPart::finalize() projection->getDataPartStorage().precommitTransaction(); } +void MergeTreeDataWriter::TemporaryPart::prewarmCaches() +{ + /// This method must be called after rename and commit of part + /// because a correct path is required for the keys of caches. + + if (auto mark_cache = part->storage.getMarkCacheToPrewarm()) + { + for (const auto & stream : streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache.get()); + } + } + + if (auto index_cache = part->storage.getPrimaryIndexCacheToPrewarm()) + { + /// Index was already set during writing. Now move it to cache. + part->moveIndexToCache(*index_cache); + } +} + std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfoPtr async_insert_info, const IColumn::Selector & selector, size_t partition_num) { if (nullptr == async_insert_info) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 863c951d957..c2224a72683 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -75,6 +75,7 @@ public: void cancel(); void finalize(); + void prewarmCaches(); }; /** All rows must correspond to same partition. diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 6de4fa4feef..d65d1f3212f 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -247,21 +247,7 @@ void MergeTreeSink::finishDelayedChunk() /// Part can be deduplicated, so increment counters and add to part log only if it's really added if (added) { - if (auto mark_cache = storage.getMarkCacheToPrewarm()) - { - for (const auto & stream : partition.temp_part.streams) - { - auto marks = stream.stream->releaseCachedMarks(); - addMarksToCache(*part, marks, mark_cache.get()); - } - } - - if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) - { - /// Move index to cache and reset it here because we need - /// a correct part name after rename for a key of cache entry. - part->moveIndexToCache(*index_cache); - } + partition.temp_part.prewarmCaches(); auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot)); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 0aa0778ebe4..ab07bbd424e 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -17,7 +17,6 @@ namespace ErrorCodes namespace MergeTreeSetting { - extern const MergeTreeSettingsBool use_primary_index_cache; extern const MergeTreeSettingsBool enable_index_granularity_compression; } @@ -38,7 +37,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( , default_codec(default_codec_) , write_settings(write_settings_) { + /// Save marks in memory if prewarm is enabled to avoid rereading marks file. bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid rereading marks file. bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); MergeTreeWriterSettings writer_settings( diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index a330fe55aa0..46754d005ba 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -13,11 +13,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -namespace MergeTreeSetting -{ - extern const MergeTreeSettingsBool use_primary_index_cache; -} - MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeMutableDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, @@ -29,7 +24,9 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( WrittenOffsetColumns * offset_columns) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { + /// Save marks in memory if prewarm is enabled to avoid rereading marks file. bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid rereading marks file. bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); /// Granularity is never recomputed while writing only columns. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index d884b3aca7e..0a25a4582d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -466,28 +466,6 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) ++num_blocks_processed; } -template -void ReplicatedMergeTreeSinkImpl::prewarmCaches(const MergeTreeDataWriter::TemporaryPart & temp_part) const -{ - const auto & part = temp_part.part; - - if (auto mark_cache = storage.getMarkCacheToPrewarm()) - { - for (const auto & stream : temp_part.streams) - { - auto marks = stream.stream->releaseCachedMarks(); - addMarksToCache(*part, marks, mark_cache.get()); - } - } - - if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) - { - /// Move index to cache and reset it here because we need - /// a correct part name after rename for a key of cache entry. - part->moveIndexToCache(*index_cache); - } -} - template<> void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { @@ -510,7 +488,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; if (!error) - prewarmCaches(partition.temp_part); + partition.temp_part.prewarmCaches(); auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); @@ -555,7 +533,7 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa if (conflict_block_ids.empty()) { - prewarmCaches(partition.temp_part); + partition.temp_part.prewarmCaches(); auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0600beacd83..79ca6fd4c97 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5091,6 +5091,11 @@ bool StorageReplicatedMergeTree::fetchPart( part->loadMarksToCache(column_names, mark_cache.get()); } + if (auto index_cache = getPrimaryIndexCacheToPrewarm()) + { + part->loadIndexToCache(*index_cache); + } + write_part_log({}); } else diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index de6df7ac021..dddedb25f5a 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -114,6 +114,8 @@ SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYS SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM PREWARM MARK CACHE ['SYSTEM PREWARM MARK','PREWARM MARK CACHE','PREWARM MARKS'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE +SYSTEM PREWARM PRIMARY INDEX CACHE ['SYSTEM PREWARM PRIMARY INDEX','PREWARM PRIMARY INDEX CACHE','PREWARM PRIMARY INDEX'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP PRIMARY INDEX CACHE ['SYSTEM DROP PRIMARY INDEX','DROP PRIMARY INDEX CACHE','DROP PRIMARY INDEX'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP MMAP CACHE ['SYSTEM DROP MMAP','DROP MMAP CACHE','DROP MMAP'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP QUERY CACHE ['SYSTEM DROP QUERY','DROP QUERY CACHE','DROP QUERY'] GLOBAL SYSTEM DROP CACHE diff --git a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference index ed46312d77d..1a9e1167eb4 100644 --- a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference +++ b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.reference @@ -1,16 +1,22 @@ +449 0 -PrimaryIndexCacheBytes 1280 -PrimaryIndexCacheFiles 2 -99 +449 0 -PrimaryIndexCacheBytes 1280 -PrimaryIndexCacheFiles 2 +898 +898 0 -PrimaryIndexCacheBytes 1280 -PrimaryIndexCacheFiles 2 -49 +898 +898 +0 +898 +0 +898 +0 +0 +0 +0 +0 +0 +0 +1 0 -PrimaryIndexCacheBytes 1280 -PrimaryIndexCacheFiles 2 -0 0 0 -0 0 0 diff --git a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql index e9e04cff6ec..08c41d0fc72 100644 --- a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql +++ b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql @@ -1,46 +1,74 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-shared-merge-tree -DROP TABLE IF EXISTS t_primary_index_cache_2; +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; +DROP TABLE IF EXISTS t_prewarm_cache_rmt_2; -CREATE TABLE t_primary_index_cache_2 (a UInt64, b UInt64) -ENGINE = MergeTree ORDER BY a PARTITION BY a % 2 -SETTINGS use_primary_index_cache = 1, prewarm_primary_index_cache = 1, index_granularity = 64, index_granularity_bytes = '10M', min_bytes_for_wide_part = 0; +CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03274_prewarm_mark_cache_smt/t_prewarm_cache', '1') +ORDER BY a PARTITION BY a % 2 +SETTINGS prewarm_primary_index_cache = 1, use_primary_index_cache = 1; + +CREATE TABLE t_prewarm_cache_rmt_2 (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03274_prewarm_mark_cache_smt/t_prewarm_cache', '2') +ORDER BY a PARTITION BY a % 2 +SETTINGS prewarm_primary_index_cache = 1, use_primary_index_cache = 1; + +SYSTEM DROP PRIMARY INDEX CACHE; +SYSTEM STOP FETCHES t_prewarm_cache_rmt_2; + +-- Check that prewarm works on insert. +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(20000); + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); + +-- Check that prewarm works on fetch. +SYSTEM DROP PRIMARY INDEX CACHE; +SYSTEM START FETCHES t_prewarm_cache_rmt_2; +SYSTEM SYNC REPLICA t_prewarm_cache_rmt_2; + +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); + +-- Check that prewarm works on merge. +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(20000); +OPTIMIZE TABLE t_prewarm_cache_rmt_1 FINAL; + +SYSTEM SYNC REPLICA t_prewarm_cache_rmt_2; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); + +-- Check that prewarm works on restart. +SYSTEM DROP PRIMARY INDEX CACHE; + +DETACH TABLE t_prewarm_cache_rmt_1; +DETACH TABLE t_prewarm_cache_rmt_2; + +ATTACH TABLE t_prewarm_cache_rmt_1; +ATTACH TABLE t_prewarm_cache_rmt_2; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT count() FROM t_prewarm_cache_rmt_2 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); SYSTEM DROP PRIMARY INDEX CACHE; -INSERT INTO t_primary_index_cache_2 SELECT number, number FROM numbers(10000); +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; +--- Check that system query works. +SYSTEM PREWARM PRIMARY INDEX CACHE t_prewarm_cache_rmt_1; -SELECT count() FROM t_primary_index_cache_2 WHERE a > 100 AND a < 200; - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; - -SYSTEM DROP PRIMARY INDEX CACHE; -SYSTEM PREWARM PRIMARY INDEX CACHE t_primary_index_cache_2; - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; - -SELECT count() FROM t_primary_index_cache_2 WHERE a > 100 AND a < 200 AND a % 2 = 0; - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE table = 't_primary_index_cache_2' AND active; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'PrimaryIndexCacheBytes') ORDER BY metric; +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a > 100 AND a < 1000; +SELECT sum(primary_key_bytes_in_memory) FROM system.parts WHERE database = currentDatabase() AND table IN ('t_prewarm_cache_rmt_1', 't_prewarm_cache_rmt_2'); SYSTEM FLUSH LOGS; -SELECT - ProfileEvents['LoadedPrimaryIndexFiles'], - ProfileEvents['LoadedPrimaryIndexRows'], - ProfileEvents['LoadedPrimaryIndexBytes'] -FROM system.query_log -WHERE query LIKE 'SELECT count() FROM t_primary_index_cache_2%' AND current_database = currentDatabase() AND type = 'QueryFinish' +SELECT ProfileEvents['LoadedPrimaryIndexFiles'] FROM system.query_log +WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache%' ORDER BY event_time_microseconds; -DROP TABLE t_primary_index_cache_2; \ No newline at end of file +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; +DROP TABLE IF EXISTS t_prewarm_cache_rmt_2; From 5a95219e89aa9ac9f8d56980073d8bc18aea18d1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Nov 2024 22:07:57 +0000 Subject: [PATCH 125/433] Add setting parallel_replicas_skip_index_analysis_on_workers --- src/Core/Settings.cpp | 3 +++ src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +++- .../integration/test_parallel_replicas_all_marks_read/test.py | 1 + 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 140a77011dd..2cd3d272490 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5652,6 +5652,9 @@ Parts virtually divided into segments to be distributed between replicas for par )", BETA) \ DECLARE(Bool, parallel_replicas_local_plan, true, R"( Build local plan for local replica +)", BETA) \ + DECLARE(Bool, parallel_replicas_skip_index_analysis_on_workers, true, R"( +Skip index analysis on workers. Effective only with enabled parallel_replicas_local_plan )", BETA) \ \ DECLARE(Bool, allow_experimental_analyzer, true, R"( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 36bc7588542..f1c0a781502 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -81,6 +81,7 @@ namespace Setting extern const SettingsUInt64 parallel_replicas_count; extern const SettingsParallelReplicasMode parallel_replicas_mode; extern const SettingsBool parallel_replicas_local_plan; + extern const SettingsBool parallel_replicas_skip_index_analysis_on_workers; } namespace MergeTreeSetting @@ -634,7 +635,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { const Settings & settings = context->getSettingsRef(); - if (context->canUseParallelReplicasOnFollower() && settings[Setting::parallel_replicas_local_plan]) + if (context->canUseParallelReplicasOnFollower() && settings[Setting::parallel_replicas_local_plan] + && settings[Setting::parallel_replicas_skip_index_analysis_on_workers]) { RangesInDataParts parts_with_ranges; parts_with_ranges.reserve(parts.size()); diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py index 593b98126df..9f91a980474 100644 --- a/tests/integration/test_parallel_replicas_all_marks_read/test.py +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -71,6 +71,7 @@ def _get_result_with_parallel_replicas( "cluster_for_parallel_replicas": f"{cluster_name}", "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, "query_id": query_id, + "parallel_replicas_skip_index_analysis_on_workers": True, }, ) From cc6fd38dc616b2ab98b3c9fce6ae10877fb28e0f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 20 Nov 2024 22:41:13 +0000 Subject: [PATCH 126/433] preliminary: prevent uploading corpus if running in PR --- tests/ci/libfuzzer_test_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 2616fbe3f5d..d1e6b34309e 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -286,7 +286,11 @@ def main(): retcode = process.wait() if retcode == 0: logging.info("Run successfully") - upload_corpus(fuzzers_path) + if pr_info.number == 0 and pr_info.base_ref == "master" and pr_info.head_ref == "master": + logging.info("Uploading corpus - running in master") + # upload_corpus(fuzzers_path) + else: + logging.info("Not uploading corpus - running in PR") else: logging.info("Run failed") From 6d4c26dda5ab36b9a37b64f23686cce57ae37fc7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 20 Nov 2024 22:50:39 +0000 Subject: [PATCH 127/433] Automatic style fix --- tests/ci/libfuzzer_test_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index d1e6b34309e..55db6f8e235 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -286,7 +286,11 @@ def main(): retcode = process.wait() if retcode == 0: logging.info("Run successfully") - if pr_info.number == 0 and pr_info.base_ref == "master" and pr_info.head_ref == "master": + if ( + pr_info.number == 0 + and pr_info.base_ref == "master" + and pr_info.head_ref == "master" + ): logging.info("Uploading corpus - running in master") # upload_corpus(fuzzers_path) else: From c0195f5482ddae3ac5b8a76d04b3c9c0ad1cbb6e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 20 Nov 2024 22:56:36 +0000 Subject: [PATCH 128/433] Update settings history --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index f0d3e001362..c4cf5cf22a1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -90,6 +90,7 @@ static std::initializer_list Date: Wed, 20 Nov 2024 22:57:09 +0000 Subject: [PATCH 129/433] Fix test_parallel_replicas_all_marks_read --- tests/integration/test_parallel_replicas_all_marks_read/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_all_marks_read/test.py b/tests/integration/test_parallel_replicas_all_marks_read/test.py index 9f91a980474..92317afabbe 100644 --- a/tests/integration/test_parallel_replicas_all_marks_read/test.py +++ b/tests/integration/test_parallel_replicas_all_marks_read/test.py @@ -71,7 +71,7 @@ def _get_result_with_parallel_replicas( "cluster_for_parallel_replicas": f"{cluster_name}", "parallel_replicas_mark_segment_size": parallel_replicas_mark_segment_size, "query_id": query_id, - "parallel_replicas_skip_index_analysis_on_workers": True, + "parallel_replicas_skip_index_analysis_on_workers": False, }, ) From 2313b7ca78b734dc8300ba07007ce5fa95c4d098 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 01:59:12 +0000 Subject: [PATCH 130/433] update: prevent uploading corpus if running in PR, add sanitizers --- tests/ci/ci_config.py | 36 ++++++++++++++++++++++++++++++++ tests/ci/ci_definitions.py | 4 ++++ tests/ci/libfuzzer_test_check.py | 2 +- 3 files changed, 41 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index a170dfdd8d1..cca93f5a488 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -278,6 +278,42 @@ class CI: ), run_by_labels=[Tags.libFuzzer], ), + BuildNames.FUZZERS_ASAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS_ASAN, + compiler="clang-18", + sanitizer="address", + package_type="fuzzers", + ), + run_by_labels=[Tags.libFuzzer], + ), + BuildNames.FUZZERS_UBSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS_UBSAN, + compiler="clang-18", + sanitizer="undefined", + package_type="fuzzers", + ), + run_by_labels=[Tags.libFuzzer], + ), + BuildNames.FUZZERS_TSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS_TSAN, + compiler="clang-18", + sanitizer="thread", + package_type="fuzzers", + ), + run_by_labels=[Tags.libFuzzer], + ), + BuildNames.FUZZERS_MSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS_MSAN, + compiler="clang-18", + sanitizer="memory", + package_type="fuzzers", + ), + run_by_labels=[Tags.libFuzzer], + ), JobNames.BUILD_CHECK: CommonJobConfigs.BUILD_REPORT.with_properties(), JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index d55e347c3ee..ad15d692944 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -123,6 +123,10 @@ class BuildNames(metaclass=WithIter): BINARY_S390X = "binary_s390x" BINARY_LOONGARCH64 = "binary_loongarch64" FUZZERS = "fuzzers" + FUZZERS_ASAN = "fuzzers_asan" + FUZZERS_UBSAN = "fuzzers_ubsan" + FUZZERS_TSAN = "fuzzers_tsan" + FUZZERS_MSAN = "fuzzers_msan" class JobNames(metaclass=WithIter): diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 55db6f8e235..6783f5c1613 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -292,7 +292,7 @@ def main(): and pr_info.head_ref == "master" ): logging.info("Uploading corpus - running in master") - # upload_corpus(fuzzers_path) + upload_corpus(fuzzers_path) else: logging.info("Not uploading corpus - running in PR") else: From e87ac61ff31d88da51cb614f919ca43785b0d0f8 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 02:36:58 +0000 Subject: [PATCH 131/433] fix: only asan --- tests/ci/ci_config.py | 35 ----------------------------------- 1 file changed, 35 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index cca93f5a488..adb0881ecd5 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -271,14 +271,6 @@ class CI: ), ), BuildNames.FUZZERS: CommonJobConfigs.BUILD.with_properties( - build_config=BuildConfig( - name=BuildNames.FUZZERS, - compiler="clang-18", - package_type="fuzzers", - ), - run_by_labels=[Tags.libFuzzer], - ), - BuildNames.FUZZERS_ASAN: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( name=BuildNames.FUZZERS_ASAN, compiler="clang-18", @@ -287,33 +279,6 @@ class CI: ), run_by_labels=[Tags.libFuzzer], ), - BuildNames.FUZZERS_UBSAN: CommonJobConfigs.BUILD.with_properties( - build_config=BuildConfig( - name=BuildNames.FUZZERS_UBSAN, - compiler="clang-18", - sanitizer="undefined", - package_type="fuzzers", - ), - run_by_labels=[Tags.libFuzzer], - ), - BuildNames.FUZZERS_TSAN: CommonJobConfigs.BUILD.with_properties( - build_config=BuildConfig( - name=BuildNames.FUZZERS_TSAN, - compiler="clang-18", - sanitizer="thread", - package_type="fuzzers", - ), - run_by_labels=[Tags.libFuzzer], - ), - BuildNames.FUZZERS_MSAN: CommonJobConfigs.BUILD.with_properties( - build_config=BuildConfig( - name=BuildNames.FUZZERS_MSAN, - compiler="clang-18", - sanitizer="memory", - package_type="fuzzers", - ), - run_by_labels=[Tags.libFuzzer], - ), JobNames.BUILD_CHECK: CommonJobConfigs.BUILD_REPORT.with_properties(), JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] From c9ffcb0ef11fa7ff43712350efcac6f6e2337b3b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 03:00:16 +0000 Subject: [PATCH 132/433] fix --- tests/ci/ci_config.py | 2 +- tests/ci/ci_definitions.py | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index adb0881ecd5..d0f62b5d770 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -272,7 +272,7 @@ class CI: ), BuildNames.FUZZERS: CommonJobConfigs.BUILD.with_properties( build_config=BuildConfig( - name=BuildNames.FUZZERS_ASAN, + name=BuildNames.FUZZERS, compiler="clang-18", sanitizer="address", package_type="fuzzers", diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index ad15d692944..d55e347c3ee 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -123,10 +123,6 @@ class BuildNames(metaclass=WithIter): BINARY_S390X = "binary_s390x" BINARY_LOONGARCH64 = "binary_loongarch64" FUZZERS = "fuzzers" - FUZZERS_ASAN = "fuzzers_asan" - FUZZERS_UBSAN = "fuzzers_ubsan" - FUZZERS_TSAN = "fuzzers_tsan" - FUZZERS_MSAN = "fuzzers_msan" class JobNames(metaclass=WithIter): From bfafaf93dcc7429093566457c3a1a8586f189039 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 03:17:01 +0000 Subject: [PATCH 133/433] trigger build From 52ab45b6eecdc2b547005dd73a07083593c141cc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 03:29:56 +0000 Subject: [PATCH 134/433] trigger build --- src/DataTypes/fuzzers/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 8940586fc70..8dedd3470e2 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,3 +1,2 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) - target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms) From 163f2b11215f0e57fecb802415300485dd56b512 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 05:57:12 +0000 Subject: [PATCH 135/433] run time to 5 min per test --- tests/ci/libfuzzer_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 6783f5c1613..003a256f18c 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -21,7 +21,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen -TIMEOUT = 60 +TIMEOUT = 60 * 5 NO_CHANGES_MSG = "Nothing to run" s3 = S3Helper() From cb4741f7704bc862882e044d0d3e9bd0850bccae Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 31 Oct 2024 12:34:53 +0000 Subject: [PATCH 136/433] Fix zombie processes after library brigde crash. --- src/Common/ShellCommand.cpp | 169 +++++++++++------- src/Common/ShellCommand.h | 3 + tests/integration/helpers/cluster.py | 6 + tests/integration/test_library_bridge/test.py | 11 ++ 4 files changed, 126 insertions(+), 63 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 0d41669816c..fe3bd8f8b9c 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -25,6 +25,7 @@ namespace CANNOT_EXEC = 0x55555558, CANNOT_DUP_READ_DESCRIPTOR = 0x55555559, CANNOT_DUP_WRITE_DESCRIPTOR = 0x55555560, + CANNOT_VFORK_IN_CHILD = 0x55555561, }; } @@ -138,7 +139,7 @@ std::unique_ptr ShellCommand::executeImpl( * http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html * Therefore, separate the resolving of the symbol from the call. */ - static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork"); + static void * real_vfork = dlsym(RTLD_DEFAULT, "fork"); #else /// If we use Musl with static linking, there is no dlsym and no issue with vfork. static void * real_vfork = reinterpret_cast(&vfork); @@ -151,6 +152,10 @@ std::unique_ptr ShellCommand::executeImpl( PipeFDs pipe_stdout; PipeFDs pipe_stderr; + PipeFDs pipe_with_child; + WriteBufferFromFile write_buffer_for_child(pipe_with_child.fds_rw[1]); + ReadBufferFromFile read_buffer_for_parent(pipe_with_child.fds_rw[0]); + std::vector> read_pipe_fds; std::vector> write_pipe_fds; @@ -160,64 +165,93 @@ std::unique_ptr ShellCommand::executeImpl( for (size_t i = 0; i < config.write_fds.size(); ++i) write_pipe_fds.emplace_back(std::make_unique()); - pid_t pid = reinterpret_cast(real_vfork)(); + pid_t child_pid = reinterpret_cast(real_vfork)(); - if (pid == -1) + if (child_pid == -1) throw ErrnoException(ErrorCodes::CANNOT_FORK, "Cannot vfork"); - if (0 == pid) + /// Here we are using double vfork technique to prevent zombie process. + if (0 == child_pid) { - /// We are in the freshly created process. + pid_t pid = reinterpret_cast(real_vfork)(); - /// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage. - /// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens. + if (pid == -1) + _exit(static_cast(ReturnCodes::CANNOT_VFORK_IN_CHILD)); - /// Replace the file descriptors with the ends of our pipes. - if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDIN)); - - if (!config.pipe_stdin_only) + if (pid == 0) { - if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDOUT)); + /// We are in the freshly created process. - if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDERR)); + /// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage. + /// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens. + + /// Replace the file descriptors with the ends of our pipes. + if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDIN)); + + if (!config.pipe_stdin_only) + { + if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDOUT)); + + if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDERR)); + } + + for (size_t i = 0; i < config.read_fds.size(); ++i) + { + auto & fds = *read_pipe_fds[i]; + auto fd = config.read_fds[i]; + + if (fd != dup2(fds.fds_rw[1], fd)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); + } + + for (size_t i = 0; i < config.write_fds.size(); ++i) + { + auto & fds = *write_pipe_fds[i]; + auto fd = config.write_fds[i]; + + if (fd != dup2(fds.fds_rw[0], fd)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); + } + + // Reset the signal mask: it may be non-empty and will be inherited + // by the child process, which might not expect this. + sigset_t mask; + sigemptyset(&mask); + sigprocmask(0, nullptr, &mask); // NOLINT(concurrency-mt-unsafe) + sigprocmask(SIG_UNBLOCK, &mask, nullptr); // NOLINT(concurrency-mt-unsafe) + + execv(filename, argv); + /// If the process is running, then `execv` does not return here. + + _exit(static_cast(ReturnCodes::CANNOT_EXEC)); } - - for (size_t i = 0; i < config.read_fds.size(); ++i) + else { - auto & fds = *read_pipe_fds[i]; - auto fd = config.read_fds[i]; - - if (fd != dup2(fds.fds_rw[1], fd)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); + DB::writeIntBinary(pid, write_buffer_for_child); + write_buffer_for_child.next(); + _exit(0); } - - for (size_t i = 0; i < config.write_fds.size(); ++i) + } + else + { + int status = 0; + while (waitpid(child_pid, &status, 0) < 0) { - auto & fds = *write_pipe_fds[i]; - auto fd = config.write_fds[i]; - - if (fd != dup2(fds.fds_rw[0], fd)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); + if (errno != EINTR) + throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid"); } - - // Reset the signal mask: it may be non-empty and will be inherited - // by the child process, which might not expect this. - sigset_t mask; - sigemptyset(&mask); - sigprocmask(0, nullptr, &mask); // NOLINT(concurrency-mt-unsafe) - sigprocmask(SIG_UNBLOCK, &mask, nullptr); // NOLINT(concurrency-mt-unsafe) - - execv(filename, argv); - /// If the process is running, then `execv` does not return here. - - _exit(static_cast(ReturnCodes::CANNOT_EXEC)); + int return_code = handleWaitStatus(child_pid, status); + handleProceesReturnCode(child_pid, return_code); } + pid_t grandchild_pid = 0; + DB::readIntBinary(grandchild_pid, read_buffer_for_parent); + std::unique_ptr res(new ShellCommand( - pid, + grandchild_pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], @@ -241,7 +275,7 @@ std::unique_ptr ShellCommand::executeImpl( getLogger(), "Started shell command '{}' with pid {} and file descriptors: out {}, err {}", filename, - pid, + grandchild_pid, res->out.getFD(), res->err.getFD()); @@ -317,44 +351,53 @@ int ShellCommand::tryWait() LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status); - if (WIFEXITED(status)) - return WEXITSTATUS(status); - - if (WIFSIGNALED(status)) - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was terminated by signal {}", toString(WTERMSIG(status))); - - if (WIFSTOPPED(status)) - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was stopped by signal {}", toString(WSTOPSIG(status))); - - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was not exited normally by unknown reason"); + return handleWaitStatus(pid, status); } - void ShellCommand::wait() { int retcode = tryWait(); + handleProceesReturnCode(pid, retcode); +} +int ShellCommand::handleWaitStatus(pid_t pid, int status) +{ + if (WIFEXITED(status)) + return WEXITSTATUS(status); + + if (WIFSIGNALED(status)) + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was terminated by signal {}", pid, toString(WTERMSIG(status))); + + if (WIFSTOPPED(status)) + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was stopped by signal {}", pid, toString(WSTOPSIG(status))); + + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was not exited normally by unknown reason", pid); +} + +void ShellCommand::handleProceesReturnCode(pid_t pid, int retcode) +{ if (retcode != EXIT_SUCCESS) { switch (retcode) { case static_cast(ReturnCodes::CANNOT_DUP_STDIN): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdin of child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdin of child process: {}", pid); case static_cast(ReturnCodes::CANNOT_DUP_STDOUT): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdout of child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdout of child process {}", pid); case static_cast(ReturnCodes::CANNOT_DUP_STDERR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stderr of child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stderr of child process {}", pid); case static_cast(ReturnCodes::CANNOT_EXEC): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot execv in child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot execv in child process {}", pid); case static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 read descriptor of child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 read descriptor of child process {}", pid); case static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 write descriptor of child process"); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 write descriptor of child process {}", pid); + case static_cast(ReturnCodes::CANNOT_VFORK_IN_CHILD): + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot vfork in child procces {}", pid); default: - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was exited with return code {}", toString(retcode)); + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was exited with return code {}", pid, toString(retcode)); } } } - } diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index 5ebc1daefa1..17106db9698 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -99,6 +99,9 @@ private: static LoggerPtr getLogger(); + static int handleWaitStatus(pid_t pid, int status); + static void handleProceesReturnCode(pid_t pid, int exit_code); + /// Print command name and the list of arguments to log. NOTE: No escaping of arguments is performed. static void logCommand(const char * filename, char * const argv[]); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eae0f9fec2d..949343e9036 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1657,6 +1657,7 @@ class ClickHouseCluster: extra_configs=[], extra_args="", randomize_settings=True, + use_docker_init_flag=False, ) -> "ClickHouseInstance": """Add an instance to the cluster. @@ -1762,6 +1763,7 @@ class ClickHouseCluster: config_root_name=config_root_name, extra_configs=extra_configs, randomize_settings=randomize_settings, + use_docker_init_flag=use_docker_init_flag, ) docker_compose_yml_dir = get_docker_compose_path() @@ -3353,6 +3355,7 @@ services: {ipv6_address} {net_aliases} {net_alias1} + init: {init_flag} """ @@ -3419,6 +3422,7 @@ class ClickHouseInstance: config_root_name="clickhouse", extra_configs=[], randomize_settings=True, + use_docker_init_flag=False, ): self.name = name self.base_cmd = cluster.base_cmd @@ -3545,6 +3549,7 @@ class ClickHouseInstance: self.with_installed_binary = with_installed_binary self.is_up = False self.config_root_name = config_root_name + self.docker_init_flag = use_docker_init_flag def is_built_with_sanitizer(self, sanitizer_name=""): build_opts = self.query( @@ -4838,6 +4843,7 @@ class ClickHouseInstance: ipv6_address=ipv6_address, net_aliases=net_aliases, net_alias1=net_alias1, + init_flag="true" if self.docker_init_flag else "false", ) ) diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 6254735a18f..79ec17271be 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -14,6 +14,11 @@ instance = cluster.add_instance( dictionaries=["configs/dictionaries/dict1.xml"], main_configs=["configs/config.d/config.xml"], stay_alive=True, + # WA for the problem with zombie processes inside the docker container. + # This is important here because we are checking that there are no zombie processes + # after craches inside the library bridge. + # https://forums.docker.com/t/what-the-latest-with-the-zombie-process-reaping-problem/50758/2 + use_docker_init_flag=True, ) @@ -263,6 +268,12 @@ def test_recover_after_bridge_crash(ch_cluster): instance.exec_in_container( ["bash", "-c", "kill -9 `pidof clickhouse-library-bridge`"], user="root" ) + ## There are no zombie processes. + res = instance.exec_in_container( + ["bash", "-c", "ps ax -ostat,pid | grep -e '[zZ]' | wc -l"], user="root" + ) + assert res == "0\n" + instance.query("DROP DICTIONARY lib_dict_c") From f12a582bd09545e831a19ed635d47457b14a9f9e Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 31 Oct 2024 14:11:18 +0000 Subject: [PATCH 137/433] vfork --- src/Common/ShellCommand.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index fe3bd8f8b9c..c2d8cc44ab3 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -139,7 +139,7 @@ std::unique_ptr ShellCommand::executeImpl( * http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html * Therefore, separate the resolving of the symbol from the call. */ - static void * real_vfork = dlsym(RTLD_DEFAULT, "fork"); + static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork"); #else /// If we use Musl with static linking, there is no dlsym and no issue with vfork. static void * real_vfork = reinterpret_cast(&vfork); From 4cc8d36419067a551cf3e8736af11da89f137dc5 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 5 Nov 2024 15:49:01 +0000 Subject: [PATCH 138/433] Redesigned --- src/BridgeHelper/LibraryBridgeHelper.cpp | 2 +- src/BridgeHelper/XDBCBridgeHelper.h | 2 +- src/Common/BackgroundShellCommandHolder.cpp | 37 ++++ src/Common/BackgroundShellCommandHolder.h | 30 ++++ src/Common/ShellCommand.cpp | 163 +++++++----------- src/Common/ShellCommand.h | 8 +- src/Common/SignalHandlers.cpp | 20 +++ src/Common/SignalHandlers.h | 1 + src/Daemon/BaseDaemon.cpp | 1 + src/Interpreters/Context.cpp | 16 +- src/Interpreters/Context.h | 7 +- tests/integration/test_library_bridge/test.py | 15 +- 12 files changed, 182 insertions(+), 120 deletions(-) create mode 100644 src/Common/BackgroundShellCommandHolder.cpp create mode 100644 src/Common/BackgroundShellCommandHolder.h diff --git a/src/BridgeHelper/LibraryBridgeHelper.cpp b/src/BridgeHelper/LibraryBridgeHelper.cpp index 58b84aa1ffd..e9fd1078658 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.cpp +++ b/src/BridgeHelper/LibraryBridgeHelper.cpp @@ -29,7 +29,7 @@ LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_) void LibraryBridgeHelper::startBridge(std::unique_ptr cmd) const { - getContext()->addBridgeCommand(std::move(cmd)); + getContext()->addBackgroundShellCommand(std::move(cmd)); } diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 2e1328b4057..0851d1289db 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -144,7 +144,7 @@ protected: void startBridge(std::unique_ptr cmd) const override { - getContext()->addBridgeCommand(std::move(cmd)); + getContext()->addBackgroundShellCommand(std::move(cmd)); } diff --git a/src/Common/BackgroundShellCommandHolder.cpp b/src/Common/BackgroundShellCommandHolder.cpp new file mode 100644 index 00000000000..8d239356218 --- /dev/null +++ b/src/Common/BackgroundShellCommandHolder.cpp @@ -0,0 +1,37 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +LoggerPtr BackgroundShellCommandHolder::getLogger() +{ + return ::getLogger("BackgroundShellCommandHolder"); +} + + +void BackgroundShellCommandHolder::removeCommand(pid_t pid) +{ + std::lock_guard lock(mutex); + bool is_erased = active_shell_commands.erase(pid); + LOG_TRACE(getLogger(), "Try to erase command with the pid {}, is_erased: {}", pid, is_erased); +} + +void BackgroundShellCommandHolder::addCommand(std::unique_ptr command) +{ + std::lock_guard lock(mutex); + pid_t command_pid = command->getPid(); + + auto [iterator, is_inserted] = active_shell_commands.emplace(std::make_pair(command_pid, std::move(command))); + if (!is_inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't insert proccess PID {} into active shell commands, because there are running proccess with same PID", command_pid); + + LOG_TRACE(getLogger(), "Inserted the command with pid {}", command_pid); +} +} diff --git a/src/Common/BackgroundShellCommandHolder.h b/src/Common/BackgroundShellCommandHolder.h new file mode 100644 index 00000000000..58bbbefcec1 --- /dev/null +++ b/src/Common/BackgroundShellCommandHolder.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +#include +#include + + +namespace DB +{ + +/** The holder class for running background shell processes. +*/ +class BackgroundShellCommandHolder final +{ +public: + void removeCommand(pid_t pid); + void addCommand(std::unique_ptr command); + +private: + using ActiveShellCommandsCollection = std::unordered_map>; + + std::mutex mutex; + ActiveShellCommandsCollection active_shell_commands TSA_GUARDED_BY(mutex); + + static LoggerPtr getLogger(); +}; + +} diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index c2d8cc44ab3..0d41669816c 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -25,7 +25,6 @@ namespace CANNOT_EXEC = 0x55555558, CANNOT_DUP_READ_DESCRIPTOR = 0x55555559, CANNOT_DUP_WRITE_DESCRIPTOR = 0x55555560, - CANNOT_VFORK_IN_CHILD = 0x55555561, }; } @@ -152,10 +151,6 @@ std::unique_ptr ShellCommand::executeImpl( PipeFDs pipe_stdout; PipeFDs pipe_stderr; - PipeFDs pipe_with_child; - WriteBufferFromFile write_buffer_for_child(pipe_with_child.fds_rw[1]); - ReadBufferFromFile read_buffer_for_parent(pipe_with_child.fds_rw[0]); - std::vector> read_pipe_fds; std::vector> write_pipe_fds; @@ -165,93 +160,64 @@ std::unique_ptr ShellCommand::executeImpl( for (size_t i = 0; i < config.write_fds.size(); ++i) write_pipe_fds.emplace_back(std::make_unique()); - pid_t child_pid = reinterpret_cast(real_vfork)(); + pid_t pid = reinterpret_cast(real_vfork)(); - if (child_pid == -1) + if (pid == -1) throw ErrnoException(ErrorCodes::CANNOT_FORK, "Cannot vfork"); - /// Here we are using double vfork technique to prevent zombie process. - if (0 == child_pid) + if (0 == pid) { - pid_t pid = reinterpret_cast(real_vfork)(); + /// We are in the freshly created process. - if (pid == -1) - _exit(static_cast(ReturnCodes::CANNOT_VFORK_IN_CHILD)); + /// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage. + /// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens. - if (pid == 0) + /// Replace the file descriptors with the ends of our pipes. + if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDIN)); + + if (!config.pipe_stdin_only) { - /// We are in the freshly created process. + if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDOUT)); - /// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage. - /// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens. - - /// Replace the file descriptors with the ends of our pipes. - if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDIN)); - - if (!config.pipe_stdin_only) - { - if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDOUT)); - - if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_STDERR)); - } - - for (size_t i = 0; i < config.read_fds.size(); ++i) - { - auto & fds = *read_pipe_fds[i]; - auto fd = config.read_fds[i]; - - if (fd != dup2(fds.fds_rw[1], fd)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); - } - - for (size_t i = 0; i < config.write_fds.size(); ++i) - { - auto & fds = *write_pipe_fds[i]; - auto fd = config.write_fds[i]; - - if (fd != dup2(fds.fds_rw[0], fd)) - _exit(static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); - } - - // Reset the signal mask: it may be non-empty and will be inherited - // by the child process, which might not expect this. - sigset_t mask; - sigemptyset(&mask); - sigprocmask(0, nullptr, &mask); // NOLINT(concurrency-mt-unsafe) - sigprocmask(SIG_UNBLOCK, &mask, nullptr); // NOLINT(concurrency-mt-unsafe) - - execv(filename, argv); - /// If the process is running, then `execv` does not return here. - - _exit(static_cast(ReturnCodes::CANNOT_EXEC)); + if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_STDERR)); } - else + + for (size_t i = 0; i < config.read_fds.size(); ++i) { - DB::writeIntBinary(pid, write_buffer_for_child); - write_buffer_for_child.next(); - _exit(0); + auto & fds = *read_pipe_fds[i]; + auto fd = config.read_fds[i]; + + if (fd != dup2(fds.fds_rw[1], fd)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR)); } + + for (size_t i = 0; i < config.write_fds.size(); ++i) + { + auto & fds = *write_pipe_fds[i]; + auto fd = config.write_fds[i]; + + if (fd != dup2(fds.fds_rw[0], fd)) + _exit(static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR)); + } + + // Reset the signal mask: it may be non-empty and will be inherited + // by the child process, which might not expect this. + sigset_t mask; + sigemptyset(&mask); + sigprocmask(0, nullptr, &mask); // NOLINT(concurrency-mt-unsafe) + sigprocmask(SIG_UNBLOCK, &mask, nullptr); // NOLINT(concurrency-mt-unsafe) + + execv(filename, argv); + /// If the process is running, then `execv` does not return here. + + _exit(static_cast(ReturnCodes::CANNOT_EXEC)); } - else - { - int status = 0; - while (waitpid(child_pid, &status, 0) < 0) - { - if (errno != EINTR) - throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid"); - } - int return_code = handleWaitStatus(child_pid, status); - handleProceesReturnCode(child_pid, return_code); - } - - pid_t grandchild_pid = 0; - DB::readIntBinary(grandchild_pid, read_buffer_for_parent); std::unique_ptr res(new ShellCommand( - grandchild_pid, + pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], @@ -275,7 +241,7 @@ std::unique_ptr ShellCommand::executeImpl( getLogger(), "Started shell command '{}' with pid {} and file descriptors: out {}, err {}", filename, - grandchild_pid, + pid, res->out.getFD(), res->err.getFD()); @@ -351,53 +317,44 @@ int ShellCommand::tryWait() LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status); - return handleWaitStatus(pid, status); -} - -void ShellCommand::wait() -{ - int retcode = tryWait(); - handleProceesReturnCode(pid, retcode); -} - -int ShellCommand::handleWaitStatus(pid_t pid, int status) -{ if (WIFEXITED(status)) return WEXITSTATUS(status); if (WIFSIGNALED(status)) - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was terminated by signal {}", pid, toString(WTERMSIG(status))); + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was terminated by signal {}", toString(WTERMSIG(status))); if (WIFSTOPPED(status)) - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was stopped by signal {}", pid, toString(WSTOPSIG(status))); + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was stopped by signal {}", toString(WSTOPSIG(status))); - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was not exited normally by unknown reason", pid); + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was not exited normally by unknown reason"); } -void ShellCommand::handleProceesReturnCode(pid_t pid, int retcode) + +void ShellCommand::wait() { + int retcode = tryWait(); + if (retcode != EXIT_SUCCESS) { switch (retcode) { case static_cast(ReturnCodes::CANNOT_DUP_STDIN): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdin of child process: {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdin of child process"); case static_cast(ReturnCodes::CANNOT_DUP_STDOUT): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdout of child process {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stdout of child process"); case static_cast(ReturnCodes::CANNOT_DUP_STDERR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stderr of child process {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 stderr of child process"); case static_cast(ReturnCodes::CANNOT_EXEC): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot execv in child process {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot execv in child process"); case static_cast(ReturnCodes::CANNOT_DUP_READ_DESCRIPTOR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 read descriptor of child process {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 read descriptor of child process"); case static_cast(ReturnCodes::CANNOT_DUP_WRITE_DESCRIPTOR): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 write descriptor of child process {}", pid); - case static_cast(ReturnCodes::CANNOT_VFORK_IN_CHILD): - throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot vfork in child procces {}", pid); + throw Exception(ErrorCodes::CANNOT_CREATE_CHILD_PROCESS, "Cannot dup2 write descriptor of child process"); default: - throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process {} was exited with return code {}", pid, toString(retcode)); + throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was exited with return code {}", toString(retcode)); } } } + } diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index 17106db9698..7e068baab4c 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -67,6 +67,11 @@ public: DestructorStrategy terminate_in_destructor_strategy = DestructorStrategy(false, 0); }; + pid_t getPid() + { + return pid; + } + /// Run the command using /bin/sh -c. /// If terminate_in_destructor is true, send terminate signal in destructor and don't wait process. static std::unique_ptr execute(const Config & config); @@ -99,9 +104,6 @@ private: static LoggerPtr getLogger(); - static int handleWaitStatus(pid_t pid, int status); - static void handleProceesReturnCode(pid_t pid, int exit_code); - /// Print command name and the list of arguments to log. NOTE: No escaping of arguments is performed. static void logCommand(const char * filename, char * const argv[]); diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 08261fb1cc1..7ee2ee031ae 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -68,6 +68,20 @@ void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) writeSignalIDtoSignalPipe(sig); } +void childSignalHandler(int sig, siginfo_t * info, void *) +{ + DENY_ALLOCATIONS_IN_SCOPE; + auto saved_errno = errno; /// We must restore previous value of errno in signal handler. + + char buf[signal_pipe_buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); + writeBinary(sig, out); + writeBinary(info->si_pid, out); + + out.next(); + errno = saved_errno; +} void signalHandler(int sig, siginfo_t * info, void * context) { @@ -294,6 +308,12 @@ void SignalListener::run() if (daemon) daemon->handleSignal(sig); } + else if (sig == SIGCHLD) + { + pid_t child_pid = 0; + readBinary(child_pid, in); + Context::getGlobalContextInstance()->terminateBackgroundShellCommand(child_pid); + } else { siginfo_t info{}; diff --git a/src/Common/SignalHandlers.h b/src/Common/SignalHandlers.h index e7519f7aee2..0ac6d1d2428 100644 --- a/src/Common/SignalHandlers.h +++ b/src/Common/SignalHandlers.h @@ -33,6 +33,7 @@ void closeLogsSignalHandler(int sig, siginfo_t *, void *); void terminateRequestedSignalHandler(int sig, siginfo_t *, void *); +void childSignalHandler(int sig, siginfo_t * info, void *); /** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 8a8dd3c759c..9cbf9fecdb1 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -440,6 +440,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() HandledSignals::instance().setupCommonDeadlySignalHandlers(); HandledSignals::instance().setupCommonTerminateRequestSignalHandlers(); HandledSignals::instance().addSignalHandler({SIGHUP}, closeLogsSignalHandler, true); + HandledSignals::instance().addSignalHandler({SIGCHLD}, childSignalHandler, true); /// Set up Poco ErrorHandler for Poco Threads. static KillingErrorHandler killing_error_handler; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f0ad013c1d..85fd07360a4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -100,6 +100,7 @@ #include #include #include +#include #include #include #include @@ -540,8 +541,9 @@ struct ContextSharedPart : boost::noncopyable /// No lock required for application_type modified only during initialization Context::ApplicationType application_type = Context::ApplicationType::SERVER; - /// vector of xdbc-bridge commands, they will be killed when Context will be destroyed - std::vector> bridge_commands TSA_GUARDED_BY(mutex); + /// Manager of running background shell commands. + /// They will be killed when Context will be destroyed or with SIGCHLD signal. + BackgroundShellCommandHolder background_active_shell_commands; /// No lock required for config_reload_callback, start_servers_callback, stop_servers_callback modified only during initialization Context::ConfigReloadCallback config_reload_callback; @@ -5067,10 +5069,14 @@ void Context::addQueryParameters(const NameToNameMap & parameters) query_parameters.insert_or_assign(name, value); } -void Context::addBridgeCommand(std::unique_ptr cmd) const +void Context::addBackgroundShellCommand(std::unique_ptr cmd) const { - std::lock_guard lock(shared->mutex); - shared->bridge_commands.emplace_back(std::move(cmd)); + shared->background_active_shell_commands.addCommand(std::move(cmd)); +} + +void Context::terminateBackgroundShellCommand(pid_t pid) const +{ + shared->background_active_shell_commands.removeCommand(pid); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 327ac0af5fd..f18676c1472 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1288,8 +1288,11 @@ public: /// Overrides values of existing parameters. void addQueryParameters(const NameToNameMap & parameters); - /// Add started bridge command. It will be killed after context destruction - void addBridgeCommand(std::unique_ptr cmd) const; + /// Add background shell command. It will be killed after context destruction or with SIGCHLD. + void addBackgroundShellCommand(std::unique_ptr cmd) const; + + /// Terminate background shell command. + void terminateBackgroundShellCommand(pid_t pid) const; IHostContextPtr & getHostContext(); const IHostContextPtr & getHostContext() const; diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 79ec17271be..7a31dfbda5c 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -39,6 +39,13 @@ def create_dict_simple(ch_instance): ) +def check_no_zombie_processes(instance): + res = instance.exec_in_container( + ["bash", "-c", "ps ax -ostat,pid | grep -e '[zZ]' | wc -l"], user="root" + ) + assert res == "0\n" + + @pytest.fixture(scope="module") def ch_cluster(): try: @@ -268,12 +275,8 @@ def test_recover_after_bridge_crash(ch_cluster): instance.exec_in_container( ["bash", "-c", "kill -9 `pidof clickhouse-library-bridge`"], user="root" ) - ## There are no zombie processes. - res = instance.exec_in_container( - ["bash", "-c", "ps ax -ostat,pid | grep -e '[zZ]' | wc -l"], user="root" - ) - assert res == "0\n" + check_no_zombie_processes(instance) instance.query("DROP DICTIONARY lib_dict_c") @@ -299,6 +302,8 @@ def test_server_restart_bridge_might_be_stil_alive(ch_cluster): result = instance.query("""select dictGet(lib_dict_c, 'value1', toUInt64(1));""") assert result.strip() == "101" + check_no_zombie_processes(instance) + instance.query("DROP DICTIONARY lib_dict_c") From b57938df4e8f31304aaebc50821d5338e12db803 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 5 Nov 2024 18:41:47 +0000 Subject: [PATCH 139/433] Style --- src/Common/BackgroundShellCommandHolder.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/BackgroundShellCommandHolder.cpp b/src/Common/BackgroundShellCommandHolder.cpp index 8d239356218..035ab01abbf 100644 --- a/src/Common/BackgroundShellCommandHolder.cpp +++ b/src/Common/BackgroundShellCommandHolder.cpp @@ -30,7 +30,7 @@ void BackgroundShellCommandHolder::addCommand(std::unique_ptr comm auto [iterator, is_inserted] = active_shell_commands.emplace(std::make_pair(command_pid, std::move(command))); if (!is_inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't insert proccess PID {} into active shell commands, because there are running proccess with same PID", command_pid); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't insert process PID {} into active shell commands, because there are running process with same PID", command_pid); LOG_TRACE(getLogger(), "Inserted the command with pid {}", command_pid); } From ca8a93e1ddb68ba15e837b066be1538e7b2a381b Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 6 Nov 2024 08:19:41 +0000 Subject: [PATCH 140/433] Fix tidy --- src/Common/ShellCommand.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index 7e068baab4c..e07b769610f 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -67,7 +67,7 @@ public: DestructorStrategy terminate_in_destructor_strategy = DestructorStrategy(false, 0); }; - pid_t getPid() + pid_t getPid() const { return pid; } From 89096b554a4ec5fc75c924628e0b605938987369 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 6 Nov 2024 10:10:41 +0000 Subject: [PATCH 141/433] Restart Ci From e16c231c580030647e0ec09431cb6f42a04778b0 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 7 Nov 2024 10:52:52 +0000 Subject: [PATCH 142/433] Review --- ...llCommandHolder.cpp => ShellCommandsHolder.cpp} | 14 +++++++------- ...dShellCommandHolder.h => ShellCommandsHolder.h} | 10 +++++----- src/Common/SignalHandlers.cpp | 2 +- src/Interpreters/Context.cpp | 6 +++--- src/Interpreters/Context.h | 2 +- 5 files changed, 17 insertions(+), 17 deletions(-) rename src/Common/{BackgroundShellCommandHolder.cpp => ShellCommandsHolder.cpp} (57%) rename src/Common/{BackgroundShellCommandHolder.h => ShellCommandsHolder.h} (61%) diff --git a/src/Common/BackgroundShellCommandHolder.cpp b/src/Common/ShellCommandsHolder.cpp similarity index 57% rename from src/Common/BackgroundShellCommandHolder.cpp rename to src/Common/ShellCommandsHolder.cpp index 035ab01abbf..3b66c6ba5fc 100644 --- a/src/Common/BackgroundShellCommandHolder.cpp +++ b/src/Common/ShellCommandsHolder.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { @@ -10,25 +10,25 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -LoggerPtr BackgroundShellCommandHolder::getLogger() +LoggerPtr ShellCommandsHolder::getLogger() { - return ::getLogger("BackgroundShellCommandHolder"); + return ::getLogger("ShellCommandsHolder"); } -void BackgroundShellCommandHolder::removeCommand(pid_t pid) +void ShellCommandsHolder::removeCommand(pid_t pid) { std::lock_guard lock(mutex); - bool is_erased = active_shell_commands.erase(pid); + bool is_erased = shell_commands.erase(pid); LOG_TRACE(getLogger(), "Try to erase command with the pid {}, is_erased: {}", pid, is_erased); } -void BackgroundShellCommandHolder::addCommand(std::unique_ptr command) +void ShellCommandsHolder::addCommand(std::unique_ptr command) { std::lock_guard lock(mutex); pid_t command_pid = command->getPid(); - auto [iterator, is_inserted] = active_shell_commands.emplace(std::make_pair(command_pid, std::move(command))); + auto [iterator, is_inserted] = shell_commands.emplace(std::make_pair(command_pid, std::move(command))); if (!is_inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't insert process PID {} into active shell commands, because there are running process with same PID", command_pid); diff --git a/src/Common/BackgroundShellCommandHolder.h b/src/Common/ShellCommandsHolder.h similarity index 61% rename from src/Common/BackgroundShellCommandHolder.h rename to src/Common/ShellCommandsHolder.h index 58bbbefcec1..95db7622ebd 100644 --- a/src/Common/BackgroundShellCommandHolder.h +++ b/src/Common/ShellCommandsHolder.h @@ -1,9 +1,9 @@ #pragma once +#include +#include #include #include - -#include #include @@ -12,17 +12,17 @@ namespace DB /** The holder class for running background shell processes. */ -class BackgroundShellCommandHolder final +class ShellCommandsHolder final : public boost::noncopyable { public: void removeCommand(pid_t pid); void addCommand(std::unique_ptr command); private: - using ActiveShellCommandsCollection = std::unordered_map>; + using ShellCommands = std::unordered_map>; std::mutex mutex; - ActiveShellCommandsCollection active_shell_commands TSA_GUARDED_BY(mutex); + ShellCommands shell_commands TSA_GUARDED_BY(mutex); static LoggerPtr getLogger(); }; diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 7ee2ee031ae..440cfe0b57e 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -312,7 +312,7 @@ void SignalListener::run() { pid_t child_pid = 0; readBinary(child_pid, in); - Context::getGlobalContextInstance()->terminateBackgroundShellCommand(child_pid); + Context::getGlobalContextInstance()->removeBackgroundShellCommand(child_pid); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 85fd07360a4..951571dd1db 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -100,7 +100,7 @@ #include #include #include -#include +#include #include #include #include @@ -543,7 +543,7 @@ struct ContextSharedPart : boost::noncopyable /// Manager of running background shell commands. /// They will be killed when Context will be destroyed or with SIGCHLD signal. - BackgroundShellCommandHolder background_active_shell_commands; + ShellCommandsHolder background_active_shell_commands; /// No lock required for config_reload_callback, start_servers_callback, stop_servers_callback modified only during initialization Context::ConfigReloadCallback config_reload_callback; @@ -5074,7 +5074,7 @@ void Context::addBackgroundShellCommand(std::unique_ptr cmd) const shared->background_active_shell_commands.addCommand(std::move(cmd)); } -void Context::terminateBackgroundShellCommand(pid_t pid) const +void Context::removeBackgroundShellCommand(pid_t pid) const { shared->background_active_shell_commands.removeCommand(pid); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f18676c1472..51f6e299c95 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1292,7 +1292,7 @@ public: void addBackgroundShellCommand(std::unique_ptr cmd) const; /// Terminate background shell command. - void terminateBackgroundShellCommand(pid_t pid) const; + void removeBackgroundShellCommand(pid_t pid) const; IHostContextPtr & getHostContext(); const IHostContextPtr & getHostContext() const; From ea375aecb3f83da1b2f3245dc33e0f91b2adb784 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 7 Nov 2024 10:53:38 +0000 Subject: [PATCH 143/433] Increase timeout --- tests/integration/test_library_bridge/test_exiled.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_library_bridge/test_exiled.py b/tests/integration/test_library_bridge/test_exiled.py index 46a6ac1eaa4..b6d31a218fd 100644 --- a/tests/integration/test_library_bridge/test_exiled.py +++ b/tests/integration/test_library_bridge/test_exiled.py @@ -71,13 +71,13 @@ def test_bridge_dies_with_parent(ch_cluster): except: pass - for i in range(30): + for i in range(60): time.sleep(1) clickhouse_pid = instance.get_process_pid("clickhouse server") if clickhouse_pid is None: break - for i in range(30): + for i in range(60): time.sleep(1) bridge_pid = instance.get_process_pid("library-bridge") if bridge_pid is None: @@ -95,5 +95,5 @@ def test_bridge_dies_with_parent(ch_cluster): assert clickhouse_pid is None assert bridge_pid is None finally: - instance.start_clickhouse(20) + instance.start_clickhouse(60) instance.query("DROP DICTIONARY lib_dict_c") From d69427c35f20ba739bb514170cb5f916b8aa1852 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 7 Nov 2024 16:24:02 +0000 Subject: [PATCH 144/433] More review --- src/Common/ShellCommand.cpp | 74 ++++++++++++++++++++++++------ src/Common/ShellCommand.h | 20 ++++++++ src/Common/ShellCommandsHolder.cpp | 29 ++++++++++-- 3 files changed, 104 insertions(+), 19 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index 0d41669816c..b540cee9e82 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -61,6 +61,9 @@ LoggerPtr ShellCommand::getLogger() ShellCommand::~ShellCommand() { + if (is_manualy_terminated) + return; + if (wait_called) return; @@ -291,11 +294,45 @@ std::unique_ptr ShellCommand::executeDirect(const ShellCommand::Co return executeImpl(path.data(), argv.data(), config); } +struct ShellCommand::tryWaitResult +{ + bool is_process_terminated = false; + int retcode = -1; +}; int ShellCommand::tryWait() { + return tryWaitImpl(false).retcode; +} + +ShellCommand::tryWaitResult ShellCommand::tryWaitImpl(bool blocking) +{ + LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid); + + ShellCommand::tryWaitResult result; + + int options = ((blocking) ? WNOHANG : 0); + int status = 0; + int waitpid_retcode = -1; + + while (waitpid_retcode < 0) + { + waitpid_retcode = waitpid(pid, &status, options); + + if (blocking && !waitpid_retcode) + { + result.is_process_terminated = false; + return result; + } + if (errno != EINTR) + throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid"); + } + + LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status); + wait_called = true; + result.is_process_terminated = true; in.close(); out.close(); err.close(); @@ -306,19 +343,11 @@ int ShellCommand::tryWait() for (auto & [_, fd] : read_fds) fd.close(); - LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid); - - int status = 0; - while (waitpid(pid, &status, 0) < 0) - { - if (errno != EINTR) - throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid"); - } - - LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status); - if (WIFEXITED(status)) - return WEXITSTATUS(status); + { + result.retcode = WEXITSTATUS(status); + return result; + } if (WIFSIGNALED(status)) throw Exception(ErrorCodes::CHILD_WAS_NOT_EXITED_NORMALLY, "Child process was terminated by signal {}", toString(WTERMSIG(status))); @@ -330,10 +359,8 @@ int ShellCommand::tryWait() } -void ShellCommand::wait() +void ShellCommand::handleProcessRetcode(int retcode) const { - int retcode = tryWait(); - if (retcode != EXIT_SUCCESS) { switch (retcode) @@ -356,5 +383,22 @@ void ShellCommand::wait() } } +bool ShellCommand::waitIfProccesTerminated() +{ + auto proc_status = tryWaitImpl(true); + if (proc_status.is_process_terminated) + { + handleProcessRetcode(proc_status.retcode); + } + return proc_status.is_process_terminated; +} + + +void ShellCommand::wait() +{ + int retcode = tryWaitImpl(false).retcode; + handleProcessRetcode(retcode); +} + } diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index e07b769610f..6a4a8328b82 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -72,6 +72,16 @@ public: return pid; } + bool isWaitCalled() const + { + return wait_called; + } + + void setManuallyTerminated() + { + is_manualy_terminated = true; + } + /// Run the command using /bin/sh -c. /// If terminate_in_destructor is true, send terminate signal in destructor and don't wait process. static std::unique_ptr execute(const Config & config); @@ -86,6 +96,10 @@ public: /// Wait for the process to finish, see the return code. To throw an exception if the process was not completed independently. int tryWait(); + /// Returns if process terminated. + /// If process terminated, then handle return code. + bool waitIfProccesTerminated(); + WriteBufferFromFile in; /// If the command reads from stdin, do not forget to call in.close() after writing all the data there. ReadBufferFromFile out; ReadBufferFromFile err; @@ -97,10 +111,16 @@ private: pid_t pid; Config config; bool wait_called = false; + bool is_manualy_terminated = false; ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd_, const Config & config); bool tryWaitProcessWithTimeout(size_t timeout_in_seconds); + struct tryWaitResult; + + tryWaitResult tryWaitImpl(bool blocking); + + void handleProcessRetcode(int retcode) const; static LoggerPtr getLogger(); diff --git a/src/Common/ShellCommandsHolder.cpp b/src/Common/ShellCommandsHolder.cpp index 3b66c6ba5fc..242c30f1286 100644 --- a/src/Common/ShellCommandsHolder.cpp +++ b/src/Common/ShellCommandsHolder.cpp @@ -27,11 +27,32 @@ void ShellCommandsHolder::addCommand(std::unique_ptr command) { std::lock_guard lock(mutex); pid_t command_pid = command->getPid(); + if (command->waitIfProccesTerminated()) + { + LOG_TRACE(getLogger(), "Pid {} already finished. Do not insert it.", command_pid); + return; + } - auto [iterator, is_inserted] = shell_commands.emplace(std::make_pair(command_pid, std::move(command))); - if (!is_inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't insert process PID {} into active shell commands, because there are running process with same PID", command_pid); + auto [iterator, is_inserted] = shell_commands.try_emplace(command_pid, std::move(command)); + if (is_inserted) + { + LOG_TRACE(getLogger(), "Inserted the command with pid {}", command_pid); + return; + } - LOG_TRACE(getLogger(), "Inserted the command with pid {}", command_pid); + if (iterator->second->isWaitCalled()) + { + iterator->second = std::move(command); + LOG_TRACE(getLogger(), "Replaced the command with pid {}", command_pid); + return; + } + + /// We got two active ShellCommand with the same pid. + /// Probably it is a bug, will try to replace the old shell command with a new one. + + LOG_WARNING(getLogger(), "The PID already presented in active shell commands, will try to replace with a new one."); + + iterator->second->setManuallyTerminated(); + iterator->second = std::move(command); } } From cca7da6664650f2794b63d2b77a4aa6977b75b26 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 7 Nov 2024 18:54:24 +0000 Subject: [PATCH 145/433] Style --- src/Common/ShellCommandsHolder.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Common/ShellCommandsHolder.cpp b/src/Common/ShellCommandsHolder.cpp index 242c30f1286..eb3f981a6e4 100644 --- a/src/Common/ShellCommandsHolder.cpp +++ b/src/Common/ShellCommandsHolder.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - LoggerPtr ShellCommandsHolder::getLogger() { return ::getLogger("ShellCommandsHolder"); From 2e83d0f61fdc6baa6198efaa5ac167ae825cad97 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 14 Nov 2024 09:55:06 +0000 Subject: [PATCH 146/433] Remove holder from config --- src/BridgeHelper/LibraryBridgeHelper.cpp | 3 ++- src/BridgeHelper/XDBCBridgeHelper.h | 3 ++- src/Common/ShellCommand.cpp | 15 +++++++++------ src/Common/ShellCommandsHolder.cpp | 6 ++++++ src/Common/ShellCommandsHolder.h | 2 ++ src/Common/SignalHandlers.cpp | 3 ++- src/Interpreters/Context.cpp | 15 --------------- src/Interpreters/Context.h | 5 ----- 8 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/BridgeHelper/LibraryBridgeHelper.cpp b/src/BridgeHelper/LibraryBridgeHelper.cpp index e9fd1078658..cb675da7fa5 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.cpp +++ b/src/BridgeHelper/LibraryBridgeHelper.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -29,7 +30,7 @@ LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_) void LibraryBridgeHelper::startBridge(std::unique_ptr cmd) const { - getContext()->addBackgroundShellCommand(std::move(cmd)); + ShellCommandsHolder::instance().addCommand(std::move(cmd)); } diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index 0851d1289db..cf1720a29ec 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -144,7 +145,7 @@ protected: void startBridge(std::unique_ptr cmd) const override { - getContext()->addBackgroundShellCommand(std::move(cmd)); + ShellCommandsHolder::instance().addCommand(std::move(cmd)); } diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index b540cee9e82..bef7b7a0e9d 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -302,7 +302,7 @@ struct ShellCommand::tryWaitResult int ShellCommand::tryWait() { - return tryWaitImpl(false).retcode; + return tryWaitImpl(true).retcode; } ShellCommand::tryWaitResult ShellCommand::tryWaitImpl(bool blocking) @@ -311,15 +311,18 @@ ShellCommand::tryWaitResult ShellCommand::tryWaitImpl(bool blocking) ShellCommand::tryWaitResult result; - int options = ((blocking) ? WNOHANG : 0); + int options = ((!blocking) ? WNOHANG : 0); int status = 0; int waitpid_retcode = -1; while (waitpid_retcode < 0) { waitpid_retcode = waitpid(pid, &status, options); - - if (blocking && !waitpid_retcode) + if (waitpid_retcode > 0) + { + break; + } + if (!blocking && !waitpid_retcode) { result.is_process_terminated = false; return result; @@ -385,7 +388,7 @@ void ShellCommand::handleProcessRetcode(int retcode) const bool ShellCommand::waitIfProccesTerminated() { - auto proc_status = tryWaitImpl(true); + auto proc_status = tryWaitImpl(false); if (proc_status.is_process_terminated) { handleProcessRetcode(proc_status.retcode); @@ -396,7 +399,7 @@ bool ShellCommand::waitIfProccesTerminated() void ShellCommand::wait() { - int retcode = tryWaitImpl(false).retcode; + int retcode = tryWaitImpl(true).retcode; handleProcessRetcode(retcode); } diff --git a/src/Common/ShellCommandsHolder.cpp b/src/Common/ShellCommandsHolder.cpp index eb3f981a6e4..1354cd2e4b1 100644 --- a/src/Common/ShellCommandsHolder.cpp +++ b/src/Common/ShellCommandsHolder.cpp @@ -5,6 +5,12 @@ namespace DB { +ShellCommandsHolder & ShellCommandsHolder::instance() +{ + static ShellCommandsHolder instance; + return instance; +} + LoggerPtr ShellCommandsHolder::getLogger() { return ::getLogger("ShellCommandsHolder"); diff --git a/src/Common/ShellCommandsHolder.h b/src/Common/ShellCommandsHolder.h index 95db7622ebd..70934365768 100644 --- a/src/Common/ShellCommandsHolder.h +++ b/src/Common/ShellCommandsHolder.h @@ -15,6 +15,8 @@ namespace DB class ShellCommandsHolder final : public boost::noncopyable { public: + static ShellCommandsHolder & instance(); + void removeCommand(pid_t pid); void addCommand(std::unique_ptr command); diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 440cfe0b57e..63ac2df272b 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -312,7 +313,7 @@ void SignalListener::run() { pid_t child_pid = 0; readBinary(child_pid, in); - Context::getGlobalContextInstance()->removeBackgroundShellCommand(child_pid); + ShellCommandsHolder::instance().removeCommand(child_pid); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 951571dd1db..78d41a336b6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -100,7 +100,6 @@ #include #include #include -#include #include #include #include @@ -541,10 +540,6 @@ struct ContextSharedPart : boost::noncopyable /// No lock required for application_type modified only during initialization Context::ApplicationType application_type = Context::ApplicationType::SERVER; - /// Manager of running background shell commands. - /// They will be killed when Context will be destroyed or with SIGCHLD signal. - ShellCommandsHolder background_active_shell_commands; - /// No lock required for config_reload_callback, start_servers_callback, stop_servers_callback modified only during initialization Context::ConfigReloadCallback config_reload_callback; Context::StartStopServersCallback start_servers_callback; @@ -5069,16 +5064,6 @@ void Context::addQueryParameters(const NameToNameMap & parameters) query_parameters.insert_or_assign(name, value); } -void Context::addBackgroundShellCommand(std::unique_ptr cmd) const -{ - shared->background_active_shell_commands.addCommand(std::move(cmd)); -} - -void Context::removeBackgroundShellCommand(pid_t pid) const -{ - shared->background_active_shell_commands.removeCommand(pid); -} - IHostContextPtr & Context::getHostContext() { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 51f6e299c95..cb8bf9634e2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1288,11 +1288,6 @@ public: /// Overrides values of existing parameters. void addQueryParameters(const NameToNameMap & parameters); - /// Add background shell command. It will be killed after context destruction or with SIGCHLD. - void addBackgroundShellCommand(std::unique_ptr cmd) const; - - /// Terminate background shell command. - void removeBackgroundShellCommand(pid_t pid) const; IHostContextPtr & getHostContext(); const IHostContextPtr & getHostContext() const; From 296b88efdfce55db122648e1d658bfe76873be53 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 20 Nov 2024 15:37:30 +0000 Subject: [PATCH 147/433] Review --- src/Common/ShellCommand.cpp | 2 +- src/Common/ShellCommand.h | 6 +++--- src/Common/ShellCommandsHolder.cpp | 20 +++++++------------- src/Common/ShellCommandsHolder.h | 2 +- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index bef7b7a0e9d..bd2c95629fe 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -61,7 +61,7 @@ LoggerPtr ShellCommand::getLogger() ShellCommand::~ShellCommand() { - if (is_manualy_terminated) + if (do_not_terminate) return; if (wait_called) diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index 6a4a8328b82..7aff975efa6 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -77,9 +77,9 @@ public: return wait_called; } - void setManuallyTerminated() + void setDoNotTerminate() { - is_manualy_terminated = true; + do_not_terminate = true; } /// Run the command using /bin/sh -c. @@ -111,7 +111,7 @@ private: pid_t pid; Config config; bool wait_called = false; - bool is_manualy_terminated = false; + bool do_not_terminate = false; ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd_, const Config & config); diff --git a/src/Common/ShellCommandsHolder.cpp b/src/Common/ShellCommandsHolder.cpp index 1354cd2e4b1..6cc44f27fa7 100644 --- a/src/Common/ShellCommandsHolder.cpp +++ b/src/Common/ShellCommandsHolder.cpp @@ -11,17 +11,11 @@ ShellCommandsHolder & ShellCommandsHolder::instance() return instance; } -LoggerPtr ShellCommandsHolder::getLogger() -{ - return ::getLogger("ShellCommandsHolder"); -} - - void ShellCommandsHolder::removeCommand(pid_t pid) { std::lock_guard lock(mutex); bool is_erased = shell_commands.erase(pid); - LOG_TRACE(getLogger(), "Try to erase command with the pid {}, is_erased: {}", pid, is_erased); + LOG_TRACE(log, "Try to erase command with the pid {}, is_erased: {}", pid, is_erased); } void ShellCommandsHolder::addCommand(std::unique_ptr command) @@ -30,30 +24,30 @@ void ShellCommandsHolder::addCommand(std::unique_ptr command) pid_t command_pid = command->getPid(); if (command->waitIfProccesTerminated()) { - LOG_TRACE(getLogger(), "Pid {} already finished. Do not insert it.", command_pid); + LOG_TRACE(log, "Pid {} already finished. Do not insert it.", command_pid); return; } auto [iterator, is_inserted] = shell_commands.try_emplace(command_pid, std::move(command)); if (is_inserted) { - LOG_TRACE(getLogger(), "Inserted the command with pid {}", command_pid); + LOG_TRACE(log, "Inserted the command with pid {}", command_pid); return; } if (iterator->second->isWaitCalled()) { iterator->second = std::move(command); - LOG_TRACE(getLogger(), "Replaced the command with pid {}", command_pid); + LOG_TRACE(log, "Replaced the command with pid {}", command_pid); return; } - /// We got two active ShellCommand with the same pid. /// Probably it is a bug, will try to replace the old shell command with a new one. + chassert(false); - LOG_WARNING(getLogger(), "The PID already presented in active shell commands, will try to replace with a new one."); + LOG_WARNING(log, "The PID already presented in active shell commands, will try to replace with a new one."); - iterator->second->setManuallyTerminated(); + iterator->second->setDoNotTerminate(); iterator->second = std::move(command); } } diff --git a/src/Common/ShellCommandsHolder.h b/src/Common/ShellCommandsHolder.h index 70934365768..2326d4042bc 100644 --- a/src/Common/ShellCommandsHolder.h +++ b/src/Common/ShellCommandsHolder.h @@ -26,7 +26,7 @@ private: std::mutex mutex; ShellCommands shell_commands TSA_GUARDED_BY(mutex); - static LoggerPtr getLogger(); + LoggerPtr log = getLogger("ShellCommandsHolder"); }; } From 2875cf5230629697b6dd643fdbf2c5f3c8430a74 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Thu, 21 Nov 2024 10:04:10 +0000 Subject: [PATCH 148/433] Fix with new master --- src/Common/SignalHandlers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 63ac2df272b..47c10147ade 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -80,7 +80,7 @@ void childSignalHandler(int sig, siginfo_t * info, void *) writeBinary(sig, out); writeBinary(info->si_pid, out); - out.next(); + out.finalize(); errno = saved_errno; } From 7baaf24fc32264de2a845a17306cece48a618e61 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 21 Nov 2024 12:35:14 +0100 Subject: [PATCH 149/433] Enable setting --- tests/queries/0_stateless/03274_udf_in_join.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03274_udf_in_join.sh b/tests/queries/0_stateless/03274_udf_in_join.sh index 052534d7a3c..a7518a661b9 100755 --- a/tests/queries/0_stateless/03274_udf_in_join.sh +++ b/tests/queries/0_stateless/03274_udf_in_join.sh @@ -8,7 +8,7 @@ $CLICKHOUSE_CLIENT -q " CREATE VIEW v0 AS SELECT 1 AS c0; CREATE FUNCTION ${CLICKHOUSE_DATABASE}_second AS (x, y) -> y; CREATE FUNCTION ${CLICKHOUSE_DATABASE}_equals AS (x, y) -> x = y; - -- SET optimize_rewrite_array_exists_to_has = 1; + SET optimize_rewrite_array_exists_to_has = 1; EXPLAIN SYNTAX SELECT 1 FROM v0 JOIN v0 vx ON ${CLICKHOUSE_DATABASE}_second(v0.c0, vx.c0); -- { serverError INVALID_JOIN_ON_EXPRESSION } EXPLAIN SYNTAX SELECT 1 FROM v0 JOIN v0 vx ON ${CLICKHOUSE_DATABASE}_equals(v0.c0, vx.c0); From 51356a4e5677719162e3ded7dddd19659ccd8120 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 21 Nov 2024 13:18:38 +0100 Subject: [PATCH 150/433] This test is not fast --- tests/queries/0_stateless/03141_fetches_errors_stress.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03141_fetches_errors_stress.sql b/tests/queries/0_stateless/03141_fetches_errors_stress.sql index 0528ab5ba6a..9db355f2f66 100644 --- a/tests/queries/0_stateless/03141_fetches_errors_stress.sql +++ b/tests/queries/0_stateless/03141_fetches_errors_stress.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-fasttest, no-parallel -- Tag no-parallel -- due to failpoints create table data_r1 (key Int, value String) engine=ReplicatedMergeTree('/tables/{database}/data', '{table}') order by tuple(); From c0a6cc14fd54cbea1e4d6eb5c69a8cdc5ff3ec46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 12:53:24 +0000 Subject: [PATCH 151/433] test timeout to 7200 --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d0f62b5d770..9f92f9af7e3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -546,7 +546,7 @@ class CI: JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], run_by_labels=[Tags.libFuzzer], - timeout=5400, + timeout=7200, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, ), From 10cd060fb317e77e2510a5a3d5e537b14b6be0a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 21 Nov 2024 14:25:49 +0100 Subject: [PATCH 152/433] Stop wasting disk space and link time --- src/CMakeLists.txt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 55228b2d1ec..ecd559ebab8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,11 +107,6 @@ list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_dele add_headers_and_sources(clickhouse_compression Compression) -add_headers_and_sources(clickhouse_compression Parsers) -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_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) From 334bf3bfe02cd3b9c19aecd09f6e14f90bf4222f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Nov 2024 13:37:31 +0000 Subject: [PATCH 153/433] Fix calculating dynamic columns sizes on vertical merge --- .../MergeTree/IMergeTreeDataPartWriter.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 4 ++- src/Storages/MergeTree/MergeTask.h | 1 + .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 17 ++++++++++--- .../MergeTree/MergedBlockOutputStream.h | 6 +++-- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + ...amic_column_sizes_vertical_merge.reference | 1 + ...74_dynamic_column_sizes_vertical_merge.sql | 25 +++++++++++++++++++ 9 files changed, 50 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.reference create mode 100644 tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index d6d8cbd115b..878122f6428 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -55,7 +55,7 @@ public: MergeTreeIndexGranularityPtr getIndexGranularity() const { return index_granularity; } - virtual Block getColumnsSample() const = 0; + virtual const Block & getColumnsSample() const = 0; protected: SerializationPtr getSerialization(const String & column_name) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index b978beae14b..abcee67b47c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1167,6 +1167,8 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const ctx->executor.reset(); auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns); global_ctx->checksums_gathered_columns.add(std::move(changed_checksums)); + const auto & columns_sample = ctx->column_to->getColumnsSample().getColumnsWithTypeAndName(); + global_ctx->gathered_columns_samples.insert(global_ctx->gathered_columns_samples.end(), columns_sample.begin(), columns_sample.end()); auto cached_marks = ctx->column_to->releaseCachedMarks(); for (auto & [name, marks] : cached_marks) @@ -1316,7 +1318,7 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical) global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync); else - global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns); + global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns, &global_ctx->gathered_columns_samples); auto cached_marks = global_ctx->to->releaseCachedMarks(); for (auto & [name, marks] : cached_marks) diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 721a2a933e1..3dbc885735c 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -194,6 +194,7 @@ private: NamesAndTypesList merging_columns{}; NamesAndTypesList storage_columns{}; MergeTreeData::DataPart::Checksums checksums_gathered_columns{}; + ColumnsWithTypeAndName gathered_columns_samples{}; IndicesDescription merging_skip_indexes; std::unordered_map skip_indexes_by_column; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 5b869c252c3..2e289ab8324 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -133,7 +133,7 @@ public: void cancel() noexcept override; - Block getColumnsSample() const override { return block_sample; } + const Block & getColumnsSample() const override { return block_sample; } protected: /// Count index_granularity for block and store in `index_granularity` diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 979b4698738..2dcf88e3bc2 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -193,16 +193,18 @@ void MergedBlockOutputStream::finalizePart( const MergeTreeMutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list, - MergeTreeData::DataPart::Checksums * additional_column_checksums) + MergeTreeData::DataPart::Checksums * additional_column_checksums, + ColumnsWithTypeAndName * additional_columns_samples) { - finalizePartAsync(new_part, sync, total_columns_list, additional_column_checksums).finish(); + finalizePartAsync(new_part, sync, total_columns_list, additional_column_checksums, additional_columns_samples).finish(); } MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( const MergeTreeMutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list, - MergeTreeData::DataPart::Checksums * additional_column_checksums) + MergeTreeData::DataPart::Checksums * additional_column_checksums, + ColumnsWithTypeAndName * additional_columns_samples) { /// Finish write and get checksums. MergeTreeData::DataPart::Checksums checksums; @@ -248,7 +250,14 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk()); new_part->index_granularity = writer->getIndexGranularity(); - new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(writer->getColumnsSample()); + + auto columns_sample = writer->getColumnsSample(); + if (additional_columns_samples) + { + for (const auto & column : *additional_columns_samples) + columns_sample.insert(column); + } + new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(columns_sample); if ((*new_part->storage.getSettings())[MergeTreeSetting::enable_index_granularity_compression]) { diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index afa2eaf18ec..0149c0f2101 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -64,13 +64,15 @@ public: const MergeTreeMutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list = nullptr, - MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); + MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr, + ColumnsWithTypeAndName * additional_columns_samples = nullptr); void finalizePart( const MergeTreeMutableDataPartPtr & new_part, bool sync, const NamesAndTypesList * total_columns_list = nullptr, - MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr); + MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr, + ColumnsWithTypeAndName * additional_columns_samples = nullptr); private: /** If `permutation` is given, it rearranges the values in the columns when writing. diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index f2f2f10d6ff..62159a8b7fd 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -30,6 +30,7 @@ public: MergeTreeData::DataPart::Checksums fillChecksums(MergeTreeData::MutableDataPartPtr & new_part, MergeTreeData::DataPart::Checksums & all_checksums); + const Block & getColumnsSample() const { return writer->getColumnsSample(); } void finish(bool sync); void cancel() noexcept override; }; diff --git a/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.reference b/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.reference new file mode 100644 index 00000000000..777c6e539df --- /dev/null +++ b/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.reference @@ -0,0 +1 @@ +test 2000000 70 7 7 diff --git a/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.sql b/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.sql new file mode 100644 index 00000000000..b3b1c080114 --- /dev/null +++ b/tests/queries/0_stateless/03274_dynamic_column_sizes_vertical_merge.sql @@ -0,0 +1,25 @@ +-- Tags: no-random-settings, no-fasttest + +set allow_experimental_dynamic_type = 1; +set allow_experimental_json_type = 1; + + +drop table if exists test; +create table test (d Dynamic, json JSON) engine=MergeTree order by tuple() settings min_rows_for_wide_part=0, min_bytes_for_wide_part=0, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=0; +insert into test select number, '{"a" : 42, "b" : "Hello, World"}' from numbers(1000000); +insert into test select number, '{"a" : 42, "b" : "Hello, World"}' from numbers(1000000); +optimize table test final; + +SELECT + `table`, + sum(rows) AS rows, + floor(sum(data_uncompressed_bytes) / (1024 * 1024)) AS data_size_uncompressed, + floor(sum(data_compressed_bytes) / (1024 * 1024)) AS data_size_compressed, + floor(sum(bytes_on_disk) / (1024 * 1024)) AS total_size_on_disk +FROM system.parts +WHERE active AND (database = currentDatabase()) AND (`table` = 'test') +GROUP BY `table` +ORDER BY `table` ASC; + +drop table test; + From 59572989fd9e56839c5169bd1c95a2f3b418a192 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 21 Nov 2024 13:43:33 +0000 Subject: [PATCH 154/433] Polishing --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f1c0a781502..c255269cf0d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -638,16 +638,14 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (context->canUseParallelReplicasOnFollower() && settings[Setting::parallel_replicas_local_plan] && settings[Setting::parallel_replicas_skip_index_analysis_on_workers]) { + // Skip index analysis and return parts with all marks + // The coordinator will chose ranges to read for workers based on index analysis on its side RangesInDataParts parts_with_ranges; parts_with_ranges.reserve(parts.size()); for (size_t part_index = 0; part_index < parts.size(); ++part_index) { const auto & part = parts[part_index]; - // LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "part {}", part->getNameWithState()); - - MarkRanges ranges; - ranges.emplace_back(0, part->getMarksCount()); - parts_with_ranges.emplace_back(part, part_index, std::move(ranges)); + parts_with_ranges.emplace_back(part, part_index, MarkRanges{{0, part->getMarksCount()}}); } return parts_with_ranges; } From ddb8f3d57f8112748ce1f8bab858997b5d1c8f77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=9A=D0=B8=D1=80=D0=B8=D0=BB=D0=BB=20=D0=93=D0=B0=D1=80?= =?UTF-8?q?=D0=B1=D0=B0=D1=80?= Date: Thu, 21 Nov 2024 16:48:52 +0300 Subject: [PATCH 155/433] Fix docs and build --- docs/en/sql-reference/statements/attach.md | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index ca90165f53b..379d5457ded 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -76,7 +76,7 @@ Allows to attach non-replicated MergeTree table as ReplicatedMergeTree. Replicat Note that table's data in ZooKeeper is not affected in this query. This means you have to add metadata in ZooKeeper using `SYSTEM RESTORE REPLICA` or clear it with `SYSTEM DROP REPLICA ... FROM ZKPATH ...` after attach. -If you are trying to add a replica to an existing ReplicatedMergeTree table, keep in mind that all the data in converted MergeTree table will be replicated as well. +If you are trying to add a replica to an existing ReplicatedMergeTree table, keep in mind that all the local data in converted MergeTree table will be detached. **Syntax** diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a420b4d2094..bb1f30d137a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -2444,7 +2444,7 @@ void InterpreterCreateQuery::convertMergeTreeTableIfPossible(ASTCreateQuery & cr WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL); writeString(statement, out); out.next(); - if (getContext()->getSettingsRef().fsync_metadata) + if (getContext()->getSettingsRef()[Setting::fsync_metadata]) out.sync(); out.close(); } From 99916f85fc1918a624b9577421581fff7db73597 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Nov 2024 14:29:11 +0000 Subject: [PATCH 156/433] fix setting of priamry index --- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 6 +++--- src/Storages/MergeTree/MutateTask.cpp | 4 +++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ab07bbd424e..01359af021e 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -249,9 +249,6 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); - if (auto computed_index = writer->releaseIndexColumns()) - new_part->setIndex(std::move(*computed_index)); - new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk()); @@ -264,6 +261,9 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( new_part->index_granularity = std::move(new_index_granularity); } + if (auto computed_index = writer->releaseIndexColumns()) + new_part->setIndex(std::move(*computed_index)); + /// In mutation, existing_rows_count is already calculated in PartMergerWriter /// In merge situation, lightweight deleted rows was physically deleted, existing_rows_count equals rows_count if (!new_part->existing_rows_count.has_value()) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 750fb91c557..8c8c07fa266 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -985,7 +985,6 @@ void finalizeMutatedPart( new_data_part->rows_count = source_part->rows_count; new_data_part->index_granularity = source_part->index_granularity; - new_data_part->setIndex(*source_part->getIndex()); new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); @@ -995,6 +994,9 @@ void finalizeMutatedPart( new_data_part->index_granularity = std::move(new_index_granularity); } + if (!new_data_part->storage.getPrimaryIndexCache()) + new_data_part->setIndex(*source_part->getIndex()); + /// Load rest projections which are hardlinked bool noop; new_data_part->loadProjections(false, false, noop, true /* if_not_loaded */); From 41279ca59b83b8cc55e5bc105a5894bc3176ea27 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 14:59:07 +0000 Subject: [PATCH 157/433] test timeout to 10800 --- tests/ci/ci_config.py | 2 +- tests/fuzz/runner.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9f92f9af7e3..e10c60135cb 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -546,7 +546,7 @@ class CI: JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], run_by_labels=[Tags.libFuzzer], - timeout=7200, + timeout=10800, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, ), diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index f4c66e00117..9be0af579fb 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -30,7 +30,7 @@ class Stopwatch: def run_fuzzer(fuzzer: str, timeout: int): - logging.info("Running fuzzer %s...", fuzzer) + logging.info("Running fuzzer %s for %d seconds...", fuzzer, timeout) seed_corpus_dir = f"{fuzzer}.in" with Path(seed_corpus_dir) as path: From 04ccefe447630584947a5db4ff2fa7a926b12e53 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 21 Nov 2024 16:02:33 +0100 Subject: [PATCH 158/433] debug --- .../0_stateless/03167_improvement_table_name_too_long.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 48665c92a4e..9819a668e64 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -12,6 +12,7 @@ getconf NAME_MAX /fasttest-workspace/db-fasttest/store long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" -$CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" +# $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" +# $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" +# $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" +# ToHqNPfhZpLevFeVCquBvDPDdVSFxSTRaEkCIPQzhXCVAwbmHXNRJiEFqQGgNzLULAufPDpRIjXoTxYWcdnNStNLdWDIjNiHoKltMpdGbDzeVsXFPIynefYqJPLqCuuuckRpBjOFPGoAKndFzAOGwCcZaIsGFFkOOpPwTesCZfbjvtZWrGYeYqAWJsLPQPIRzFHfebEYLOguMMNjQ From e19ac0aa1181f9215f51935aa8e4fc9a7ec607aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 21 Nov 2024 16:27:51 +0100 Subject: [PATCH 159/433] Remove duplicate building and linking of common files --- src/CMakeLists.txt | 3 +-- src/Common/Config/CMakeLists.txt | 9 --------- src/Common/ZooKeeper/CMakeLists.txt | 23 ++++++++++++++--------- 3 files changed, 15 insertions(+), 20 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ecd559ebab8..ac03f40cd93 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -109,7 +109,6 @@ list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_dele add_headers_and_sources(clickhouse_compression Compression) add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources}) - add_headers_and_sources(dbms Disks/IO) add_headers_and_sources(dbms Disks/ObjectStorages) if (TARGET ch_contrib::sqlite) @@ -217,7 +216,6 @@ add_object_library(clickhouse_access Access) add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) -add_object_library(clickhouse_compression Compression) add_object_library(clickhouse_querypipeline QueryPipeline) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) @@ -424,6 +422,7 @@ dbms_target_link_libraries ( Poco::JSON PUBLIC boost::system + clickhouse_compression clickhouse_common_io Poco::Redis ) diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index 2bd32b98bda..e91a01568d5 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -16,15 +16,6 @@ target_link_libraries(clickhouse_common_config Poco::XML ) -add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) -target_link_libraries(clickhouse_common_config_no_zookeeper_log - PUBLIC - clickhouse_common_zookeeper_no_log - common - Poco::XML -) - if (TARGET ch_contrib::yaml_cpp) target_link_libraries(clickhouse_common_config PRIVATE ch_contrib::yaml_cpp) - target_link_libraries(clickhouse_common_config_no_zookeeper_log PRIVATE ch_contrib::yaml_cpp) endif() diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index 8b6c420e565..b70a2299ba6 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -2,25 +2,30 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) -list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) +# Needs to be built differently depending on ZOOKEEPER_LOG +list(REMOVE_ITEM clickhouse_common_zookeeper_sources "ZooKeeperImpl.cpp") -# for clickhouse server -add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) -target_link_libraries (clickhouse_common_zookeeper +add_library(clickhouse_common_zookeeper_base ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) +target_link_libraries (clickhouse_common_zookeeper_base PUBLIC clickhouse_common_io clickhouse_compression common ) +# for clickhouse server +add_library(clickhouse_common_zookeeper ZooKeeperImpl.cpp) +target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) +target_link_libraries (clickhouse_common_zookeeper + PUBLIC + clickhouse_common_zookeeper_base +) + # for examples -- no logging (to avoid extra dependencies) -add_library(clickhouse_common_zookeeper_no_log ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) +add_library(clickhouse_common_zookeeper_no_log ZooKeeperImpl.cpp) target_link_libraries (clickhouse_common_zookeeper_no_log PUBLIC - clickhouse_common_io - clickhouse_compression - common + clickhouse_common_zookeeper_base ) if (ENABLE_EXAMPLES) add_subdirectory(examples) From a329ea7768acab65b920dd5f41a7175ef751bab8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Nov 2024 16:03:35 +0000 Subject: [PATCH 160/433] fix test for lazy index load --- .../03128_merge_tree_index_lazy_load.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03128_merge_tree_index_lazy_load.reference b/tests/queries/0_stateless/03128_merge_tree_index_lazy_load.reference index 022457178ec..5022e543cd2 100644 --- a/tests/queries/0_stateless/03128_merge_tree_index_lazy_load.reference +++ b/tests/queries/0_stateless/03128_merge_tree_index_lazy_load.reference @@ -1,8 +1,8 @@ 0 0 0 -1 4 4 -2 8 8 -3 9 9 -0 0 0 +1 4 0 +2 8 0 +3 9 0 +0 0 0 1 4 0 2 8 0 3 9 0 From ad97f0440b6e8ace7e5222daeb2e2423dcf3b8b1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 16:59:31 +0000 Subject: [PATCH 161/433] set timeout as a libfuzzer argument and hard timeout +60seconds --- tests/fuzz/runner.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 9be0af579fb..159f4e7f990 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -30,7 +30,8 @@ class Stopwatch: def run_fuzzer(fuzzer: str, timeout: int): - logging.info("Running fuzzer %s for %d seconds...", fuzzer, timeout) + timeout_hard = timeout + 60 + logging.info("Running fuzzer %s for %d seconds (hard timeout is %d)...", fuzzer, timeout, timeout_hard) seed_corpus_dir = f"{fuzzer}.in" with Path(seed_corpus_dir) as path: @@ -89,6 +90,8 @@ def run_fuzzer(fuzzer: str, timeout: int): custom_libfuzzer_options += f" -dict={fuzzer}.dict" custom_libfuzzer_options += f" -exact_artifact_path={exact_artifact_path}" + custom_libfuzzer_options += f" -timeout={timeout}" + libfuzzer_corpora = f"{active_corpus_dir} {seed_corpus_dir}" cmd_line = f"{DEBUGGER} ./{fuzzer} {fuzzer_arguments}" @@ -115,7 +118,7 @@ def run_fuzzer(fuzzer: str, timeout: int): check=True, shell=False, errors="replace", - timeout=timeout, + timeout=timeout_hard, env=env, ) except subprocess.CalledProcessError: From ee1cd0acf3275bfd836fd657125f72b9402b931a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 21 Nov 2024 17:59:31 +0100 Subject: [PATCH 162/433] fix style --- .../0_stateless/03167_improvement_table_name_too_long.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 9819a668e64..03c8e9bd834 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -8,9 +8,8 @@ allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLeng let excess_length=allowed_name_length+1 getconf NAME_MAX /fasttest-workspace/db-fasttest/store - -long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) -allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) +# long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) +# allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) # $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" # $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" From 0e6284f7bc1538c4a9414f32665c80a9e0167ba4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 21 Nov 2024 17:14:19 +0000 Subject: [PATCH 163/433] Automatic style fix --- tests/fuzz/runner.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/fuzz/runner.py b/tests/fuzz/runner.py index 159f4e7f990..470d07b2bbe 100644 --- a/tests/fuzz/runner.py +++ b/tests/fuzz/runner.py @@ -31,7 +31,12 @@ class Stopwatch: def run_fuzzer(fuzzer: str, timeout: int): timeout_hard = timeout + 60 - logging.info("Running fuzzer %s for %d seconds (hard timeout is %d)...", fuzzer, timeout, timeout_hard) + logging.info( + "Running fuzzer %s for %d seconds (hard timeout is %d)...", + fuzzer, + timeout, + timeout_hard, + ) seed_corpus_dir = f"{fuzzer}.in" with Path(seed_corpus_dir) as path: From 625cb7012873df9e867e03cad2af97613a5aaf3f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 21 Nov 2024 18:18:49 +0100 Subject: [PATCH 164/433] debug --- .../0_stateless/03167_improvement_table_name_too_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 03c8e9bd834..afcc3dee4b6 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -4,8 +4,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") -let excess_length=allowed_name_length+1 +# allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") +# let excess_length=allowed_name_length+1 getconf NAME_MAX /fasttest-workspace/db-fasttest/store # long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) From c69de20aceadd8ed0c01be164eeafffef7f0f0f1 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 21 Nov 2024 19:03:04 +0100 Subject: [PATCH 165/433] add the right include --- src/Functions/getMaxTableNameLength.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/getMaxTableNameLength.cpp b/src/Functions/getMaxTableNameLength.cpp index 5b84f88c1a9..4e74da8204d 100644 --- a/src/Functions/getMaxTableNameLength.cpp +++ b/src/Functions/getMaxTableNameLength.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include From ac881626dc0d9e9607ce56c853a6a4af780643d8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Nov 2024 19:36:43 +0100 Subject: [PATCH 166/433] Update 03167_attach_as_replicated_errors.sh --- tests/queries/0_stateless/03167_attach_as_replicated_errors.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index b8df052edb3..42aa122709c 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database, no-ordinary-database +# Tags: zookeeper, no-replicated-database, no-ordinary-database, no-shared-merge-tree CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4fc776779e849a9aab7028d5ffa21a45fc25a481 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 19:19:02 +0000 Subject: [PATCH 167/433] ENABLE_CHECK_HEAVY_BUILDS 0 for fuzzers --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index a165be799c0..b5a448099c2 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -105,7 +105,7 @@ if (ENABLE_FUZZING) set (ENABLE_THINLTO 0) set (ENABLE_TCMALLOC 0) set (ENABLE_JEMALLOC 0) - set (ENABLE_CHECK_HEAVY_BUILDS 1) + set (ENABLE_CHECK_HEAVY_BUILDS 0) set (GLIBC_COMPATIBILITY OFF) set (ENABLE_BENCHMARKS 0) From 24218238b42166698d427e0ded6fe49d30b023be Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 22:22:55 +0000 Subject: [PATCH 168/433] increase build limits for fuzzers --- CMakeLists.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index b5a448099c2..1b101abb62c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -63,9 +63,9 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # Sanitizers are too heavy. Some architectures too. if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64 OR ARCH_LOONGARCH64) - # Twice as large - set (RLIMIT_DATA 10000000000) - set (RLIMIT_AS 20000000000) + # Three times as large + set (RLIMIT_DATA 15000000000) + set (RLIMIT_AS 30000000000) endif() # For some files currently building RISCV64/LOONGARCH64 might be too slow. @@ -105,7 +105,7 @@ if (ENABLE_FUZZING) set (ENABLE_THINLTO 0) set (ENABLE_TCMALLOC 0) set (ENABLE_JEMALLOC 0) - set (ENABLE_CHECK_HEAVY_BUILDS 0) + set (ENABLE_CHECK_HEAVY_BUILDS 1) set (GLIBC_COMPATIBILITY OFF) set (ENABLE_BENCHMARKS 0) From e5cfc7daee6e376c9f635ac4033ddf57554ee469 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 21 Nov 2024 23:48:45 +0100 Subject: [PATCH 169/433] impl --- src/Interpreters/GraceHashJoin.cpp | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 3fb83c3ce47..2cda7d4d856 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -13,6 +13,7 @@ #include #include +#include #include @@ -524,6 +525,7 @@ public: Block nextImpl() override { ExtraBlockPtr not_processed = nullptr; + std::shared_lock shared(eof_mutex); { std::lock_guard lock(extra_block_mutex); @@ -557,7 +559,24 @@ public: block = left_reader.read(); if (!block) { - return {}; + shared.unlock(); + bool there_are_still_might_be_rows_to_process = false; + { + /// The following race condition could happen without this mutex: + /// * we're called from `IBlocksStream::next()` + /// * another thread just read the last block from `left_reader` and now is in the process of or about to call `joinBlock()` + /// * it might be that `joinBlock()` will leave some rows in the `not_processed` + /// * but if the current thread will return now an empty block `finished` will be set to true in `IBlocksStream::next()` and + /// these not processed rows will be lost + /// So we shouldn't finish execution while there is at least one in-flight `joinBlock()` call. Let's wait until we're alone + /// and double check if there are any not processed rows left. + std::unique_lock exclusive(eof_mutex); + + std::lock_guard lock(extra_block_mutex); + if (!not_processed_blocks.empty()) + there_are_still_might_be_rows_to_process = true; + } + return there_are_still_might_be_rows_to_process ? nextImpl() : Block(); } // block comes from left_reader, need to join with right table to get the result. @@ -592,7 +611,7 @@ public: return block; } - size_t current_bucket; + const size_t current_bucket; Buckets buckets; InMemoryJoinPtr hash_join; @@ -603,6 +622,8 @@ public: std::mutex extra_block_mutex; std::list not_processed_blocks TSA_GUARDED_BY(extra_block_mutex); + + std::shared_mutex eof_mutex; }; IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() From f20e494a8d56d031b18ec492293c99ead8f79387 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 21 Nov 2024 23:19:58 +0000 Subject: [PATCH 170/433] another try --- CMakeLists.txt | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 1b101abb62c..03c20325f1e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -63,9 +63,10 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # Sanitizers are too heavy. Some architectures too. if (SANITIZE OR SANITIZE_COVERAGE OR WITH_COVERAGE OR ARCH_RISCV64 OR ARCH_LOONGARCH64) - # Three times as large - set (RLIMIT_DATA 15000000000) - set (RLIMIT_AS 30000000000) + # Twice as large + set (RLIMIT_DATA 10000000000) + set (RLIMIT_AS 20000000000) + set (RLIMIT_CPU 2000) endif() # For some files currently building RISCV64/LOONGARCH64 might be too slow. From ea31e2775e6f4fd860a74b993200cfed2c124766 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 22 Nov 2024 01:19:31 +0000 Subject: [PATCH 171/433] try without ENABLE_CHECK_HEAVY_BUILDS --- docker/packager/binary-builder/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary-builder/build.sh b/docker/packager/binary-builder/build.sh index f18a6c2a798..b38d204d123 100755 --- a/docker/packager/binary-builder/build.sh +++ b/docker/packager/binary-builder/build.sh @@ -91,7 +91,7 @@ then fi # Build everything -cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS_ARRAY[@]}" .. +cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=0 "${CMAKE_FLAGS_ARRAY[@]}" .. # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. From 30625c78e1609098bb6bfc6b3c90e9d6f9450d05 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 22 Nov 2024 02:52:55 +0000 Subject: [PATCH 172/433] trigger build --- src/DataTypes/fuzzers/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/fuzzers/CMakeLists.txt b/src/DataTypes/fuzzers/CMakeLists.txt index 8dedd3470e2..8940586fc70 100644 --- a/src/DataTypes/fuzzers/CMakeLists.txt +++ b/src/DataTypes/fuzzers/CMakeLists.txt @@ -1,2 +1,3 @@ clickhouse_add_executable(data_type_deserialization_fuzzer data_type_deserialization_fuzzer.cpp ${SRCS}) + target_link_libraries(data_type_deserialization_fuzzer PRIVATE clickhouse_aggregate_functions dbms) From ed45abbc1a2dc621b96f37584abfae41e55ebc98 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 11:42:21 +0100 Subject: [PATCH 173/433] add test --- ..._hash_max_joined_block_size_rows_bug.reference | 1 + ..._grace_hash_max_joined_block_size_rows_bug.sql | 15 +++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.reference create mode 100644 tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.sql diff --git a/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.reference b/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.reference new file mode 100644 index 00000000000..8b3a805c6a2 --- /dev/null +++ b/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.reference @@ -0,0 +1 @@ +180 15 diff --git a/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.sql b/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.sql new file mode 100644 index 00000000000..c91e226fdef --- /dev/null +++ b/tests/queries/0_stateless/03274_grace_hash_max_joined_block_size_rows_bug.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; + +CREATE TABLE t0 (x UInt64) ENGINE = MergeTree ORDER BY x; +INSERT INTO t0 SELECT number from numbers(20); + +CREATE TABLE t1 (x UInt64) ENGINE = MergeTree ORDER BY x; +INSERT INTO t1 SELECT number from numbers(5, 20); + +SET max_joined_block_size_rows = 1; +SET grace_hash_join_initial_buckets = 2; +SET join_algorithm = 'grace_hash'; + +SELECT sum(x), count() FROM t0 JOIN t1 USING x; + From db2be8c91e70e79163419d8ec76caf078f00c98d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 21 Nov 2024 20:22:32 +0100 Subject: [PATCH 174/433] Initial 24.11 changelog --- CHANGELOG.md | 695 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 695 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index dacee73440f..8480b5fc457 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v24.11, 2024-11-26](#2411)**
**[ClickHouse release v24.10, 2024-10-31](#2410)**
**[ClickHouse release v24.9, 2024-09-26](#249)**
**[ClickHouse release v24.8 LTS, 2024-08-20](#248)**
@@ -13,6 +14,700 @@ # 2024 Changelog +###
ClickHouse release 24.11, 2024-11-26 + +#### Backward Incompatible Change +* Remove system tables `generate_series` and `generateSeries`. They were added by mistake here: [#59390](https://github.com/ClickHouse/ClickHouse/issues/59390). [#71091](https://github.com/ClickHouse/ClickHouse/pull/71091) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600). ### Documentation entry for user-facing changes. [#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). +* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). +* The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the `SOURCES` hierarchy. Add grants to any non-default database users that create tables with these engine types. [#71250](https://github.com/ClickHouse/ClickHouse/pull/71250) ([Christoph Wurm](https://github.com/cwurm)). +* Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. [#71300](https://github.com/ClickHouse/ClickHouse/pull/71300) ([Christoph Wurm](https://github.com/cwurm)). +* Rename filesystem cache setting `skip_download_if_exceeds_query_cache` to `filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit`. [#71578](https://github.com/ClickHouse/ClickHouse/pull/71578) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for `Enum` as well as `UInt128` and `UInt256` arguments in `deltaSumTimestamp`. Remove support for `Int8`, `UInt8`, `Int16`, and `UInt16` of the second ("timestamp") argument of `deltaSumTimestamp`. [#71790](https://github.com/ClickHouse/ClickHouse/pull/71790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ~~Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled by `query_plan_join_inner_table_selection`, which can be set to `auto`. In this mode, ClickHouse will try to choose the table with the smallest number of rows.~~ Resubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. [#68682](https://github.com/ClickHouse/ClickHouse/pull/68682) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). +* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). +* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). +* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). +* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). +* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). +* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). +* Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). +* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). +* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). +* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). +* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). +* Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). + +#### Performance Improvement +* Add 2 new settings `short_circuit_function_evaluation_for_nulls` and `short_circuit_function_evaluation_for_nulls_threshold` that allow to execute functions over `Nullable` columns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. [#60129](https://github.com/ClickHouse/ClickHouse/pull/60129) ([李扬](https://github.com/taiyang-li)). +* Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). +* Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Improvement +* Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). +* Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). +* Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). +* Fix use-after-dtor logic in HashTable destroyElements. [#65279](https://github.com/ClickHouse/ClickHouse/pull/65279) ([cangyin](https://github.com/cangyin)). +* Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). +* Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). +* 1. Refactor `DDLQueryStatusSource`: * Rename `DDLQueryStatusSource` to `DistributedQueryStatusSource`, and make it a base class * Create two subclasses `DDLOnClusterQueryStatusSource` and `ReplicatedDatabaseQueryStatusSource` derived from `DDLQueryStatusSource` to query the status of DDL tasks from `DDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in `DDLOnClusterQueryStatusSource`. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). +* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). +* Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). +* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). +* Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). +* Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). +* Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). +* Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). +* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). +* Boolean support for parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). +* Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). +* `system.session_log` is quite okay. This closes [#51760](https://github.com/ClickHouse/ClickHouse/issues/51760). [#71150](https://github.com/ClickHouse/ClickHouse/pull/71150) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). +* Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). +* Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add per host dashboards `Overview (host)` and `Cloud overview (host)` to advanced dashboard. [#71422](https://github.com/ClickHouse/ClickHouse/pull/71422) ([alesapin](https://github.com/alesapin)). +* The methods `removeObject` and `removeObjects` are not idempotent. When retries happen due to network errors, the result could be `object not found` because it has been deleted at previous attempts. [#71529](https://github.com/ClickHouse/ClickHouse/pull/71529) ([Sema Checherinda](https://github.com/CheSema)). +* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). +* Allow using clickhouse with a file argument as --queries-file. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). +* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). +* `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. ### Documentation entry for user-facing changes. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). +* Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). +* Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). +* Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). +* Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Turn-off filesystem cache setting `boundary_alignment` for non-disk read. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). +* Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). +* Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). +* Do not increment the `ILLEGAL_TYPE_OF_ARGUMENT` counter in the `system.errors` table when the `bitmapTransform` function is used, and argument types are valid. [#71971](https://github.com/ClickHouse/ClickHouse/pull/71971) ([Dmitry Novik](https://github.com/novikd)). +* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). +* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). +* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). +* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). +* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). +* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). +* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). +* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). +* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). +* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). +* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). +* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). +* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). +* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). +* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). +* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). +* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). +* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). +* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). +* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). +* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). +* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). +* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). +* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). +* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). +* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). +* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). +* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). +* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). +* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). +* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). +* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). +* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). +* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### Build/Testing/Packaging Improvement +* Add the script to update sources of [docker official library](https://github.com/ClickHouse/docker-library). [#57203](https://github.com/ClickHouse/ClickHouse/pull/57203) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The build system will prevent libraries with unexpected licenses. [#70988](https://github.com/ClickHouse/ClickHouse/pull/70988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Upgrade docker base image for clickhouse-server and keeper to `ubuntu:22.04`. **Breaking change**: the minimal supported docker version is `20.10.10`. [#71505](https://github.com/ClickHouse/ClickHouse/pull/71505) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve clickhouse-server Dockerfile.ubuntu. Deprecate `CLICKHOUSE_UID/CLICKHOUSE_GID` envs. Remove `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` processing to complien requirements. Consistent `clickhouse/clickhouse-server/clickhouse-keeper` execution to not have it plain in one place and `/usr/bin/clickhouse*` in another. [#71573](https://github.com/ClickHouse/ClickHouse/pull/71573) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Miscellaneous"'. [#71083](https://github.com/ClickHouse/ClickHouse/pull/71083) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. [#71084](https://github.com/ClickHouse/ClickHouse/pull/71084) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. [#71251](https://github.com/ClickHouse/ClickHouse/pull/71251) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. [#71266](https://github.com/ClickHouse/ClickHouse/pull/71266) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Selection of hash join inner table"'. [#71527](https://github.com/ClickHouse/ClickHouse/pull/71527) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. [#71619](https://github.com/ClickHouse/ClickHouse/pull/71619) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. [#71914](https://github.com/ClickHouse/ClickHouse/pull/71914) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. [#71945](https://github.com/ClickHouse/ClickHouse/pull/71945) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. [#71974](https://github.com/ClickHouse/ClickHouse/pull/71974) ([Max Kainov](https://github.com/maxknv)). +* NO CL ENTRY: 'CI: Functional Tests with praktika'. [#71976](https://github.com/ClickHouse/ClickHouse/pull/71976) ([Max Kainov](https://github.com/maxknv)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Refactor TempDataOnDisk. [#66606](https://github.com/ClickHouse/ClickHouse/pull/66606) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Bump krb5 from v1.21.2 to v1.21.3. [#69360](https://github.com/ClickHouse/ClickHouse/pull/69360) ([Robert Schulze](https://github.com/rschu1ze)). +* USearch: Enable SimSIMD backend + enable dynamic dispatch. [#69387](https://github.com/ClickHouse/ClickHouse/pull/69387) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: enable libfuzzer. [#70112](https://github.com/ClickHouse/ClickHouse/pull/70112) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Use `clang-19`. [#70414](https://github.com/ClickHouse/ClickHouse/pull/70414) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Randomize Keeper feature flags in integration tests. [#70523](https://github.com/ClickHouse/ClickHouse/pull/70523) ([Antonio Andelic](https://github.com/antonio2368)). +* All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. [#70691](https://github.com/ClickHouse/ClickHouse/pull/70691) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). +* Add a settings `filesystem_cache_enable_background_download_for_metadata_files` for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. [#70806](https://github.com/ClickHouse/ClickHouse/pull/70806) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make ParquetMetadata say whether bloom filter is present. [#70947](https://github.com/ClickHouse/ClickHouse/pull/70947) ([Michael Kolupaev](https://github.com/al13n321)). +* test for reproducing that ReplacingMergeTree depends on the order of part attachment. [#71010](https://github.com/ClickHouse/ClickHouse/pull/71010) ([Konstantin Morozov](https://github.com/k-morozov)). +* CI: Build Job with praktika. [#71015](https://github.com/ClickHouse/ClickHouse/pull/71015) ([Max Kainov](https://github.com/maxknv)). +* Fix bad test `01524_do_not_merge_across_partitions_select_final.sql`. [#71035](https://github.com/ClickHouse/ClickHouse/pull/71035) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable enable_job_stack_trace by default. [#71039](https://github.com/ClickHouse/ClickHouse/pull/71039) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix two logical errors when reading from stdin in clickhouse local. [#71046](https://github.com/ClickHouse/ClickHouse/pull/71046) ([Michael Kolupaev](https://github.com/al13n321)). +* Sync changes to `ProtocolServerAdapter`. [#71058](https://github.com/ClickHouse/ClickHouse/pull/71058) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a typo. [#71067](https://github.com/ClickHouse/ClickHouse/pull/71067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71070](https://github.com/ClickHouse/ClickHouse/pull/71070) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove bad test `test_system_replicated_fetches`. [#71071](https://github.com/ClickHouse/ClickHouse/pull/71071) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version after release. [#71076](https://github.com/ClickHouse/ClickHouse/pull/71076) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* CI: Functional tests for ARM + ASAN binary. [#71079](https://github.com/ClickHouse/ClickHouse/pull/71079) ([Max Kainov](https://github.com/maxknv)). +* CI: Functional Tests with praktika. [#71081](https://github.com/ClickHouse/ClickHouse/pull/71081) ([Max Kainov](https://github.com/maxknv)). +* Fixup of TrivialMergeSelector. [#71082](https://github.com/ClickHouse/ClickHouse/pull/71082) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Sync integration test with private. [#71096](https://github.com/ClickHouse/ClickHouse/pull/71096) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unique symbols in the `system.coverage_log`. [#71099](https://github.com/ClickHouse/ClickHouse/pull/71099) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better log messages. [#71102](https://github.com/ClickHouse/ClickHouse/pull/71102) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix error in Replicated database. [#71103](https://github.com/ClickHouse/ClickHouse/pull/71103) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update error message for JSONAsObject format. [#71123](https://github.com/ClickHouse/ClickHouse/pull/71123) ([Pavel Kruglov](https://github.com/Avogar)). +* Initial changelog for 24.10. [#71127](https://github.com/ClickHouse/ClickHouse/pull/71127) ([Raúl Marín](https://github.com/Algunenano)). +* Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). +* Try fix rabbitmq. [#71143](https://github.com/ClickHouse/ClickHouse/pull/71143) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Randomize setting `enable_vertical_final`. [#71144](https://github.com/ClickHouse/ClickHouse/pull/71144) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bad test `02561_sorting_constants_and_distinct_crash`. [#71147](https://github.com/ClickHouse/ClickHouse/pull/71147) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test to verify [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308) works. [#71149](https://github.com/ClickHouse/ClickHouse/pull/71149) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix 02932_refreshable_materialized_views_1 flakiness. [#71160](https://github.com/ClickHouse/ClickHouse/pull/71160) ([Michael Kolupaev](https://github.com/al13n321)). +* Use `_minmax_count_projection` instead of `Optimized trivial count` for `ReadFromPreparedSource` node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses [#70939](https://github.com/ClickHouse/ClickHouse/issues/70939). [#71166](https://github.com/ClickHouse/ClickHouse/pull/71166) ([Amos Bird](https://github.com/amosbird)). +* Close [#8687](https://github.com/ClickHouse/ClickHouse/issues/8687). [#71169](https://github.com/ClickHouse/ClickHouse/pull/71169) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). +* Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). +* Print compression method in `clickhouse-compressor --stat`. Useful for inspecting random data files. [#71192](https://github.com/ClickHouse/ClickHouse/pull/71192) ([Amos Bird](https://github.com/amosbird)). +* Updating the events into the recent category and adding the new york event. [#71194](https://github.com/ClickHouse/ClickHouse/pull/71194) ([Zoe Steinkamp](https://github.com/zoesteinkamp)). +* Improve error and log messages around memory usage. [#71195](https://github.com/ClickHouse/ClickHouse/pull/71195) ([Raúl Marín](https://github.com/Algunenano)). +* Minor test adjustments. [#71199](https://github.com/ClickHouse/ClickHouse/pull/71199) ([Raúl Marín](https://github.com/Algunenano)). +* Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). +* Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#71223](https://github.com/ClickHouse/ClickHouse/pull/71223) ([Tanya Bragin](https://github.com/tbragin)). +* Fix `WITH TOTALS` in subquery with parallel replicas. [#71224](https://github.com/ClickHouse/ClickHouse/pull/71224) ([Nikita Taranov](https://github.com/nickitat)). +* Ignore `No such key` exceptions in some cases. [#71236](https://github.com/ClickHouse/ClickHouse/pull/71236) ([Antonio Andelic](https://github.com/antonio2368)). +* Make cloud sync title shorter. [#71255](https://github.com/ClickHouse/ClickHouse/pull/71255) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Meetups update. [#71271](https://github.com/ClickHouse/ClickHouse/pull/71271) ([Tanya Bragin](https://github.com/tbragin)). +* Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). +* Expose one more simple merge selector setting. [#71313](https://github.com/ClickHouse/ClickHouse/pull/71313) ([alesapin](https://github.com/alesapin)). +* Better style for some sever-level settings. [#71319](https://github.com/ClickHouse/ClickHouse/pull/71319) ([alesapin](https://github.com/alesapin)). +* Sync some changes. [#71321](https://github.com/ClickHouse/ClickHouse/pull/71321) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add library to requirements for style-check and fix warning. [#71322](https://github.com/ClickHouse/ClickHouse/pull/71322) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix test `test_workload_entity_keeper_storage`: add more retries. [#71325](https://github.com/ClickHouse/ClickHouse/pull/71325) ([Sergei Trifonov](https://github.com/serxa)). +* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. [#71335](https://github.com/ClickHouse/ClickHouse/pull/71335) ([Chang chen](https://github.com/baibaichen)). +* Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. [#71336](https://github.com/ClickHouse/ClickHouse/pull/71336) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Remove deprecated release script. [#71341](https://github.com/ClickHouse/ClickHouse/pull/71341) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.1.2812-stable. [#71343](https://github.com/ClickHouse/ClickHouse/pull/71343) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Bump USearch to 2.16.0 and add more tests. [#71344](https://github.com/ClickHouse/ClickHouse/pull/71344) ([Robert Schulze](https://github.com/rschu1ze)). +* check-doc-aspell: Print full path to script in CI report. [#71345](https://github.com/ClickHouse/ClickHouse/pull/71345) ([Vladimir Cherkasov](https://github.com/vdimir)). +* CI: Fix fedora version in create release workflow. [#71347](https://github.com/ClickHouse/ClickHouse/pull/71347) ([Max Kainov](https://github.com/maxknv)). +* fs cache: add assertions. [#71348](https://github.com/ClickHouse/ClickHouse/pull/71348) ([Kseniia Sumarokova](https://github.com/kssenii)). +* More info in TOO_SLOW exception. [#71365](https://github.com/ClickHouse/ClickHouse/pull/71365) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix unused variables around WorkloadEntityStorageBase. [#71367](https://github.com/ClickHouse/ClickHouse/pull/71367) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow to prewarm mark cache by system command without enabled setting. [#71368](https://github.com/ClickHouse/ClickHouse/pull/71368) ([Anton Popov](https://github.com/CurtizJ)). +* Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. [#71380](https://github.com/ClickHouse/ClickHouse/pull/71380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Rename `compare8` to `compare16` for consistency. [#71416](https://github.com/ClickHouse/ClickHouse/pull/71416) ([Nikita Taranov](https://github.com/nickitat)). +* Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. [#71418](https://github.com/ClickHouse/ClickHouse/pull/71418) ([Azat Khuzhin](https://github.com/azat)). +* [Experiment] Analyzer: Check what happens after if-condition removal. [#71425](https://github.com/ClickHouse/ClickHouse/pull/71425) ([Dmitry Novik](https://github.com/novikd)). +* Update version_date.tsv and changelog after v24.8.6.70-lts. [#71428](https://github.com/ClickHouse/ClickHouse/pull/71428) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix upgrade check (24.11). [#71438](https://github.com/ClickHouse/ClickHouse/pull/71438) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assert during insert into vector similarity index in presence of other skipping indexes. [#71457](https://github.com/ClickHouse/ClickHouse/pull/71457) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid `seconds left [-3]` in cache await. [#71468](https://github.com/ClickHouse/ClickHouse/pull/71468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added tests for corner cases for 24.10. [#71469](https://github.com/ClickHouse/ClickHouse/pull/71469) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Expose base setting for merge selector. [#71497](https://github.com/ClickHouse/ClickHouse/pull/71497) ([alesapin](https://github.com/alesapin)). +* Fixed incorrect settings order `max_parser_depth` and `max_parser_backtracks`. [#71498](https://github.com/ClickHouse/ClickHouse/pull/71498) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support the endpoint of oss accelerator. [#71502](https://github.com/ClickHouse/ClickHouse/pull/71502) ([Kai Zhu](https://github.com/nauu)). +* Fix flaky test_drop_complex_columns. [#71504](https://github.com/ClickHouse/ClickHouse/pull/71504) ([Ilya Golshtein](https://github.com/ilejn)). +* Move bitShift function changelog entries to backward incompatible. [#71510](https://github.com/ClickHouse/ClickHouse/pull/71510) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix copy/paste error. [#71513](https://github.com/ClickHouse/ClickHouse/pull/71513) ([Denny Crane](https://github.com/den-crane)). +* Allow specifying cmdline flags in integration test. It's needed by [#71452](https://github.com/ClickHouse/ClickHouse/issues/71452) to validate a bugfix. [#71523](https://github.com/ClickHouse/ClickHouse/pull/71523) ([Amos Bird](https://github.com/amosbird)). +* Add ProfileEvents for merge selector timings. [#71524](https://github.com/ClickHouse/ClickHouse/pull/71524) ([alesapin](https://github.com/alesapin)). +* Minor: Remove "experimental" mention of analyzer. [#71525](https://github.com/ClickHouse/ClickHouse/pull/71525) ([Robert Schulze](https://github.com/rschu1ze)). +* Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. [#71530](https://github.com/ClickHouse/ClickHouse/pull/71530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add test to check that accessing system.functions does not populate query_log used_functions. [#71535](https://github.com/ClickHouse/ClickHouse/pull/71535) ([Raúl Marín](https://github.com/Algunenano)). +* Improve `query_plan_merge_filters` optimization. Fixes [#71408](https://github.com/ClickHouse/ClickHouse/issues/71408). [#71539](https://github.com/ClickHouse/ClickHouse/pull/71539) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix compatibility with refreshable materialized views created by old clickhouse servers. [#71556](https://github.com/ClickHouse/ClickHouse/pull/71556) ([Michael Kolupaev](https://github.com/al13n321)). +* Vector similarity index: Re-introduce support for legacy index creation syntax. [#71572](https://github.com/ClickHouse/ClickHouse/pull/71572) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid port clash in CoordinationTest/0.TestSummingRaft1. [#71584](https://github.com/ClickHouse/ClickHouse/pull/71584) ([Raúl Marín](https://github.com/Algunenano)). +* Fix for `00180_no_seek_avoiding_when_reading_from_cache`. [#71596](https://github.com/ClickHouse/ClickHouse/pull/71596) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix for `test_storage_s3_queue::test_shards_distributed[ordered-2]`. [#71597](https://github.com/ClickHouse/ClickHouse/pull/71597) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable enable_job_stack_trace by default, second attempt. [#71625](https://github.com/ClickHouse/ClickHouse/pull/71625) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update version_date.tsv and changelog after v24.3.13.40-lts. [#71627](https://github.com/ClickHouse/ClickHouse/pull/71627) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fixes the bug regarding max rows/bytes to read. [#71634](https://github.com/ClickHouse/ClickHouse/pull/71634) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* SimSIMD: Improve suppression for msan false positive. [#71635](https://github.com/ClickHouse/ClickHouse/pull/71635) ([Robert Schulze](https://github.com/rschu1ze)). +* Add `min_parts_to_merge_at_once` merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less than `e` (2.71...) data parts at once because it increases both write amplification and parts number. [#71637](https://github.com/ClickHouse/ClickHouse/pull/71637) ([alesapin](https://github.com/alesapin)). +* Fix broken 03247_ghdata_string_to_json_alter. [#71638](https://github.com/ClickHouse/ClickHouse/pull/71638) ([Pavel Kruglov](https://github.com/Avogar)). +* Update test. [#71654](https://github.com/ClickHouse/ClickHouse/pull/71654) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: fix mysql containers using improper log directory. [#71655](https://github.com/ClickHouse/ClickHouse/pull/71655) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update README.md - Update meetups. [#71657](https://github.com/ClickHouse/ClickHouse/pull/71657) ([Tanya Bragin](https://github.com/tbragin)). +* Add index granularity size column to system.parts. [#71658](https://github.com/ClickHouse/ClickHouse/pull/71658) ([alesapin](https://github.com/alesapin)). +* Update PULL_REQUEST_TEMPLATE.md. [#71687](https://github.com/ClickHouse/ClickHouse/pull/71687) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Historically we have a strange cherry-pick branches naming, e.g. `cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e`. The `cherrypick/24.3/62297` looks nicer and more straightforward. [#71698](https://github.com/ClickHouse/ClickHouse/pull/71698) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. [#71715](https://github.com/ClickHouse/ClickHouse/pull/71715) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix flaky test test_prometheus_protocols. [#71772](https://github.com/ClickHouse/ClickHouse/pull/71772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. [#71782](https://github.com/ClickHouse/ClickHouse/pull/71782) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix processors profile is not collected for subqueries in some cases. [#71787](https://github.com/ClickHouse/ClickHouse/pull/71787) ([Nikita Taranov](https://github.com/nickitat)). +* Lint some stuff. [#71795](https://github.com/ClickHouse/ClickHouse/pull/71795) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Closes [#71780](https://github.com/ClickHouse/ClickHouse/issues/71780). [#71818](https://github.com/ClickHouse/ClickHouse/pull/71818) ([Kseniia Sumarokova](https://github.com/kssenii)). +* relax memory limit for 00755_avg_value_size_hint_passing.sql. [#71820](https://github.com/ClickHouse/ClickHouse/pull/71820) ([Sema Checherinda](https://github.com/CheSema)). +* The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper `Dockerfile.ubuntu` there. [#71825](https://github.com/ClickHouse/ClickHouse/pull/71825) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests for parseDateTime64InJodaSyntax. [#71829](https://github.com/ClickHouse/ClickHouse/pull/71829) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Enable build profiling in pull requests. [#71847](https://github.com/ClickHouse/ClickHouse/pull/71847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add check and assertion. [#71856](https://github.com/ClickHouse/ClickHouse/pull/71856) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Some healthcheck is better than nothing. [#71865](https://github.com/ClickHouse/ClickHouse/pull/71865) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* More accurate `calculateCacheKey` implementation. [#71868](https://github.com/ClickHouse/ClickHouse/pull/71868) ([Nikita Taranov](https://github.com/nickitat)). +* add test 03248_max_parts_to_move. [#71869](https://github.com/ClickHouse/ClickHouse/pull/71869) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update README.md - Update meetups. [#71872](https://github.com/ClickHouse/ClickHouse/pull/71872) ([Tanya Bragin](https://github.com/tbragin)). +* Prevents listing files from s3 while inserting. [#71889](https://github.com/ClickHouse/ClickHouse/pull/71889) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Keep query_plan_merge_filters disabled by default. [#71890](https://github.com/ClickHouse/ClickHouse/pull/71890) ([Raúl Marín](https://github.com/Algunenano)). +* Remove useless code. [#71900](https://github.com/ClickHouse/ClickHouse/pull/71900) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* TreeRewriter: fix typo: `parititon` => `partition`. [#71907](https://github.com/ClickHouse/ClickHouse/pull/71907) ([yun](https://github.com/yokofly)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. [#71912](https://github.com/ClickHouse/ClickHouse/pull/71912) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix test_storage_mongodb/test.py::test_secure_connection_uri. [#71924](https://github.com/ClickHouse/ClickHouse/pull/71924) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Adapt some test to run in non-CI configurations. [#71928](https://github.com/ClickHouse/ClickHouse/pull/71928) ([Raúl Marín](https://github.com/Algunenano)). +* Fix build after [#71179](https://github.com/ClickHouse/ClickHouse/issues/71179). Clang-19 checks for unused variables inside of if-conditions. [#71929](https://github.com/ClickHouse/ClickHouse/pull/71929) ([Dmitry Novik](https://github.com/novikd)). +* Fix flaky test 03262_column_sizes_with_dynamic_structure. [#71931](https://github.com/ClickHouse/ClickHouse/pull/71931) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. [#71944](https://github.com/ClickHouse/ClickHouse/pull/71944) ([Nikita Taranov](https://github.com/nickitat)). +* Enabling `query_plan_merge_filters` again after [#71890](https://github.com/ClickHouse/ClickHouse/issues/71890). [#71964](https://github.com/ClickHouse/ClickHouse/pull/71964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add comment about bf16 to CMake docs. [#71973](https://github.com/ClickHouse/ClickHouse/pull/71973) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 00098_primary_key_memory_allocated. [#71977](https://github.com/ClickHouse/ClickHouse/pull/71977) ([Alexander Gololobov](https://github.com/davenger)). +* Add a test for [#71908](https://github.com/ClickHouse/ClickHouse/issues/71908). [#71986](https://github.com/ClickHouse/ClickHouse/pull/71986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71987](https://github.com/ClickHouse/ClickHouse/pull/71987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor follow-up to [#71581](https://github.com/ClickHouse/ClickHouse/issues/71581). [#71993](https://github.com/ClickHouse/ClickHouse/pull/71993) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Stress test with praktika. [#71995](https://github.com/ClickHouse/ClickHouse/pull/71995) ([Max Kainov](https://github.com/maxknv)). +* Fix prewarm of mark cache after adding a new column. [#71996](https://github.com/ClickHouse/ClickHouse/pull/71996) ([Anton Popov](https://github.com/CurtizJ)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. [#72018](https://github.com/ClickHouse/ClickHouse/pull/72018) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix comments. [#72023](https://github.com/ClickHouse/ClickHouse/pull/72023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test for 33604. [#72026](https://github.com/ClickHouse/ClickHouse/pull/72026) ([Nikita Taranov](https://github.com/nickitat)). +* CI: Remove unsafe secret_envs input from yml workflows. [#72028](https://github.com/ClickHouse/ClickHouse/pull/72028) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.2.80-stable. [#72029](https://github.com/ClickHouse/ClickHouse/pull/72029) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.8.7.41-lts. [#72037](https://github.com/ClickHouse/ClickHouse/pull/72037) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.9.3.128-stable. [#72041](https://github.com/ClickHouse/ClickHouse/pull/72041) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.3.14.35-lts. [#72042](https://github.com/ClickHouse/ClickHouse/pull/72042) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Save several minutes of build time. [#72046](https://github.com/ClickHouse/ClickHouse/pull/72046) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#72048](https://github.com/ClickHouse/ClickHouse/pull/72048) ([Tanya Bragin](https://github.com/tbragin)). +* Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix test_disk_over_web_server/. [#72075](https://github.com/ClickHouse/ClickHouse/pull/72075) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Minor improvement for system.query_metric_log stateless test. [#72076](https://github.com/ClickHouse/ClickHouse/pull/72076) ([Pablo Marcos](https://github.com/pamarcos)). +* A follow-up for [#72057](https://github.com/ClickHouse/ClickHouse/issues/72057) and https://github.com/ClickHouse/ClickHouse/pull/71505. [#72079](https://github.com/ClickHouse/ClickHouse/pull/72079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add google-cloud-cpp submodule. [#72092](https://github.com/ClickHouse/ClickHouse/pull/72092) ([Pablo Marcos](https://github.com/pamarcos)). +* CI: Enable fuzzer job in Nightly workflow. [#72101](https://github.com/ClickHouse/ClickHouse/pull/72101) ([Max Kainov](https://github.com/maxknv)). +* Get rid of code duplication after adding `CHECK GRANT` in https://github.com/ClickHouse/ClickHouse/pull/68885. [#72103](https://github.com/ClickHouse/ClickHouse/pull/72103) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add jwt-cpp submodule. [#72104](https://github.com/ClickHouse/ClickHouse/pull/72104) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix list-licenses.sh with OSX. [#72107](https://github.com/ClickHouse/ClickHouse/pull/72107) ([Raúl Marín](https://github.com/Algunenano)). +* fix cancelation for PartitionedSink. [#72126](https://github.com/ClickHouse/ClickHouse/pull/72126) ([Sema Checherinda](https://github.com/CheSema)). +* FIx 02374_analyzer_join_using. [#72145](https://github.com/ClickHouse/ClickHouse/pull/72145) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed a test which was flaky-flaky. [#72147](https://github.com/ClickHouse/ClickHouse/pull/72147) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Backward Incompatible Change +* Remove system tables `generate_series` and `generateSeries`. They were added by mistake here: [#59390](https://github.com/ClickHouse/ClickHouse/issues/59390). [#71091](https://github.com/ClickHouse/ClickHouse/pull/71091) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600). ### Documentation entry for user-facing changes. [#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). +* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). +* The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the `SOURCES` hierarchy. Add grants to any non-default database users that create tables with these engine types. [#71250](https://github.com/ClickHouse/ClickHouse/pull/71250) ([Christoph Wurm](https://github.com/cwurm)). +* Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. [#71300](https://github.com/ClickHouse/ClickHouse/pull/71300) ([Christoph Wurm](https://github.com/cwurm)). +* Rename filesystem cache setting `skip_download_if_exceeds_query_cache` to `filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit`. [#71578](https://github.com/ClickHouse/ClickHouse/pull/71578) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove support for `Enum` as well as `UInt128` and `UInt256` arguments in `deltaSumTimestamp`. Remove support for `Int8`, `UInt8`, `Int16`, and `UInt16` of the second ("timestamp") argument of `deltaSumTimestamp`. [#71790](https://github.com/ClickHouse/ClickHouse/pull/71790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ~~Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled by `query_plan_join_inner_table_selection`, which can be set to `auto`. In this mode, ClickHouse will try to choose the table with the smallest number of rows.~~ Resubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. [#68682](https://github.com/ClickHouse/ClickHouse/pull/68682) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). +* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). +* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). +* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). +* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). +* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). +* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). +* Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). +* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). +* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). +* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). +* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). +* Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). + +#### Performance Improvement +* Add 2 new settings `short_circuit_function_evaluation_for_nulls` and `short_circuit_function_evaluation_for_nulls_threshold` that allow to execute functions over `Nullable` columns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. [#60129](https://github.com/ClickHouse/ClickHouse/pull/60129) ([李扬](https://github.com/taiyang-li)). +* Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). +* Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). + +#### Improvement +* Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). +* Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). +* Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). +* Fix use-after-dtor logic in HashTable destroyElements. [#65279](https://github.com/ClickHouse/ClickHouse/pull/65279) ([cangyin](https://github.com/cangyin)). +* Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). +* Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). +* 1. Refactor `DDLQueryStatusSource`: * Rename `DDLQueryStatusSource` to `DistributedQueryStatusSource`, and make it a base class * Create two subclasses `DDLOnClusterQueryStatusSource` and `ReplicatedDatabaseQueryStatusSource` derived from `DDLQueryStatusSource` to query the status of DDL tasks from `DDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in `DDLOnClusterQueryStatusSource`. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). +* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). +* Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). +* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). +* Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). +* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). +* Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). +* Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). +* Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). +* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). +* Boolean support for parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). +* Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). +* `system.session_log` is quite okay. This closes [#51760](https://github.com/ClickHouse/ClickHouse/issues/51760). [#71150](https://github.com/ClickHouse/ClickHouse/pull/71150) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). +* Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). +* Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Add per host dashboards `Overview (host)` and `Cloud overview (host)` to advanced dashboard. [#71422](https://github.com/ClickHouse/ClickHouse/pull/71422) ([alesapin](https://github.com/alesapin)). +* The methods `removeObject` and `removeObjects` are not idempotent. When retries happen due to network errors, the result could be `object not found` because it has been deleted at previous attempts. [#71529](https://github.com/ClickHouse/ClickHouse/pull/71529) ([Sema Checherinda](https://github.com/CheSema)). +* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). +* Allow using clickhouse with a file argument as --queries-file. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). +* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). +* `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. ### Documentation entry for user-facing changes. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). +* Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). +* Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). +* Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). +* Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Turn-off filesystem cache setting `boundary_alignment` for non-disk read. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). +* Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). +* Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). +* Do not increment the `ILLEGAL_TYPE_OF_ARGUMENT` counter in the `system.errors` table when the `bitmapTransform` function is used, and argument types are valid. [#71971](https://github.com/ClickHouse/ClickHouse/pull/71971) ([Dmitry Novik](https://github.com/novikd)). +* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). +* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). +* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). +* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). +* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). +* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). +* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). +* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). +* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). +* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). +* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). +* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). +* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). +* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). +* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). +* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). +* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). +* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). +* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). +* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). +* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). +* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). +* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). +* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). +* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). +* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). +* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). +* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). +* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). +* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). +* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). +* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). +* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). +* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### Build/Testing/Packaging Improvement +* Add the script to update sources of [docker official library](https://github.com/ClickHouse/docker-library). [#57203](https://github.com/ClickHouse/ClickHouse/pull/57203) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* The build system will prevent libraries with unexpected licenses. [#70988](https://github.com/ClickHouse/ClickHouse/pull/70988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Upgrade docker base image for clickhouse-server and keeper to `ubuntu:22.04`. **Breaking change**: the minimal supported docker version is `20.10.10`. [#71505](https://github.com/ClickHouse/ClickHouse/pull/71505) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve clickhouse-server Dockerfile.ubuntu. Deprecate `CLICKHOUSE_UID/CLICKHOUSE_GID` envs. Remove `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` processing to complien requirements. Consistent `clickhouse/clickhouse-server/clickhouse-keeper` execution to not have it plain in one place and `/usr/bin/clickhouse*` in another. [#71573](https://github.com/ClickHouse/ClickHouse/pull/71573) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Miscellaneous"'. [#71083](https://github.com/ClickHouse/ClickHouse/pull/71083) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. [#71084](https://github.com/ClickHouse/ClickHouse/pull/71084) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. [#71251](https://github.com/ClickHouse/ClickHouse/pull/71251) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. [#71266](https://github.com/ClickHouse/ClickHouse/pull/71266) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Selection of hash join inner table"'. [#71527](https://github.com/ClickHouse/ClickHouse/pull/71527) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. [#71619](https://github.com/ClickHouse/ClickHouse/pull/71619) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. [#71914](https://github.com/ClickHouse/ClickHouse/pull/71914) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. [#71945](https://github.com/ClickHouse/ClickHouse/pull/71945) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. [#71974](https://github.com/ClickHouse/ClickHouse/pull/71974) ([Max Kainov](https://github.com/maxknv)). +* NO CL ENTRY: 'CI: Functional Tests with praktika'. [#71976](https://github.com/ClickHouse/ClickHouse/pull/71976) ([Max Kainov](https://github.com/maxknv)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Refactor TempDataOnDisk. [#66606](https://github.com/ClickHouse/ClickHouse/pull/66606) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Bump krb5 from v1.21.2 to v1.21.3. [#69360](https://github.com/ClickHouse/ClickHouse/pull/69360) ([Robert Schulze](https://github.com/rschu1ze)). +* USearch: Enable SimSIMD backend + enable dynamic dispatch. [#69387](https://github.com/ClickHouse/ClickHouse/pull/69387) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: enable libfuzzer. [#70112](https://github.com/ClickHouse/ClickHouse/pull/70112) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Use `clang-19`. [#70414](https://github.com/ClickHouse/ClickHouse/pull/70414) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Randomize Keeper feature flags in integration tests. [#70523](https://github.com/ClickHouse/ClickHouse/pull/70523) ([Antonio Andelic](https://github.com/antonio2368)). +* All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. [#70691](https://github.com/ClickHouse/ClickHouse/pull/70691) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). +* Add a settings `filesystem_cache_enable_background_download_for_metadata_files` for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. [#70806](https://github.com/ClickHouse/ClickHouse/pull/70806) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make ParquetMetadata say whether bloom filter is present. [#70947](https://github.com/ClickHouse/ClickHouse/pull/70947) ([Michael Kolupaev](https://github.com/al13n321)). +* test for reproducing that ReplacingMergeTree depends on the order of part attachment. [#71010](https://github.com/ClickHouse/ClickHouse/pull/71010) ([Konstantin Morozov](https://github.com/k-morozov)). +* CI: Build Job with praktika. [#71015](https://github.com/ClickHouse/ClickHouse/pull/71015) ([Max Kainov](https://github.com/maxknv)). +* Fix bad test `01524_do_not_merge_across_partitions_select_final.sql`. [#71035](https://github.com/ClickHouse/ClickHouse/pull/71035) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable enable_job_stack_trace by default. [#71039](https://github.com/ClickHouse/ClickHouse/pull/71039) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix two logical errors when reading from stdin in clickhouse local. [#71046](https://github.com/ClickHouse/ClickHouse/pull/71046) ([Michael Kolupaev](https://github.com/al13n321)). +* Sync changes to `ProtocolServerAdapter`. [#71058](https://github.com/ClickHouse/ClickHouse/pull/71058) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a typo. [#71067](https://github.com/ClickHouse/ClickHouse/pull/71067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71070](https://github.com/ClickHouse/ClickHouse/pull/71070) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove bad test `test_system_replicated_fetches`. [#71071](https://github.com/ClickHouse/ClickHouse/pull/71071) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version after release. [#71076](https://github.com/ClickHouse/ClickHouse/pull/71076) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* CI: Functional tests for ARM + ASAN binary. [#71079](https://github.com/ClickHouse/ClickHouse/pull/71079) ([Max Kainov](https://github.com/maxknv)). +* CI: Functional Tests with praktika. [#71081](https://github.com/ClickHouse/ClickHouse/pull/71081) ([Max Kainov](https://github.com/maxknv)). +* Fixup of TrivialMergeSelector. [#71082](https://github.com/ClickHouse/ClickHouse/pull/71082) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Sync integration test with private. [#71096](https://github.com/ClickHouse/ClickHouse/pull/71096) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unique symbols in the `system.coverage_log`. [#71099](https://github.com/ClickHouse/ClickHouse/pull/71099) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better log messages. [#71102](https://github.com/ClickHouse/ClickHouse/pull/71102) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix error in Replicated database. [#71103](https://github.com/ClickHouse/ClickHouse/pull/71103) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update error message for JSONAsObject format. [#71123](https://github.com/ClickHouse/ClickHouse/pull/71123) ([Pavel Kruglov](https://github.com/Avogar)). +* Initial changelog for 24.10. [#71127](https://github.com/ClickHouse/ClickHouse/pull/71127) ([Raúl Marín](https://github.com/Algunenano)). +* Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). +* Try fix rabbitmq. [#71143](https://github.com/ClickHouse/ClickHouse/pull/71143) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Randomize setting `enable_vertical_final`. [#71144](https://github.com/ClickHouse/ClickHouse/pull/71144) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bad test `02561_sorting_constants_and_distinct_crash`. [#71147](https://github.com/ClickHouse/ClickHouse/pull/71147) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test to verify [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308) works. [#71149](https://github.com/ClickHouse/ClickHouse/pull/71149) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix 02932_refreshable_materialized_views_1 flakiness. [#71160](https://github.com/ClickHouse/ClickHouse/pull/71160) ([Michael Kolupaev](https://github.com/al13n321)). +* Use `_minmax_count_projection` instead of `Optimized trivial count` for `ReadFromPreparedSource` node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses [#70939](https://github.com/ClickHouse/ClickHouse/issues/70939). [#71166](https://github.com/ClickHouse/ClickHouse/pull/71166) ([Amos Bird](https://github.com/amosbird)). +* Close [#8687](https://github.com/ClickHouse/ClickHouse/issues/8687). [#71169](https://github.com/ClickHouse/ClickHouse/pull/71169) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). +* Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). +* Print compression method in `clickhouse-compressor --stat`. Useful for inspecting random data files. [#71192](https://github.com/ClickHouse/ClickHouse/pull/71192) ([Amos Bird](https://github.com/amosbird)). +* Updating the events into the recent category and adding the new york event. [#71194](https://github.com/ClickHouse/ClickHouse/pull/71194) ([Zoe Steinkamp](https://github.com/zoesteinkamp)). +* Improve error and log messages around memory usage. [#71195](https://github.com/ClickHouse/ClickHouse/pull/71195) ([Raúl Marín](https://github.com/Algunenano)). +* Minor test adjustments. [#71199](https://github.com/ClickHouse/ClickHouse/pull/71199) ([Raúl Marín](https://github.com/Algunenano)). +* Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). +* Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#71223](https://github.com/ClickHouse/ClickHouse/pull/71223) ([Tanya Bragin](https://github.com/tbragin)). +* Fix `WITH TOTALS` in subquery with parallel replicas. [#71224](https://github.com/ClickHouse/ClickHouse/pull/71224) ([Nikita Taranov](https://github.com/nickitat)). +* Ignore `No such key` exceptions in some cases. [#71236](https://github.com/ClickHouse/ClickHouse/pull/71236) ([Antonio Andelic](https://github.com/antonio2368)). +* Make cloud sync title shorter. [#71255](https://github.com/ClickHouse/ClickHouse/pull/71255) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Meetups update. [#71271](https://github.com/ClickHouse/ClickHouse/pull/71271) ([Tanya Bragin](https://github.com/tbragin)). +* Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). +* Expose one more simple merge selector setting. [#71313](https://github.com/ClickHouse/ClickHouse/pull/71313) ([alesapin](https://github.com/alesapin)). +* Better style for some sever-level settings. [#71319](https://github.com/ClickHouse/ClickHouse/pull/71319) ([alesapin](https://github.com/alesapin)). +* Sync some changes. [#71321](https://github.com/ClickHouse/ClickHouse/pull/71321) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add library to requirements for style-check and fix warning. [#71322](https://github.com/ClickHouse/ClickHouse/pull/71322) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix test `test_workload_entity_keeper_storage`: add more retries. [#71325](https://github.com/ClickHouse/ClickHouse/pull/71325) ([Sergei Trifonov](https://github.com/serxa)). +* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. [#71335](https://github.com/ClickHouse/ClickHouse/pull/71335) ([Chang chen](https://github.com/baibaichen)). +* Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. [#71336](https://github.com/ClickHouse/ClickHouse/pull/71336) ([Michael Kolupaev](https://github.com/al13n321)). +* CI: Remove deprecated release script. [#71341](https://github.com/ClickHouse/ClickHouse/pull/71341) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.1.2812-stable. [#71343](https://github.com/ClickHouse/ClickHouse/pull/71343) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Bump USearch to 2.16.0 and add more tests. [#71344](https://github.com/ClickHouse/ClickHouse/pull/71344) ([Robert Schulze](https://github.com/rschu1ze)). +* check-doc-aspell: Print full path to script in CI report. [#71345](https://github.com/ClickHouse/ClickHouse/pull/71345) ([Vladimir Cherkasov](https://github.com/vdimir)). +* CI: Fix fedora version in create release workflow. [#71347](https://github.com/ClickHouse/ClickHouse/pull/71347) ([Max Kainov](https://github.com/maxknv)). +* fs cache: add assertions. [#71348](https://github.com/ClickHouse/ClickHouse/pull/71348) ([Kseniia Sumarokova](https://github.com/kssenii)). +* More info in TOO_SLOW exception. [#71365](https://github.com/ClickHouse/ClickHouse/pull/71365) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix unused variables around WorkloadEntityStorageBase. [#71367](https://github.com/ClickHouse/ClickHouse/pull/71367) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow to prewarm mark cache by system command without enabled setting. [#71368](https://github.com/ClickHouse/ClickHouse/pull/71368) ([Anton Popov](https://github.com/CurtizJ)). +* Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. [#71380](https://github.com/ClickHouse/ClickHouse/pull/71380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Rename `compare8` to `compare16` for consistency. [#71416](https://github.com/ClickHouse/ClickHouse/pull/71416) ([Nikita Taranov](https://github.com/nickitat)). +* Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. [#71418](https://github.com/ClickHouse/ClickHouse/pull/71418) ([Azat Khuzhin](https://github.com/azat)). +* [Experiment] Analyzer: Check what happens after if-condition removal. [#71425](https://github.com/ClickHouse/ClickHouse/pull/71425) ([Dmitry Novik](https://github.com/novikd)). +* Update version_date.tsv and changelog after v24.8.6.70-lts. [#71428](https://github.com/ClickHouse/ClickHouse/pull/71428) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix upgrade check (24.11). [#71438](https://github.com/ClickHouse/ClickHouse/pull/71438) ([Raúl Marín](https://github.com/Algunenano)). +* Fix assert during insert into vector similarity index in presence of other skipping indexes. [#71457](https://github.com/ClickHouse/ClickHouse/pull/71457) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid `seconds left [-3]` in cache await. [#71468](https://github.com/ClickHouse/ClickHouse/pull/71468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Added tests for corner cases for 24.10. [#71469](https://github.com/ClickHouse/ClickHouse/pull/71469) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Expose base setting for merge selector. [#71497](https://github.com/ClickHouse/ClickHouse/pull/71497) ([alesapin](https://github.com/alesapin)). +* Fixed incorrect settings order `max_parser_depth` and `max_parser_backtracks`. [#71498](https://github.com/ClickHouse/ClickHouse/pull/71498) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Support the endpoint of oss accelerator. [#71502](https://github.com/ClickHouse/ClickHouse/pull/71502) ([Kai Zhu](https://github.com/nauu)). +* Fix flaky test_drop_complex_columns. [#71504](https://github.com/ClickHouse/ClickHouse/pull/71504) ([Ilya Golshtein](https://github.com/ilejn)). +* Move bitShift function changelog entries to backward incompatible. [#71510](https://github.com/ClickHouse/ClickHouse/pull/71510) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix copy/paste error. [#71513](https://github.com/ClickHouse/ClickHouse/pull/71513) ([Denny Crane](https://github.com/den-crane)). +* Allow specifying cmdline flags in integration test. It's needed by [#71452](https://github.com/ClickHouse/ClickHouse/issues/71452) to validate a bugfix. [#71523](https://github.com/ClickHouse/ClickHouse/pull/71523) ([Amos Bird](https://github.com/amosbird)). +* Add ProfileEvents for merge selector timings. [#71524](https://github.com/ClickHouse/ClickHouse/pull/71524) ([alesapin](https://github.com/alesapin)). +* Minor: Remove "experimental" mention of analyzer. [#71525](https://github.com/ClickHouse/ClickHouse/pull/71525) ([Robert Schulze](https://github.com/rschu1ze)). +* Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. [#71530](https://github.com/ClickHouse/ClickHouse/pull/71530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add test to check that accessing system.functions does not populate query_log used_functions. [#71535](https://github.com/ClickHouse/ClickHouse/pull/71535) ([Raúl Marín](https://github.com/Algunenano)). +* Improve `query_plan_merge_filters` optimization. Fixes [#71408](https://github.com/ClickHouse/ClickHouse/issues/71408). [#71539](https://github.com/ClickHouse/ClickHouse/pull/71539) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix compatibility with refreshable materialized views created by old clickhouse servers. [#71556](https://github.com/ClickHouse/ClickHouse/pull/71556) ([Michael Kolupaev](https://github.com/al13n321)). +* Vector similarity index: Re-introduce support for legacy index creation syntax. [#71572](https://github.com/ClickHouse/ClickHouse/pull/71572) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid port clash in CoordinationTest/0.TestSummingRaft1. [#71584](https://github.com/ClickHouse/ClickHouse/pull/71584) ([Raúl Marín](https://github.com/Algunenano)). +* Fix for `00180_no_seek_avoiding_when_reading_from_cache`. [#71596](https://github.com/ClickHouse/ClickHouse/pull/71596) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix for `test_storage_s3_queue::test_shards_distributed[ordered-2]`. [#71597](https://github.com/ClickHouse/ClickHouse/pull/71597) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable enable_job_stack_trace by default, second attempt. [#71625](https://github.com/ClickHouse/ClickHouse/pull/71625) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update version_date.tsv and changelog after v24.3.13.40-lts. [#71627](https://github.com/ClickHouse/ClickHouse/pull/71627) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fixes the bug regarding max rows/bytes to read. [#71634](https://github.com/ClickHouse/ClickHouse/pull/71634) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* SimSIMD: Improve suppression for msan false positive. [#71635](https://github.com/ClickHouse/ClickHouse/pull/71635) ([Robert Schulze](https://github.com/rschu1ze)). +* Add `min_parts_to_merge_at_once` merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less than `e` (2.71...) data parts at once because it increases both write amplification and parts number. [#71637](https://github.com/ClickHouse/ClickHouse/pull/71637) ([alesapin](https://github.com/alesapin)). +* Fix broken 03247_ghdata_string_to_json_alter. [#71638](https://github.com/ClickHouse/ClickHouse/pull/71638) ([Pavel Kruglov](https://github.com/Avogar)). +* Update test. [#71654](https://github.com/ClickHouse/ClickHouse/pull/71654) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: fix mysql containers using improper log directory. [#71655](https://github.com/ClickHouse/ClickHouse/pull/71655) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update README.md - Update meetups. [#71657](https://github.com/ClickHouse/ClickHouse/pull/71657) ([Tanya Bragin](https://github.com/tbragin)). +* Add index granularity size column to system.parts. [#71658](https://github.com/ClickHouse/ClickHouse/pull/71658) ([alesapin](https://github.com/alesapin)). +* Update PULL_REQUEST_TEMPLATE.md. [#71687](https://github.com/ClickHouse/ClickHouse/pull/71687) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Historically we have a strange cherry-pick branches naming, e.g. `cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e`. The `cherrypick/24.3/62297` looks nicer and more straightforward. [#71698](https://github.com/ClickHouse/ClickHouse/pull/71698) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. [#71715](https://github.com/ClickHouse/ClickHouse/pull/71715) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix flaky test test_prometheus_protocols. [#71772](https://github.com/ClickHouse/ClickHouse/pull/71772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. [#71782](https://github.com/ClickHouse/ClickHouse/pull/71782) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix processors profile is not collected for subqueries in some cases. [#71787](https://github.com/ClickHouse/ClickHouse/pull/71787) ([Nikita Taranov](https://github.com/nickitat)). +* Lint some stuff. [#71795](https://github.com/ClickHouse/ClickHouse/pull/71795) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Closes [#71780](https://github.com/ClickHouse/ClickHouse/issues/71780). [#71818](https://github.com/ClickHouse/ClickHouse/pull/71818) ([Kseniia Sumarokova](https://github.com/kssenii)). +* relax memory limit for 00755_avg_value_size_hint_passing.sql. [#71820](https://github.com/ClickHouse/ClickHouse/pull/71820) ([Sema Checherinda](https://github.com/CheSema)). +* The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper `Dockerfile.ubuntu` there. [#71825](https://github.com/ClickHouse/ClickHouse/pull/71825) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests for parseDateTime64InJodaSyntax. [#71829](https://github.com/ClickHouse/ClickHouse/pull/71829) ([Max Vostrikov](https://github.com/max-vostrikov)). +* Enable build profiling in pull requests. [#71847](https://github.com/ClickHouse/ClickHouse/pull/71847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add check and assertion. [#71856](https://github.com/ClickHouse/ClickHouse/pull/71856) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Some healthcheck is better than nothing. [#71865](https://github.com/ClickHouse/ClickHouse/pull/71865) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* More accurate `calculateCacheKey` implementation. [#71868](https://github.com/ClickHouse/ClickHouse/pull/71868) ([Nikita Taranov](https://github.com/nickitat)). +* add test 03248_max_parts_to_move. [#71869](https://github.com/ClickHouse/ClickHouse/pull/71869) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Update README.md - Update meetups. [#71872](https://github.com/ClickHouse/ClickHouse/pull/71872) ([Tanya Bragin](https://github.com/tbragin)). +* Prevents listing files from s3 while inserting. [#71889](https://github.com/ClickHouse/ClickHouse/pull/71889) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Keep query_plan_merge_filters disabled by default. [#71890](https://github.com/ClickHouse/ClickHouse/pull/71890) ([Raúl Marín](https://github.com/Algunenano)). +* Remove useless code. [#71900](https://github.com/ClickHouse/ClickHouse/pull/71900) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* TreeRewriter: fix typo: `parititon` => `partition`. [#71907](https://github.com/ClickHouse/ClickHouse/pull/71907) ([yun](https://github.com/yokofly)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. [#71912](https://github.com/ClickHouse/ClickHouse/pull/71912) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix test_storage_mongodb/test.py::test_secure_connection_uri. [#71924](https://github.com/ClickHouse/ClickHouse/pull/71924) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Adapt some test to run in non-CI configurations. [#71928](https://github.com/ClickHouse/ClickHouse/pull/71928) ([Raúl Marín](https://github.com/Algunenano)). +* Fix build after [#71179](https://github.com/ClickHouse/ClickHouse/issues/71179). Clang-19 checks for unused variables inside of if-conditions. [#71929](https://github.com/ClickHouse/ClickHouse/pull/71929) ([Dmitry Novik](https://github.com/novikd)). +* Fix flaky test 03262_column_sizes_with_dynamic_structure. [#71931](https://github.com/ClickHouse/ClickHouse/pull/71931) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. [#71944](https://github.com/ClickHouse/ClickHouse/pull/71944) ([Nikita Taranov](https://github.com/nickitat)). +* Enabling `query_plan_merge_filters` again after [#71890](https://github.com/ClickHouse/ClickHouse/issues/71890). [#71964](https://github.com/ClickHouse/ClickHouse/pull/71964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add comment about bf16 to CMake docs. [#71973](https://github.com/ClickHouse/ClickHouse/pull/71973) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 00098_primary_key_memory_allocated. [#71977](https://github.com/ClickHouse/ClickHouse/pull/71977) ([Alexander Gololobov](https://github.com/davenger)). +* Add a test for [#71908](https://github.com/ClickHouse/ClickHouse/issues/71908). [#71986](https://github.com/ClickHouse/ClickHouse/pull/71986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Miscellaneous. [#71987](https://github.com/ClickHouse/ClickHouse/pull/71987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor follow-up to [#71581](https://github.com/ClickHouse/ClickHouse/issues/71581). [#71993](https://github.com/ClickHouse/ClickHouse/pull/71993) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Stress test with praktika. [#71995](https://github.com/ClickHouse/ClickHouse/pull/71995) ([Max Kainov](https://github.com/maxknv)). +* Fix prewarm of mark cache after adding a new column. [#71996](https://github.com/ClickHouse/ClickHouse/pull/71996) ([Anton Popov](https://github.com/CurtizJ)). +* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. [#72018](https://github.com/ClickHouse/ClickHouse/pull/72018) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix comments. [#72023](https://github.com/ClickHouse/ClickHouse/pull/72023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add test for 33604. [#72026](https://github.com/ClickHouse/ClickHouse/pull/72026) ([Nikita Taranov](https://github.com/nickitat)). +* CI: Remove unsafe secret_envs input from yml workflows. [#72028](https://github.com/ClickHouse/ClickHouse/pull/72028) ([Max Kainov](https://github.com/maxknv)). +* Update version_date.tsv and changelog after v24.10.2.80-stable. [#72029](https://github.com/ClickHouse/ClickHouse/pull/72029) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.8.7.41-lts. [#72037](https://github.com/ClickHouse/ClickHouse/pull/72037) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.9.3.128-stable. [#72041](https://github.com/ClickHouse/ClickHouse/pull/72041) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelog after v24.3.14.35-lts. [#72042](https://github.com/ClickHouse/ClickHouse/pull/72042) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Save several minutes of build time. [#72046](https://github.com/ClickHouse/ClickHouse/pull/72046) ([Raúl Marín](https://github.com/Algunenano)). +* Update README.md - Update meetups. [#72048](https://github.com/ClickHouse/ClickHouse/pull/72048) ([Tanya Bragin](https://github.com/tbragin)). +* Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix test_disk_over_web_server/. [#72075](https://github.com/ClickHouse/ClickHouse/pull/72075) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Minor improvement for system.query_metric_log stateless test. [#72076](https://github.com/ClickHouse/ClickHouse/pull/72076) ([Pablo Marcos](https://github.com/pamarcos)). +* A follow-up for [#72057](https://github.com/ClickHouse/ClickHouse/issues/72057) and https://github.com/ClickHouse/ClickHouse/pull/71505. [#72079](https://github.com/ClickHouse/ClickHouse/pull/72079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add google-cloud-cpp submodule. [#72092](https://github.com/ClickHouse/ClickHouse/pull/72092) ([Pablo Marcos](https://github.com/pamarcos)). +* CI: Enable fuzzer job in Nightly workflow. [#72101](https://github.com/ClickHouse/ClickHouse/pull/72101) ([Max Kainov](https://github.com/maxknv)). +* Get rid of code duplication after adding `CHECK GRANT` in https://github.com/ClickHouse/ClickHouse/pull/68885. [#72103](https://github.com/ClickHouse/ClickHouse/pull/72103) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add jwt-cpp submodule. [#72104](https://github.com/ClickHouse/ClickHouse/pull/72104) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix list-licenses.sh with OSX. [#72107](https://github.com/ClickHouse/ClickHouse/pull/72107) ([Raúl Marín](https://github.com/Algunenano)). +* fix cancelation for PartitionedSink. [#72126](https://github.com/ClickHouse/ClickHouse/pull/72126) ([Sema Checherinda](https://github.com/CheSema)). +* FIx 02374_analyzer_join_using. [#72145](https://github.com/ClickHouse/ClickHouse/pull/72145) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed a test which was flaky-flaky. [#72147](https://github.com/ClickHouse/ClickHouse/pull/72147) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + ### ClickHouse release 24.10, 2024-10-31 #### Backward Incompatible Change From 0d65fbf0e235ede30d0caf5e0b142995c22df029 Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Fri, 22 Nov 2024 12:45:29 +0100 Subject: [PATCH 175/433] enforce_index_structure_match_on_partition_manipulation 24.11 --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 1818c01a362..0b3ea97f5df 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -586,11 +586,11 @@ static std::initializer_list Date: Fri, 22 Nov 2024 12:49:30 +0100 Subject: [PATCH 176/433] Revert "Short circuit optimization for functions executed over Nullable arguments" --- src/Columns/MaskOperations.cpp | 2 +- src/Common/ProfileEvents.cpp | 2 - src/Core/Settings.cpp | 7 - src/Core/SettingsChangesHistory.cpp | 2 - src/Functions/FunctionHelpers.cpp | 121 ++++--------- src/Functions/FunctionHelpers.h | 5 - src/Functions/IFunction.cpp | 160 ++++-------------- src/Functions/IFunction.h | 4 - src/Storages/Hive/HiveCommon.cpp | 1 - src/Storages/Hive/StorageHive.cpp | 1 - .../ObjectStorage/HDFS/Configuration.cpp | 1 - .../02809_storage_set_analysis_bug.sql | 2 +- 12 files changed, 71 insertions(+), 237 deletions(-) diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 14f3e781c47..eac42400517 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -24,7 +24,7 @@ void expandDataByMask(PaddedPODArray & data, const PaddedPODArray & ma ssize_t from = data.size() - 1; ssize_t index = mask.size() - 1; - data.resize_exact(mask.size()); + data.resize(mask.size()); while (index >= 0) { if (!!mask[index] ^ inverted) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ec540dcf632..bdacf03d3bb 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -61,8 +61,6 @@ M(ArenaAllocBytes, "Number of bytes allocated for memory Arena (used for GROUP BY and similar operations)", ValueType::Bytes) \ M(FunctionExecute, "Number of SQL ordinary function calls (SQL functions are called on per-block basis, so this number represents the number of blocks).", ValueType::Number) \ M(TableFunctionExecute, "Number of table function calls.", ValueType::Number) \ - M(DefaultImplementationForNullsRows, "Number of rows processed by default implementation for nulls in function execution", ValueType::Number) \ - M(DefaultImplementationForNullsRowsWithNulls, "Number of rows which contain null values processed by default implementation for nulls in function execution", ValueType::Number) \ M(MarkCacheHits, "Number of times an entry has been found in the mark cache, so we didn't have to load a mark file.", ValueType::Number) \ M(MarkCacheMisses, "Number of times an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.", ValueType::Number) \ M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided). Only updated for SELECT queries with SETTING use_query_cache = 1.", ValueType::Number) \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f01d0474a87..5a0a5da7e17 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5520,13 +5520,6 @@ Only available in ClickHouse Cloud. Number of background threads for speculative )", 0) \ DECLARE(Int64, ignore_cold_parts_seconds, 0, R"( Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree. -)", 0) \ - DECLARE(Bool, short_circuit_function_evaluation_for_nulls, true, R"( -Allows to execute functions with Nullable arguments only on rows with non-NULL values in all arguments when ratio of NULL values in arguments exceeds short_circuit_function_evaluation_for_nulls_threshold. Applies only to functions that return NULL value for rows with at least one NULL value in arguments. -)", 0) \ - DECLARE(Double, short_circuit_function_evaluation_for_nulls_threshold, 1.0, R"( -Ratio threshold of NULL values to execute functions with Nullable arguments only on rows with non-NULL values in all arguments. Applies when setting short_circuit_function_evaluation_for_nulls is enabled. -When the ratio of rows containing NULL values to the total number of rows exceeds this threshold, these rows containing NULL values will not be evaluated. )", 0) \ DECLARE(Int64, prefer_warmed_unmerged_parts_seconds, 0, R"( Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index f0d3e001362..2df8defcaa9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,8 +64,6 @@ static std::initializer_list -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include #include @@ -13,11 +13,11 @@ namespace DB namespace ErrorCodes { -extern const int ILLEGAL_COLUMN; -extern const int LOGICAL_ERROR; -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int SIZES_OF_ARRAYS_DONT_MATCH; -extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SIZES_OF_ARRAYS_DONT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) @@ -27,7 +27,8 @@ const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * co const ColumnConst * res = assert_cast(column); - if (checkColumn(&res->getDataColumn()) || checkColumn(&res->getDataColumn())) + if (checkColumn(&res->getDataColumn()) + || checkColumn(&res->getDataColumn())) return res; return {}; @@ -100,22 +101,18 @@ String withOrdinalEnding(size_t i) { switch (i) { - case 0: - return "1st"; - case 1: - return "2nd"; - case 2: - return "3rd"; - default: - return std::to_string(i) + "th"; + case 0: return "1st"; + case 1: return "2nd"; + case 2: return "3rd"; + default: return std::to_string(i) + "th"; } + } -void validateArgumentsImpl( - const IFunction & func, - const ColumnsWithTypeAndName & arguments, - size_t argument_offset, - const FunctionArgumentDescriptors & descriptors) +void validateArgumentsImpl(const IFunction & func, + const ColumnsWithTypeAndName & arguments, + size_t argument_offset, + const FunctionArgumentDescriptors & descriptors) { for (size_t i = 0; i < descriptors.size(); ++i) { @@ -126,14 +123,13 @@ void validateArgumentsImpl( const auto & arg = arguments[i + argument_offset]; const auto & descriptor = descriptors[i]; if (int error_code = descriptor.isValid(arg.type, arg.column); error_code != 0) - throw Exception( - error_code, - "A value of illegal type was provided as {} argument '{}' to function '{}'. Expected: {}, got: {}", - withOrdinalEnding(argument_offset + i), - descriptor.name, - func.getName(), - descriptor.type_name, - arg.type ? arg.type->getName() : ""); + throw Exception(error_code, + "A value of illegal type was provided as {} argument '{}' to function '{}'. Expected: {}, got: {}", + withOrdinalEnding(argument_offset + i), + descriptor.name, + func.getName(), + descriptor.type_name, + arg.type ? arg.type->getName() : ""); } } @@ -153,35 +149,26 @@ int FunctionArgumentDescriptor::isValid(const DataTypePtr & data_type, const Col return 0; } -void validateFunctionArguments( - const IFunction & func, - const ColumnsWithTypeAndName & arguments, - const FunctionArgumentDescriptors & mandatory_args, - const FunctionArgumentDescriptors & optional_args) +void validateFunctionArguments(const IFunction & func, + const ColumnsWithTypeAndName & arguments, + const FunctionArgumentDescriptors & mandatory_args, + const FunctionArgumentDescriptors & optional_args) { if (arguments.size() < mandatory_args.size() || arguments.size() > mandatory_args.size() + optional_args.size()) { - auto argument_singular_or_plural - = [](const auto & args) -> std::string_view { return args.size() == 1 ? "argument" : "arguments"; }; + auto argument_singular_or_plural = [](const auto & args) -> std::string_view { return args.size() == 1 ? "argument" : "arguments"; }; String expected_args_string; if (!mandatory_args.empty() && !optional_args.empty()) - expected_args_string = fmt::format( - "{} mandatory {} and {} optional {}", - mandatory_args.size(), - argument_singular_or_plural(mandatory_args), - optional_args.size(), - argument_singular_or_plural(optional_args)); + expected_args_string = fmt::format("{} mandatory {} and {} optional {}", mandatory_args.size(), argument_singular_or_plural(mandatory_args), optional_args.size(), argument_singular_or_plural(optional_args)); else if (!mandatory_args.empty() && optional_args.empty()) - expected_args_string = fmt::format( - "{} {}", mandatory_args.size(), argument_singular_or_plural(mandatory_args)); /// intentionally not "_mandatory_ arguments" + expected_args_string = fmt::format("{} {}", mandatory_args.size(), argument_singular_or_plural(mandatory_args)); /// intentionally not "_mandatory_ arguments" else if (mandatory_args.empty() && !optional_args.empty()) expected_args_string = fmt::format("{} optional {}", optional_args.size(), argument_singular_or_plural(optional_args)); else expected_args_string = "0 arguments"; - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "An incorrect number of arguments was specified for function '{}'. Expected {}, got {}", func.getName(), expected_args_string, @@ -217,8 +204,7 @@ checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments) return {nested_columns, offsets->data()}; /// NOLINT(clang-analyzer-core.CallAndMessage) } -ColumnPtr -wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) +ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count) { ColumnPtr result_null_map_column; @@ -276,39 +262,6 @@ wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column); } -ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnPtr & null_map) -{ - if (src->onlyNull()) - return src; - - ColumnPtr result_null_map_column; - ColumnPtr src_not_nullable = src; - if (const auto * nullable = checkAndGetColumn(src.get())) - { - src_not_nullable = nullable->getNestedColumnPtr(); - result_null_map_column = nullable->getNullMapColumnPtr(); - - MutableColumnPtr mutable_result_null_map_column = IColumn::mutate(std::move(result_null_map_column)); - NullMap & result_null_map = assert_cast(*mutable_result_null_map_column).getData(); - const NullMap & null_map_data = assert_cast(*null_map).getData(); - for (size_t i = 0; i < result_null_map.size(); ++i) - result_null_map[i] |= null_map_data[i]; - - result_null_map_column = std::move(mutable_result_null_map_column); - return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column); - } - else if (const auto * const_src = checkAndGetColumn(src.get())) - { - const NullMap & null_map_data = assert_cast(*null_map).getData(); - ColumnPtr result_null_map = ColumnUInt8::create(1, null_map_data[0] || const_src->isNullAt(0)); - const auto * nullable_data = checkAndGetColumn(&const_src->getDataColumn()); - auto data_not_nullable = nullable_data ? nullable_data->getNestedColumnPtr() : const_src->getDataColumnPtr(); - return ColumnConst::create(ColumnNullable::create(data_not_nullable, result_null_map), const_src->size()); - } - else - return ColumnNullable::create(src->convertToFullColumnIfConst(), null_map); -} - NullPresence getNullPresense(const ColumnsWithTypeAndName & args) { NullPresence res; diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index b8d81673cd5..4f93b236bcb 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -169,11 +169,6 @@ checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments); /// Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL. ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count); -/** Return ColumnNullable of src, with input null map - * Or ColumnConst(ColumnNullable) if the result is always NULL or if the result is constant and always not NULL. - */ -ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnPtr & null_map); - struct NullPresence { bool has_nullable = false; diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index b9ce4b8f04f..759d06c8f1c 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include @@ -7,18 +7,13 @@ #include #include #include -#include -#include #include -#include #include #include #include #include #include #include -#include -#include #include #include #include @@ -32,26 +27,15 @@ # include #endif -namespace ProfileEvents -{ - extern const Event DefaultImplementationForNullsRows; - extern const Event DefaultImplementationForNullsRowsWithNulls; -} namespace DB { -namespace Setting -{ -extern const SettingsBool short_circuit_function_evaluation_for_nulls; -extern const SettingsDouble short_circuit_function_evaluation_for_nulls_threshold; -} - namespace ErrorCodes { -extern const int LOGICAL_ERROR; -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int ILLEGAL_COLUMN; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; } namespace @@ -83,7 +67,9 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( const auto * low_cardinality_type = checkAndGetDataType(column.type.get()); if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", column.type->getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incompatible type for LowCardinality column: {}", + column.type->getName()); if (can_be_executed_on_default_arguments) { @@ -136,7 +122,10 @@ ColumnPtr IExecutableFunction::defaultImplementationForConstantArguments( /// Check that these arguments are really constant. for (auto arg_num : arguments_to_remain_constants) if (arg_num < args.size() && !isColumnConst(*args[arg_num].column)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument at index {} for function {} must be constant", arg_num, getName()); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Argument at index {} for function {} must be constant", + arg_num, + getName()); if (args.empty() || !useDefaultImplementationForConstants() || !allArgumentsAreConstants(args)) return nullptr; @@ -150,16 +139,14 @@ ColumnPtr IExecutableFunction::defaultImplementationForConstantArguments( { const ColumnWithTypeAndName & column = args[arg_num]; - if (arguments_to_remain_constants.end() - != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num)) + if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num)) { temporary_columns.emplace_back(ColumnWithTypeAndName{column.column->cloneResized(1), column.type, column.name}); } else { have_converted_columns = true; - temporary_columns.emplace_back( - ColumnWithTypeAndName{assert_cast(column.column.get())->getDataColumnPtr(), column.type, column.name}); + temporary_columns.emplace_back(ColumnWithTypeAndName{ assert_cast(column.column.get())->getDataColumnPtr(), column.type, column.name }); } } @@ -167,8 +154,7 @@ ColumnPtr IExecutableFunction::defaultImplementationForConstantArguments( * not in "arguments_to_remain_constants" set. Otherwise we get infinite recursion. */ if (!have_converted_columns) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: the function requires more arguments", getName()); @@ -191,7 +177,7 @@ ColumnPtr IExecutableFunction::defaultImplementationForNulls( NullPresence null_presence = getNullPresense(args); - if (null_presence.has_null_constant || null_presence.has_nullable) + if (null_presence.has_null_constant) { // Default implementation for nulls returns null result for null arguments, // so the result type must be nullable. @@ -202,87 +188,17 @@ ColumnPtr IExecutableFunction::defaultImplementationForNulls( "is expected to return Nullable result, got {}", getName(), result_type->getName()); - } - if (null_presence.has_null_constant) - { - /// If any of the input arguments is null literal, the result is null constant. return result_type->createColumnConstWithDefaultValue(input_rows_count); } if (null_presence.has_nullable) { - /// Usually happens during analyzing. We should return non-const column to avoid wrong constant folding. - if (input_rows_count == 0) - return result_type->createColumn(); - - auto result_null_map = ColumnUInt8::create(input_rows_count, 0); - auto & result_null_map_data = result_null_map->getData(); - bool all_columns_constant = true; - for (const auto & arg : args) - { - if (!isColumnConst(*arg.column)) - all_columns_constant = false; - - if (arg.type->isNullable()) - { - if (isColumnConst(*arg.column)) - { - if (arg.column->onlyNull()) - { - /// If any of input columns contains a null constant, the result is null constant. - return result_type->createColumnConstWithDefaultValue(input_rows_count); - } - } - else - { - const auto & null_map = assert_cast(*arg.column).getNullMapData(); - for (size_t i = 0; i < input_rows_count; ++i) - result_null_map_data[i] |= null_map[i]; - } - } - } - - size_t rows_with_nulls = countBytesInFilter(result_null_map_data.data(), 0, input_rows_count); - size_t rows_without_nulls = input_rows_count - rows_with_nulls; - ProfileEvents::increment(ProfileEvents::DefaultImplementationForNullsRows, input_rows_count); - ProfileEvents::increment(ProfileEvents::DefaultImplementationForNullsRowsWithNulls, rows_with_nulls); - - if (rows_without_nulls == 0 && !all_columns_constant) - { - /// Don't need to evaluate function if each row contains at least one null value and not all input columns are constant. - return result_type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst(); - } - - double null_ratio = rows_with_nulls / static_cast(result_null_map_data.size()); - bool should_short_circuit = short_circuit_function_evaluation_for_nulls && !all_columns_constant - && null_ratio >= short_circuit_function_evaluation_for_nulls_threshold; - ColumnsWithTypeAndName temporary_columns = createBlockWithNestedColumns(args); auto temporary_result_type = removeNullable(result_type); - if (!should_short_circuit) - { - /// Each row should be evaluated if there are no nulls or short circuiting is disabled. - auto res = executeWithoutLowCardinalityColumns(temporary_columns, temporary_result_type, input_rows_count, dry_run); - auto new_res = wrapInNullable(res, std::move(result_null_map)); - return new_res; - } - else - { - /// If short circuit is enabled, we only execute the function on rows with all arguments not null - - /// Filter every column by mask - for (auto & col : temporary_columns) - col.column = col.column->filter(result_null_map_data, rows_without_nulls); - - auto res = executeWithoutLowCardinalityColumns(temporary_columns, temporary_result_type, rows_without_nulls, dry_run); - auto mutable_res = IColumn::mutate(std::move(res)); - mutable_res->expand(result_null_map_data, false); - - auto new_res = wrapInNullable(std::move(mutable_res), std::move(result_null_map)); - return new_res; - } + auto res = executeWithoutLowCardinalityColumns(temporary_columns, temporary_result_type, input_rows_count, dry_run); + return wrapInNullable(res, args, result_type, input_rows_count); } return nullptr; @@ -344,21 +260,7 @@ static void convertSparseColumnsToFull(ColumnsWithTypeAndName & args) column.column = recursiveRemoveSparse(column.column); } -IExecutableFunction::IExecutableFunction() -{ - if (CurrentThread::isInitialized()) - { - auto query_context = CurrentThread::get().getQueryContext(); - if (query_context && query_context->getSettingsRef()[Setting::short_circuit_function_evaluation_for_nulls]) - { - short_circuit_function_evaluation_for_nulls = true; - short_circuit_function_evaluation_for_nulls_threshold = query_context->getSettingsRef()[Setting::short_circuit_function_evaluation_for_nulls_threshold]; - } - } -} - -ColumnPtr IExecutableFunction::executeWithoutSparseColumns( - const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const +ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const { ColumnPtr result; if (useDefaultImplementationForLowCardinalityColumns()) @@ -371,16 +273,19 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns( const auto & dictionary_type = res_low_cardinality_type->getDictionaryType(); ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( - columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count); + columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count); - size_t new_input_rows_count - = columns_without_low_cardinality.empty() ? input_rows_count : columns_without_low_cardinality.front().column->size(); + size_t new_input_rows_count = columns_without_low_cardinality.empty() + ? input_rows_count + : columns_without_low_cardinality.front().column->size(); checkFunctionArgumentSizes(columns_without_low_cardinality, new_input_rows_count); auto res = executeWithoutLowCardinalityColumns(columns_without_low_cardinality, dictionary_type, new_input_rows_count, dry_run); bool res_is_constant = isColumnConst(*res); - auto keys = res_is_constant ? res->cloneResized(1)->convertToFullColumnIfConst() : res; + auto keys = res_is_constant + ? res->cloneResized(1)->convertToFullColumnIfConst() + : res; auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType()); ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size()); @@ -406,8 +311,7 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns( return result; } -ColumnPtr IExecutableFunction::execute( - const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const +ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const { checkFunctionArgumentSizes(arguments, input_rows_count); @@ -468,7 +372,7 @@ ColumnPtr IExecutableFunction::execute( if (!result_type->canBeInsideSparseColumns() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); - return res->createWithOffsets(offsets_data, *createColumnConst(res, 0), input_rows_count, /*shift=*/1); + return res->createWithOffsets(offsets_data, *createColumnConst(res, 0), input_rows_count, /*shift=*/ 1); } return ColumnSparse::create(res, sparse_offsets, input_rows_count); @@ -501,8 +405,7 @@ void IFunctionOverloadResolver::checkNumberOfArguments(size_t number_of_argument size_t expected_number_of_arguments = getNumberOfArguments(); if (number_of_arguments != expected_number_of_arguments) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be {}", getName(), number_of_arguments, @@ -541,8 +444,9 @@ DataTypePtr IFunctionOverloadResolver::getReturnType(const ColumnsWithTypeAndNam auto type_without_low_cardinality = getReturnTypeWithoutLowCardinality(args_without_low_cardinality); - if (canBeExecutedOnLowCardinalityDictionary() && has_low_cardinality && num_full_low_cardinality_columns <= 1 - && num_full_ordinary_columns == 0 && type_without_low_cardinality->canBeInsideLowCardinality()) + if (canBeExecutedOnLowCardinalityDictionary() && has_low_cardinality + && num_full_low_cardinality_columns <= 1 && num_full_ordinary_columns == 0 + && type_without_low_cardinality->canBeInsideLowCardinality()) return std::make_shared(type_without_low_cardinality); return type_without_low_cardinality; } @@ -664,7 +568,7 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const ValuesWith ValuesWithType unwrapped_arguments; unwrapped_arguments.reserve(arguments.size()); - std::vector is_null_values; + std::vector is_null_values; for (size_t i = 0; i < arguments.size(); ++i) { diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 2e44d14d820..0e3db156e53 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -44,7 +44,6 @@ using OptionalFieldInterval = std::optional; class IExecutableFunction { public: - IExecutableFunction(); virtual ~IExecutableFunction() = default; @@ -120,9 +119,6 @@ private: ColumnPtr executeWithoutSparseColumns( const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; - - bool short_circuit_function_evaluation_for_nulls = false; - double short_circuit_function_evaluation_for_nulls_threshold = 0.0; }; using ExecutableFunctionPtr = std::shared_ptr; diff --git a/src/Storages/Hive/HiveCommon.cpp b/src/Storages/Hive/HiveCommon.cpp index 4ea2220b9ef..d58e7b535cc 100644 --- a/src/Storages/Hive/HiveCommon.cpp +++ b/src/Storages/Hive/HiveCommon.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 313ca3c5550..902f7e05566 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 18f71642dbd..6bee4154b2f 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -2,7 +2,6 @@ #if USE_HDFS #include -#include #include #include #include diff --git a/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql index f71494673f2..531946acda0 100644 --- a/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql +++ b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS null_in__fuzz_6; set allow_suspicious_low_cardinality_types = 1; CREATE TABLE null_in__fuzz_6 (`dt` LowCardinality(UInt16), `idx` Int32, `i` Nullable(Int256), `s` Int32) ENGINE = MergeTree PARTITION BY dt ORDER BY idx; -insert into null_in__fuzz_6 select * from generateRandom() where i is not null limit 1; +insert into null_in__fuzz_6 select * from generateRandom() limit 1; SET transform_null_in = 0; From c10d68c7a0ab4413e96ddb8aafd38849d7927835 Mon Sep 17 00:00:00 2001 From: Vladimir Cherkasov Date: Fri, 22 Nov 2024 12:51:01 +0100 Subject: [PATCH 177/433] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 41445813bfc..02b131c2e2c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list Date: Fri, 22 Nov 2024 12:53:30 +0100 Subject: [PATCH 178/433] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0b3ea97f5df..e328a46cc79 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -582,11 +582,11 @@ static std::initializer_list Date: Fri, 22 Nov 2024 13:01:31 +0100 Subject: [PATCH 179/433] Do some cleanup of the changelog --- CHANGELOG.md | 629 +-------------------------------------------------- 1 file changed, 12 insertions(+), 617 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8480b5fc457..929dbfd24c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,34 +18,37 @@ #### Backward Incompatible Change * Remove system tables `generate_series` and `generateSeries`. They were added by mistake here: [#59390](https://github.com/ClickHouse/ClickHouse/issues/59390). [#71091](https://github.com/ClickHouse/ClickHouse/pull/71091) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600). ### Documentation entry for user-facing changes. [#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). -* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). +* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600).[#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). * The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the `SOURCES` hierarchy. Add grants to any non-default database users that create tables with these engine types. [#71250](https://github.com/ClickHouse/ClickHouse/pull/71250) ([Christoph Wurm](https://github.com/cwurm)). * Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. [#71300](https://github.com/ClickHouse/ClickHouse/pull/71300) ([Christoph Wurm](https://github.com/cwurm)). * Rename filesystem cache setting `skip_download_if_exceeds_query_cache` to `filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit`. [#71578](https://github.com/ClickHouse/ClickHouse/pull/71578) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove support for `Enum` as well as `UInt128` and `UInt256` arguments in `deltaSumTimestamp`. Remove support for `Int8`, `UInt8`, `Int16`, and `UInt16` of the second ("timestamp") argument of `deltaSumTimestamp`. [#71790](https://github.com/ClickHouse/ClickHouse/pull/71790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Experimental feature +* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). +* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). +* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). +* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). +* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). +* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). #### New Feature * A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* ~~Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled by `query_plan_join_inner_table_selection`, which can be set to `auto`. In this mode, ClickHouse will try to choose the table with the smallest number of rows.~~ Resubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. [#68682](https://github.com/ClickHouse/ClickHouse/pull/68682) ([Vladimir Cherkasov](https://github.com/vdimir)). * Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). * Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). * Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). * Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). * Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). -* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). * Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). * Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). * Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). -* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). * Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). * Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). * Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). -* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). * Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). #### Performance Improvement -* Add 2 new settings `short_circuit_function_evaluation_for_nulls` and `short_circuit_function_evaluation_for_nulls_threshold` that allow to execute functions over `Nullable` columns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. [#60129](https://github.com/ClickHouse/ClickHouse/pull/60129) ([李扬](https://github.com/taiyang-li)). * Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). * Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). * Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -57,20 +60,16 @@ * Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). * Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). * Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). -* Fix use-after-dtor logic in HashTable destroyElements. [#65279](https://github.com/ClickHouse/ClickHouse/pull/65279) ([cangyin](https://github.com/cangyin)). * Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). +* Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). * Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). -* 1. Refactor `DDLQueryStatusSource`: * Rename `DDLQueryStatusSource` to `DistributedQueryStatusSource`, and make it a base class * Create two subclasses `DDLOnClusterQueryStatusSource` and `ReplicatedDatabaseQueryStatusSource` derived from `DDLQueryStatusSource` to query the status of DDL tasks from `DDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in `DDLOnClusterQueryStatusSource`. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). -* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). +* Wait only on active replicas for database ON CLUSTER queries if distributed_ddl_output_mode is set to be *_only_active. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). * Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). * Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). -* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). * Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). * Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). * Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). * Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). * Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). * Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). * Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). @@ -79,8 +78,6 @@ * Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). -* `system.session_log` is quite okay. This closes [#51760](https://github.com/ClickHouse/ClickHouse/issues/51760). [#71150](https://github.com/ClickHouse/ClickHouse/pull/71150) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). * Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). * Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). * Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). @@ -96,7 +93,6 @@ * The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). * Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). * Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). * Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). @@ -105,609 +101,8 @@ * Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). * Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). * Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). -* Do not increment the `ILLEGAL_TYPE_OF_ARGUMENT` counter in the `system.errors` table when the `bitmapTransform` function is used, and argument types are valid. [#71971](https://github.com/ClickHouse/ClickHouse/pull/71971) ([Dmitry Novik](https://github.com/novikd)). -* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). -#### Bug Fix (user-visible misbehavior in an official stable release) -* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). -* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). -* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). -* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). -* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). -* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). -* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). -* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). -* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). -* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). -* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). -* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). -* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). -* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). -* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). -* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). -* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). -* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). -* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). -* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). -* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). -* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). -* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). -* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). -* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). -* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). -* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). -* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). -* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). -* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). -* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). -* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). -* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). -* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). -* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). -* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). -* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). -* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). -* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). -* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). -* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). - -#### Build/Testing/Packaging Improvement -* Add the script to update sources of [docker official library](https://github.com/ClickHouse/docker-library). [#57203](https://github.com/ClickHouse/ClickHouse/pull/57203) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* The build system will prevent libraries with unexpected licenses. [#70988](https://github.com/ClickHouse/ClickHouse/pull/70988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Upgrade docker base image for clickhouse-server and keeper to `ubuntu:22.04`. **Breaking change**: the minimal supported docker version is `20.10.10`. [#71505](https://github.com/ClickHouse/ClickHouse/pull/71505) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Improve clickhouse-server Dockerfile.ubuntu. Deprecate `CLICKHOUSE_UID/CLICKHOUSE_GID` envs. Remove `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` processing to complien requirements. Consistent `clickhouse/clickhouse-server/clickhouse-keeper` execution to not have it plain in one place and `/usr/bin/clickhouse*` in another. [#71573](https://github.com/ClickHouse/ClickHouse/pull/71573) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - -#### NO CL ENTRY - -* NO CL ENTRY: 'Revert "Miscellaneous"'. [#71083](https://github.com/ClickHouse/ClickHouse/pull/71083) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. [#71084](https://github.com/ClickHouse/ClickHouse/pull/71084) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. [#71251](https://github.com/ClickHouse/ClickHouse/pull/71251) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. [#71266](https://github.com/ClickHouse/ClickHouse/pull/71266) ([Sergei Trifonov](https://github.com/serxa)). -* NO CL ENTRY: 'Revert "Selection of hash join inner table"'. [#71527](https://github.com/ClickHouse/ClickHouse/pull/71527) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. [#71619](https://github.com/ClickHouse/ClickHouse/pull/71619) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. [#71914](https://github.com/ClickHouse/ClickHouse/pull/71914) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. [#71945](https://github.com/ClickHouse/ClickHouse/pull/71945) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. [#71974](https://github.com/ClickHouse/ClickHouse/pull/71974) ([Max Kainov](https://github.com/maxknv)). -* NO CL ENTRY: 'CI: Functional Tests with praktika'. [#71976](https://github.com/ClickHouse/ClickHouse/pull/71976) ([Max Kainov](https://github.com/maxknv)). - -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Refactor TempDataOnDisk. [#66606](https://github.com/ClickHouse/ClickHouse/pull/66606) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Bump krb5 from v1.21.2 to v1.21.3. [#69360](https://github.com/ClickHouse/ClickHouse/pull/69360) ([Robert Schulze](https://github.com/rschu1ze)). -* USearch: Enable SimSIMD backend + enable dynamic dispatch. [#69387](https://github.com/ClickHouse/ClickHouse/pull/69387) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: enable libfuzzer. [#70112](https://github.com/ClickHouse/ClickHouse/pull/70112) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Use `clang-19`. [#70414](https://github.com/ClickHouse/ClickHouse/pull/70414) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Randomize Keeper feature flags in integration tests. [#70523](https://github.com/ClickHouse/ClickHouse/pull/70523) ([Antonio Andelic](https://github.com/antonio2368)). -* All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. [#70691](https://github.com/ClickHouse/ClickHouse/pull/70691) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). -* Add a settings `filesystem_cache_enable_background_download_for_metadata_files` for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. [#70806](https://github.com/ClickHouse/ClickHouse/pull/70806) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Make ParquetMetadata say whether bloom filter is present. [#70947](https://github.com/ClickHouse/ClickHouse/pull/70947) ([Michael Kolupaev](https://github.com/al13n321)). -* test for reproducing that ReplacingMergeTree depends on the order of part attachment. [#71010](https://github.com/ClickHouse/ClickHouse/pull/71010) ([Konstantin Morozov](https://github.com/k-morozov)). -* CI: Build Job with praktika. [#71015](https://github.com/ClickHouse/ClickHouse/pull/71015) ([Max Kainov](https://github.com/maxknv)). -* Fix bad test `01524_do_not_merge_across_partitions_select_final.sql`. [#71035](https://github.com/ClickHouse/ClickHouse/pull/71035) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Enable enable_job_stack_trace by default. [#71039](https://github.com/ClickHouse/ClickHouse/pull/71039) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix two logical errors when reading from stdin in clickhouse local. [#71046](https://github.com/ClickHouse/ClickHouse/pull/71046) ([Michael Kolupaev](https://github.com/al13n321)). -* Sync changes to `ProtocolServerAdapter`. [#71058](https://github.com/ClickHouse/ClickHouse/pull/71058) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix a typo. [#71067](https://github.com/ClickHouse/ClickHouse/pull/71067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Miscellaneous. [#71070](https://github.com/ClickHouse/ClickHouse/pull/71070) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove bad test `test_system_replicated_fetches`. [#71071](https://github.com/ClickHouse/ClickHouse/pull/71071) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update version after release. [#71076](https://github.com/ClickHouse/ClickHouse/pull/71076) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* CI: Functional tests for ARM + ASAN binary. [#71079](https://github.com/ClickHouse/ClickHouse/pull/71079) ([Max Kainov](https://github.com/maxknv)). -* CI: Functional Tests with praktika. [#71081](https://github.com/ClickHouse/ClickHouse/pull/71081) ([Max Kainov](https://github.com/maxknv)). -* Fixup of TrivialMergeSelector. [#71082](https://github.com/ClickHouse/ClickHouse/pull/71082) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Sync integration test with private. [#71096](https://github.com/ClickHouse/ClickHouse/pull/71096) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Unique symbols in the `system.coverage_log`. [#71099](https://github.com/ClickHouse/ClickHouse/pull/71099) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Better log messages. [#71102](https://github.com/ClickHouse/ClickHouse/pull/71102) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix error in Replicated database. [#71103](https://github.com/ClickHouse/ClickHouse/pull/71103) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update error message for JSONAsObject format. [#71123](https://github.com/ClickHouse/ClickHouse/pull/71123) ([Pavel Kruglov](https://github.com/Avogar)). -* Initial changelog for 24.10. [#71127](https://github.com/ClickHouse/ClickHouse/pull/71127) ([Raúl Marín](https://github.com/Algunenano)). -* Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). -* Try fix rabbitmq. [#71143](https://github.com/ClickHouse/ClickHouse/pull/71143) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Randomize setting `enable_vertical_final`. [#71144](https://github.com/ClickHouse/ClickHouse/pull/71144) ([Anton Popov](https://github.com/CurtizJ)). -* Fix bad test `02561_sorting_constants_and_distinct_crash`. [#71147](https://github.com/ClickHouse/ClickHouse/pull/71147) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add test to verify [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308) works. [#71149](https://github.com/ClickHouse/ClickHouse/pull/71149) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). -* Fix 02932_refreshable_materialized_views_1 flakiness. [#71160](https://github.com/ClickHouse/ClickHouse/pull/71160) ([Michael Kolupaev](https://github.com/al13n321)). -* Use `_minmax_count_projection` instead of `Optimized trivial count` for `ReadFromPreparedSource` node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses [#70939](https://github.com/ClickHouse/ClickHouse/issues/70939). [#71166](https://github.com/ClickHouse/ClickHouse/pull/71166) ([Amos Bird](https://github.com/amosbird)). -* Close [#8687](https://github.com/ClickHouse/ClickHouse/issues/8687). [#71169](https://github.com/ClickHouse/ClickHouse/pull/71169) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). -* Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). -* Print compression method in `clickhouse-compressor --stat`. Useful for inspecting random data files. [#71192](https://github.com/ClickHouse/ClickHouse/pull/71192) ([Amos Bird](https://github.com/amosbird)). -* Updating the events into the recent category and adding the new york event. [#71194](https://github.com/ClickHouse/ClickHouse/pull/71194) ([Zoe Steinkamp](https://github.com/zoesteinkamp)). -* Improve error and log messages around memory usage. [#71195](https://github.com/ClickHouse/ClickHouse/pull/71195) ([Raúl Marín](https://github.com/Algunenano)). -* Minor test adjustments. [#71199](https://github.com/ClickHouse/ClickHouse/pull/71199) ([Raúl Marín](https://github.com/Algunenano)). -* Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). -* Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Update meetups. [#71223](https://github.com/ClickHouse/ClickHouse/pull/71223) ([Tanya Bragin](https://github.com/tbragin)). -* Fix `WITH TOTALS` in subquery with parallel replicas. [#71224](https://github.com/ClickHouse/ClickHouse/pull/71224) ([Nikita Taranov](https://github.com/nickitat)). -* Ignore `No such key` exceptions in some cases. [#71236](https://github.com/ClickHouse/ClickHouse/pull/71236) ([Antonio Andelic](https://github.com/antonio2368)). -* Make cloud sync title shorter. [#71255](https://github.com/ClickHouse/ClickHouse/pull/71255) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Meetups update. [#71271](https://github.com/ClickHouse/ClickHouse/pull/71271) ([Tanya Bragin](https://github.com/tbragin)). -* Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). -* Expose one more simple merge selector setting. [#71313](https://github.com/ClickHouse/ClickHouse/pull/71313) ([alesapin](https://github.com/alesapin)). -* Better style for some sever-level settings. [#71319](https://github.com/ClickHouse/ClickHouse/pull/71319) ([alesapin](https://github.com/alesapin)). -* Sync some changes. [#71321](https://github.com/ClickHouse/ClickHouse/pull/71321) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add library to requirements for style-check and fix warning. [#71322](https://github.com/ClickHouse/ClickHouse/pull/71322) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix test `test_workload_entity_keeper_storage`: add more retries. [#71325](https://github.com/ClickHouse/ClickHouse/pull/71325) ([Sergei Trifonov](https://github.com/serxa)). -* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. [#71335](https://github.com/ClickHouse/ClickHouse/pull/71335) ([Chang chen](https://github.com/baibaichen)). -* Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. [#71336](https://github.com/ClickHouse/ClickHouse/pull/71336) ([Michael Kolupaev](https://github.com/al13n321)). -* CI: Remove deprecated release script. [#71341](https://github.com/ClickHouse/ClickHouse/pull/71341) ([Max Kainov](https://github.com/maxknv)). -* Update version_date.tsv and changelog after v24.10.1.2812-stable. [#71343](https://github.com/ClickHouse/ClickHouse/pull/71343) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Bump USearch to 2.16.0 and add more tests. [#71344](https://github.com/ClickHouse/ClickHouse/pull/71344) ([Robert Schulze](https://github.com/rschu1ze)). -* check-doc-aspell: Print full path to script in CI report. [#71345](https://github.com/ClickHouse/ClickHouse/pull/71345) ([Vladimir Cherkasov](https://github.com/vdimir)). -* CI: Fix fedora version in create release workflow. [#71347](https://github.com/ClickHouse/ClickHouse/pull/71347) ([Max Kainov](https://github.com/maxknv)). -* fs cache: add assertions. [#71348](https://github.com/ClickHouse/ClickHouse/pull/71348) ([Kseniia Sumarokova](https://github.com/kssenii)). -* More info in TOO_SLOW exception. [#71365](https://github.com/ClickHouse/ClickHouse/pull/71365) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix unused variables around WorkloadEntityStorageBase. [#71367](https://github.com/ClickHouse/ClickHouse/pull/71367) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Allow to prewarm mark cache by system command without enabled setting. [#71368](https://github.com/ClickHouse/ClickHouse/pull/71368) ([Anton Popov](https://github.com/CurtizJ)). -* Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. [#71380](https://github.com/ClickHouse/ClickHouse/pull/71380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Rename `compare8` to `compare16` for consistency. [#71416](https://github.com/ClickHouse/ClickHouse/pull/71416) ([Nikita Taranov](https://github.com/nickitat)). -* Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. [#71418](https://github.com/ClickHouse/ClickHouse/pull/71418) ([Azat Khuzhin](https://github.com/azat)). -* [Experiment] Analyzer: Check what happens after if-condition removal. [#71425](https://github.com/ClickHouse/ClickHouse/pull/71425) ([Dmitry Novik](https://github.com/novikd)). -* Update version_date.tsv and changelog after v24.8.6.70-lts. [#71428](https://github.com/ClickHouse/ClickHouse/pull/71428) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix upgrade check (24.11). [#71438](https://github.com/ClickHouse/ClickHouse/pull/71438) ([Raúl Marín](https://github.com/Algunenano)). -* Fix assert during insert into vector similarity index in presence of other skipping indexes. [#71457](https://github.com/ClickHouse/ClickHouse/pull/71457) ([Robert Schulze](https://github.com/rschu1ze)). -* Avoid `seconds left [-3]` in cache await. [#71468](https://github.com/ClickHouse/ClickHouse/pull/71468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Added tests for corner cases for 24.10. [#71469](https://github.com/ClickHouse/ClickHouse/pull/71469) ([Max Vostrikov](https://github.com/max-vostrikov)). -* Expose base setting for merge selector. [#71497](https://github.com/ClickHouse/ClickHouse/pull/71497) ([alesapin](https://github.com/alesapin)). -* Fixed incorrect settings order `max_parser_depth` and `max_parser_backtracks`. [#71498](https://github.com/ClickHouse/ClickHouse/pull/71498) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Support the endpoint of oss accelerator. [#71502](https://github.com/ClickHouse/ClickHouse/pull/71502) ([Kai Zhu](https://github.com/nauu)). -* Fix flaky test_drop_complex_columns. [#71504](https://github.com/ClickHouse/ClickHouse/pull/71504) ([Ilya Golshtein](https://github.com/ilejn)). -* Move bitShift function changelog entries to backward incompatible. [#71510](https://github.com/ClickHouse/ClickHouse/pull/71510) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix copy/paste error. [#71513](https://github.com/ClickHouse/ClickHouse/pull/71513) ([Denny Crane](https://github.com/den-crane)). -* Allow specifying cmdline flags in integration test. It's needed by [#71452](https://github.com/ClickHouse/ClickHouse/issues/71452) to validate a bugfix. [#71523](https://github.com/ClickHouse/ClickHouse/pull/71523) ([Amos Bird](https://github.com/amosbird)). -* Add ProfileEvents for merge selector timings. [#71524](https://github.com/ClickHouse/ClickHouse/pull/71524) ([alesapin](https://github.com/alesapin)). -* Minor: Remove "experimental" mention of analyzer. [#71525](https://github.com/ClickHouse/ClickHouse/pull/71525) ([Robert Schulze](https://github.com/rschu1ze)). -* Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. [#71530](https://github.com/ClickHouse/ClickHouse/pull/71530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add test to check that accessing system.functions does not populate query_log used_functions. [#71535](https://github.com/ClickHouse/ClickHouse/pull/71535) ([Raúl Marín](https://github.com/Algunenano)). -* Improve `query_plan_merge_filters` optimization. Fixes [#71408](https://github.com/ClickHouse/ClickHouse/issues/71408). [#71539](https://github.com/ClickHouse/ClickHouse/pull/71539) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix compatibility with refreshable materialized views created by old clickhouse servers. [#71556](https://github.com/ClickHouse/ClickHouse/pull/71556) ([Michael Kolupaev](https://github.com/al13n321)). -* Vector similarity index: Re-introduce support for legacy index creation syntax. [#71572](https://github.com/ClickHouse/ClickHouse/pull/71572) ([Robert Schulze](https://github.com/rschu1ze)). -* Avoid port clash in CoordinationTest/0.TestSummingRaft1. [#71584](https://github.com/ClickHouse/ClickHouse/pull/71584) ([Raúl Marín](https://github.com/Algunenano)). -* Fix for `00180_no_seek_avoiding_when_reading_from_cache`. [#71596](https://github.com/ClickHouse/ClickHouse/pull/71596) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix for `test_storage_s3_queue::test_shards_distributed[ordered-2]`. [#71597](https://github.com/ClickHouse/ClickHouse/pull/71597) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Enable enable_job_stack_trace by default, second attempt. [#71625](https://github.com/ClickHouse/ClickHouse/pull/71625) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Update version_date.tsv and changelog after v24.3.13.40-lts. [#71627](https://github.com/ClickHouse/ClickHouse/pull/71627) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fixes the bug regarding max rows/bytes to read. [#71634](https://github.com/ClickHouse/ClickHouse/pull/71634) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* SimSIMD: Improve suppression for msan false positive. [#71635](https://github.com/ClickHouse/ClickHouse/pull/71635) ([Robert Schulze](https://github.com/rschu1ze)). -* Add `min_parts_to_merge_at_once` merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less than `e` (2.71...) data parts at once because it increases both write amplification and parts number. [#71637](https://github.com/ClickHouse/ClickHouse/pull/71637) ([alesapin](https://github.com/alesapin)). -* Fix broken 03247_ghdata_string_to_json_alter. [#71638](https://github.com/ClickHouse/ClickHouse/pull/71638) ([Pavel Kruglov](https://github.com/Avogar)). -* Update test. [#71654](https://github.com/ClickHouse/ClickHouse/pull/71654) ([Kseniia Sumarokova](https://github.com/kssenii)). -* CI: fix mysql containers using improper log directory. [#71655](https://github.com/ClickHouse/ClickHouse/pull/71655) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Update README.md - Update meetups. [#71657](https://github.com/ClickHouse/ClickHouse/pull/71657) ([Tanya Bragin](https://github.com/tbragin)). -* Add index granularity size column to system.parts. [#71658](https://github.com/ClickHouse/ClickHouse/pull/71658) ([alesapin](https://github.com/alesapin)). -* Update PULL_REQUEST_TEMPLATE.md. [#71687](https://github.com/ClickHouse/ClickHouse/pull/71687) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Historically we have a strange cherry-pick branches naming, e.g. `cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e`. The `cherrypick/24.3/62297` looks nicer and more straightforward. [#71698](https://github.com/ClickHouse/ClickHouse/pull/71698) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. [#71715](https://github.com/ClickHouse/ClickHouse/pull/71715) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix flaky test test_prometheus_protocols. [#71772](https://github.com/ClickHouse/ClickHouse/pull/71772) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. [#71782](https://github.com/ClickHouse/ClickHouse/pull/71782) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix processors profile is not collected for subqueries in some cases. [#71787](https://github.com/ClickHouse/ClickHouse/pull/71787) ([Nikita Taranov](https://github.com/nickitat)). -* Lint some stuff. [#71795](https://github.com/ClickHouse/ClickHouse/pull/71795) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Closes [#71780](https://github.com/ClickHouse/ClickHouse/issues/71780). [#71818](https://github.com/ClickHouse/ClickHouse/pull/71818) ([Kseniia Sumarokova](https://github.com/kssenii)). -* relax memory limit for 00755_avg_value_size_hint_passing.sql. [#71820](https://github.com/ClickHouse/ClickHouse/pull/71820) ([Sema Checherinda](https://github.com/CheSema)). -* The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper `Dockerfile.ubuntu` there. [#71825](https://github.com/ClickHouse/ClickHouse/pull/71825) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* tests for parseDateTime64InJodaSyntax. [#71829](https://github.com/ClickHouse/ClickHouse/pull/71829) ([Max Vostrikov](https://github.com/max-vostrikov)). -* Enable build profiling in pull requests. [#71847](https://github.com/ClickHouse/ClickHouse/pull/71847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add check and assertion. [#71856](https://github.com/ClickHouse/ClickHouse/pull/71856) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Some healthcheck is better than nothing. [#71865](https://github.com/ClickHouse/ClickHouse/pull/71865) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* More accurate `calculateCacheKey` implementation. [#71868](https://github.com/ClickHouse/ClickHouse/pull/71868) ([Nikita Taranov](https://github.com/nickitat)). -* add test 03248_max_parts_to_move. [#71869](https://github.com/ClickHouse/ClickHouse/pull/71869) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Update README.md - Update meetups. [#71872](https://github.com/ClickHouse/ClickHouse/pull/71872) ([Tanya Bragin](https://github.com/tbragin)). -* Prevents listing files from s3 while inserting. [#71889](https://github.com/ClickHouse/ClickHouse/pull/71889) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Keep query_plan_merge_filters disabled by default. [#71890](https://github.com/ClickHouse/ClickHouse/pull/71890) ([Raúl Marín](https://github.com/Algunenano)). -* Remove useless code. [#71900](https://github.com/ClickHouse/ClickHouse/pull/71900) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* TreeRewriter: fix typo: `parititon` => `partition`. [#71907](https://github.com/ClickHouse/ClickHouse/pull/71907) ([yun](https://github.com/yokofly)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. [#71912](https://github.com/ClickHouse/ClickHouse/pull/71912) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix test_storage_mongodb/test.py::test_secure_connection_uri. [#71924](https://github.com/ClickHouse/ClickHouse/pull/71924) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Adapt some test to run in non-CI configurations. [#71928](https://github.com/ClickHouse/ClickHouse/pull/71928) ([Raúl Marín](https://github.com/Algunenano)). -* Fix build after [#71179](https://github.com/ClickHouse/ClickHouse/issues/71179). Clang-19 checks for unused variables inside of if-conditions. [#71929](https://github.com/ClickHouse/ClickHouse/pull/71929) ([Dmitry Novik](https://github.com/novikd)). -* Fix flaky test 03262_column_sizes_with_dynamic_structure. [#71931](https://github.com/ClickHouse/ClickHouse/pull/71931) ([Pavel Kruglov](https://github.com/Avogar)). -* Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. [#71944](https://github.com/ClickHouse/ClickHouse/pull/71944) ([Nikita Taranov](https://github.com/nickitat)). -* Enabling `query_plan_merge_filters` again after [#71890](https://github.com/ClickHouse/ClickHouse/issues/71890). [#71964](https://github.com/ClickHouse/ClickHouse/pull/71964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Add comment about bf16 to CMake docs. [#71973](https://github.com/ClickHouse/ClickHouse/pull/71973) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix 00098_primary_key_memory_allocated. [#71977](https://github.com/ClickHouse/ClickHouse/pull/71977) ([Alexander Gololobov](https://github.com/davenger)). -* Add a test for [#71908](https://github.com/ClickHouse/ClickHouse/issues/71908). [#71986](https://github.com/ClickHouse/ClickHouse/pull/71986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Miscellaneous. [#71987](https://github.com/ClickHouse/ClickHouse/pull/71987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Minor follow-up to [#71581](https://github.com/ClickHouse/ClickHouse/issues/71581). [#71993](https://github.com/ClickHouse/ClickHouse/pull/71993) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Stress test with praktika. [#71995](https://github.com/ClickHouse/ClickHouse/pull/71995) ([Max Kainov](https://github.com/maxknv)). -* Fix prewarm of mark cache after adding a new column. [#71996](https://github.com/ClickHouse/ClickHouse/pull/71996) ([Anton Popov](https://github.com/CurtizJ)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. [#72018](https://github.com/ClickHouse/ClickHouse/pull/72018) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix comments. [#72023](https://github.com/ClickHouse/ClickHouse/pull/72023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add test for 33604. [#72026](https://github.com/ClickHouse/ClickHouse/pull/72026) ([Nikita Taranov](https://github.com/nickitat)). -* CI: Remove unsafe secret_envs input from yml workflows. [#72028](https://github.com/ClickHouse/ClickHouse/pull/72028) ([Max Kainov](https://github.com/maxknv)). -* Update version_date.tsv and changelog after v24.10.2.80-stable. [#72029](https://github.com/ClickHouse/ClickHouse/pull/72029) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.8.7.41-lts. [#72037](https://github.com/ClickHouse/ClickHouse/pull/72037) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.9.3.128-stable. [#72041](https://github.com/ClickHouse/ClickHouse/pull/72041) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.3.14.35-lts. [#72042](https://github.com/ClickHouse/ClickHouse/pull/72042) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Save several minutes of build time. [#72046](https://github.com/ClickHouse/ClickHouse/pull/72046) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Update meetups. [#72048](https://github.com/ClickHouse/ClickHouse/pull/72048) ([Tanya Bragin](https://github.com/tbragin)). -* Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix test_disk_over_web_server/. [#72075](https://github.com/ClickHouse/ClickHouse/pull/72075) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Minor improvement for system.query_metric_log stateless test. [#72076](https://github.com/ClickHouse/ClickHouse/pull/72076) ([Pablo Marcos](https://github.com/pamarcos)). -* A follow-up for [#72057](https://github.com/ClickHouse/ClickHouse/issues/72057) and https://github.com/ClickHouse/ClickHouse/pull/71505. [#72079](https://github.com/ClickHouse/ClickHouse/pull/72079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add google-cloud-cpp submodule. [#72092](https://github.com/ClickHouse/ClickHouse/pull/72092) ([Pablo Marcos](https://github.com/pamarcos)). -* CI: Enable fuzzer job in Nightly workflow. [#72101](https://github.com/ClickHouse/ClickHouse/pull/72101) ([Max Kainov](https://github.com/maxknv)). -* Get rid of code duplication after adding `CHECK GRANT` in https://github.com/ClickHouse/ClickHouse/pull/68885. [#72103](https://github.com/ClickHouse/ClickHouse/pull/72103) ([Vitaly Baranov](https://github.com/vitlibar)). -* Add jwt-cpp submodule. [#72104](https://github.com/ClickHouse/ClickHouse/pull/72104) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix list-licenses.sh with OSX. [#72107](https://github.com/ClickHouse/ClickHouse/pull/72107) ([Raúl Marín](https://github.com/Algunenano)). -* fix cancelation for PartitionedSink. [#72126](https://github.com/ClickHouse/ClickHouse/pull/72126) ([Sema Checherinda](https://github.com/CheSema)). -* FIx 02374_analyzer_join_using. [#72145](https://github.com/ClickHouse/ClickHouse/pull/72145) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed a test which was flaky-flaky. [#72147](https://github.com/ClickHouse/ClickHouse/pull/72147) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - -#### Backward Incompatible Change -* Remove system tables `generate_series` and `generateSeries`. They were added by mistake here: [#59390](https://github.com/ClickHouse/ClickHouse/issues/59390). [#71091](https://github.com/ClickHouse/ClickHouse/pull/71091) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove `StorageExternalDistributed`. Closes [#70600](https://github.com/ClickHouse/ClickHouse/issues/70600). ### Documentation entry for user-facing changes. [#71176](https://github.com/ClickHouse/ClickHouse/pull/71176) ([flynn](https://github.com/ucasfl)). -* Fix possible error `No such file or directory` due to unescaped special symbols in files for JSON subcolumns. [#71182](https://github.com/ClickHouse/ClickHouse/pull/71182) ([Pavel Kruglov](https://github.com/Avogar)). -* The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the `SOURCES` hierarchy. Add grants to any non-default database users that create tables with these engine types. [#71250](https://github.com/ClickHouse/ClickHouse/pull/71250) ([Christoph Wurm](https://github.com/cwurm)). -* Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. [#71300](https://github.com/ClickHouse/ClickHouse/pull/71300) ([Christoph Wurm](https://github.com/cwurm)). -* Rename filesystem cache setting `skip_download_if_exceeds_query_cache` to `filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit`. [#71578](https://github.com/ClickHouse/ClickHouse/pull/71578) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Remove support for `Enum` as well as `UInt128` and `UInt256` arguments in `deltaSumTimestamp`. Remove support for `Int8`, `UInt8`, `Int16`, and `UInt16` of the second ("timestamp") argument of `deltaSumTimestamp`. [#71790](https://github.com/ClickHouse/ClickHouse/pull/71790) ([Alexey Milovidov](https://github.com/alexey-milovidov)). - -#### New Feature -* A new data type, `BFloat16`, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes [#44206](https://github.com/ClickHouse/ClickHouse/issues/44206). This closes [#49937](https://github.com/ClickHouse/ClickHouse/issues/49937). [#64712](https://github.com/ClickHouse/ClickHouse/pull/64712) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* ~~Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled by `query_plan_join_inner_table_selection`, which can be set to `auto`. In this mode, ClickHouse will try to choose the table with the smallest number of rows.~~ Resubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. [#68682](https://github.com/ClickHouse/ClickHouse/pull/68682) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Add `CHECK GRANT` query to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. [#68885](https://github.com/ClickHouse/ClickHouse/pull/68885) ([Unalian](https://github.com/Unalian)). -* Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. [#69187](https://github.com/ClickHouse/ClickHouse/pull/69187) ([Sergei Trifonov](https://github.com/serxa)). -* Added server setting `async_load_system_database` that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. [#69847](https://github.com/ClickHouse/ClickHouse/pull/69847) ([Sergei Trifonov](https://github.com/serxa)). -* Allow each authentication method to have its own expiration date, remove from user entity. [#70090](https://github.com/ClickHouse/ClickHouse/pull/70090) ([Arthur Passos](https://github.com/arthurpassos)). -* Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). [#70332](https://github.com/ClickHouse/ClickHouse/pull/70332) ([Andrey Zvonov](https://github.com/zvonand)). -* Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting `merge_tree_use_v1_object_and_dynamic_serialization` (can be used during upgrade to be able to rollback the version without issues). [#70442](https://github.com/ClickHouse/ClickHouse/pull/70442) ([Pavel Kruglov](https://github.com/Avogar)). -* Added a new header type for S3 endpoints for user authentication (`access_header`). This allows to get some access header with the lowest priority, which will be overwritten with `access_key_id` from any other source (for example, a table schema or a named collection). [#71011](https://github.com/ClickHouse/ClickHouse/pull/71011) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* Initial implementation of settings tiers. [#71145](https://github.com/ClickHouse/ClickHouse/pull/71145) ([Raúl Marín](https://github.com/Algunenano)). -* Add support for staleness clause in order by with fill operator. [#71151](https://github.com/ClickHouse/ClickHouse/pull/71151) ([Mikhail Artemenko](https://github.com/Michicosun)). -* Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. [#71320](https://github.com/ClickHouse/ClickHouse/pull/71320) ([Pavel Kruglov](https://github.com/Avogar)). -* Added aliases `anyRespectNulls`, `firstValueRespectNulls`, and `anyValueRespectNulls` for aggregation function `any`. Also added aliases `anyLastRespectNulls` and `lastValueRespectNulls` for aggregation function `anyLast`. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example: `SELECT anyLastRespectNullsStateIf` instead of `anyLast_respect_nullsStateIf`. [#71403](https://github.com/ClickHouse/ClickHouse/pull/71403) ([Peter Nguyen](https://github.com/petern48)). -* Added the configuration `date_time_utc` parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. [#71560](https://github.com/ClickHouse/ClickHouse/pull/71560) ([Ali](https://github.com/xogoodnow)). -* Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting `use_const_adaptive_granularity`), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. [#71786](https://github.com/ClickHouse/ClickHouse/pull/71786) ([Anton Popov](https://github.com/CurtizJ)). -* Implement `allowed_feature_tier` as a global switch to disable all experimental / beta features. [#71841](https://github.com/ClickHouse/ClickHouse/pull/71841) ([Raúl Marín](https://github.com/Algunenano)). -* Add `iceberg[S3;HDFS;Azure]Cluster`, `deltaLakeCluster`, `hudiCluster` table functions. [#72045](https://github.com/ClickHouse/ClickHouse/pull/72045) ([Mikhail Artemenko](https://github.com/Michicosun)). - -#### Performance Improvement -* Add 2 new settings `short_circuit_function_evaluation_for_nulls` and `short_circuit_function_evaluation_for_nulls_threshold` that allow to execute functions over `Nullable` columns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. [#60129](https://github.com/ClickHouse/ClickHouse/pull/60129) ([李扬](https://github.com/taiyang-li)). -* Now we won't copy input blocks columns for `join_algorithm='parallel_hash'` when distribute them between threads for parallel processing. [#67782](https://github.com/ClickHouse/ClickHouse/pull/67782) ([Nikita Taranov](https://github.com/nickitat)). -* Optimized `Replacing` merge algorithm for non intersecting parts. [#70977](https://github.com/ClickHouse/ClickHouse/pull/70977) ([Anton Popov](https://github.com/CurtizJ)). -* Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). - -#### Improvement -* Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). -* Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). -* Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). -* Fix use-after-dtor logic in HashTable destroyElements. [#65279](https://github.com/ClickHouse/ClickHouse/pull/65279) ([cangyin](https://github.com/cangyin)). -* Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. [#68800](https://github.com/ClickHouse/ClickHouse/pull/68800) ([Sema Checherinda](https://github.com/CheSema)). -* Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. [#69658](https://github.com/ClickHouse/ClickHouse/pull/69658) ([tuanpach](https://github.com/tuanpach)). -* 1. Refactor `DDLQueryStatusSource`: * Rename `DDLQueryStatusSource` to `DistributedQueryStatusSource`, and make it a base class * Create two subclasses `DDLOnClusterQueryStatusSource` and `ReplicatedDatabaseQueryStatusSource` derived from `DDLQueryStatusSource` to query the status of DDL tasks from `DDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in `DDLOnClusterQueryStatusSource`. [#69660](https://github.com/ClickHouse/ClickHouse/pull/69660) ([tuanpach](https://github.com/tuanpach)). -* Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. [#69731](https://github.com/ClickHouse/ClickHouse/pull/69731) ([Pavel Kruglov](https://github.com/Avogar)). -* Better error-handling and cancellation of `ON CLUSTER` backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues with `test_disallow_concurrency` - now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. [#70027](https://github.com/ClickHouse/ClickHouse/pull/70027) ([Vitaly Baranov](https://github.com/vitlibar)). -* Enable `parallel_replicas_local_plan` by default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. [#70171](https://github.com/ClickHouse/ClickHouse/pull/70171) ([Igor Nikonov](https://github.com/devcrafter)). -* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). -* Add ability to set user/password in http_handlers (for `dynamic_query_handler`/`predefined_query_handler`). [#70725](https://github.com/ClickHouse/ClickHouse/pull/70725) ([Azat Khuzhin](https://github.com/azat)). -* Support `ALTER TABLE ... MODIFY/RESET SETTING ...` for certain settings in storage S3Queue. [#70811](https://github.com/ClickHouse/ClickHouse/pull/70811) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. [#70823](https://github.com/ClickHouse/ClickHouse/pull/70823) ([Julia Kartseva](https://github.com/jkartseva)). -* Add `--threads` parameter to `clickhouse-compressor`, which allows to compress data in parallel. [#70860](https://github.com/ClickHouse/ClickHouse/pull/70860) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). -* Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. [#70997](https://github.com/ClickHouse/ClickHouse/pull/70997) ([Roman Antonov](https://github.com/Romeo58rus)). -* Refactored internal structure of files which work with DataLake Storages. [#71012](https://github.com/ClickHouse/ClickHouse/pull/71012) ([Daniil Ivanik](https://github.com/divanik)). -* Make the Replxx client history size configurable. [#71014](https://github.com/ClickHouse/ClickHouse/pull/71014) ([Jiří Kozlovský](https://github.com/jirislav)). -* Added a setting `prewarm_mark_cache` which enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. [#71053](https://github.com/ClickHouse/ClickHouse/pull/71053) ([Anton Popov](https://github.com/CurtizJ)). -* Boolean support for parquet native reader. [#71055](https://github.com/ClickHouse/ClickHouse/pull/71055) ([Arthur Passos](https://github.com/arthurpassos)). -* Retry more errors when interacting with S3, such as "Malformed message". [#71088](https://github.com/ClickHouse/ClickHouse/pull/71088) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Lower log level for some messages about S3. [#71090](https://github.com/ClickHouse/ClickHouse/pull/71090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Support write hdfs files with space. [#71105](https://github.com/ClickHouse/ClickHouse/pull/71105) ([exmy](https://github.com/exmy)). -* `system.session_log` is quite okay. This closes [#51760](https://github.com/ClickHouse/ClickHouse/issues/51760). [#71150](https://github.com/ClickHouse/ClickHouse/pull/71150) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). -* Added settings limiting the number of replicated tables, dictionaries and views. [#71179](https://github.com/ClickHouse/ClickHouse/pull/71179) ([Kirill](https://github.com/kirillgarbar)). -* Use `AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE` instead of `AWS_CONTAINER_AUTHORIZATION_TOKEN` if former is available. Fixes [#71074](https://github.com/ClickHouse/ClickHouse/issues/71074). [#71269](https://github.com/ClickHouse/ClickHouse/pull/71269) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. [#71385](https://github.com/ClickHouse/ClickHouse/pull/71385) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). -* Add per host dashboards `Overview (host)` and `Cloud overview (host)` to advanced dashboard. [#71422](https://github.com/ClickHouse/ClickHouse/pull/71422) ([alesapin](https://github.com/alesapin)). -* The methods `removeObject` and `removeObjects` are not idempotent. When retries happen due to network errors, the result could be `object not found` because it has been deleted at previous attempts. [#71529](https://github.com/ClickHouse/ClickHouse/pull/71529) ([Sema Checherinda](https://github.com/CheSema)). -* Added new functions `parseDateTime64`, `parseDateTime64OrNull` and `parseDateTime64OrZero`. Compared to the existing function `parseDateTime` (and variants), they return a value of type `DateTime64` instead of `DateTime`. [#71581](https://github.com/ClickHouse/ClickHouse/pull/71581) ([kevinyhzou](https://github.com/KevinyhZou)). -* Allow using clickhouse with a file argument as --queries-file. [#71589](https://github.com/ClickHouse/ClickHouse/pull/71589) ([Raúl Marín](https://github.com/Algunenano)). -* Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. [#71595](https://github.com/ClickHouse/ClickHouse/pull/71595) ([alesapin](https://github.com/alesapin)). -* `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. ### Documentation entry for user-facing changes. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Forbid Dynamic/Variant types in min/max functions to avoid confusion. [#71761](https://github.com/ClickHouse/ClickHouse/pull/71761) ([Pavel Kruglov](https://github.com/Avogar)). -* Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). -* Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). -* Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). -* Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Turn-off filesystem cache setting `boundary_alignment` for non-disk read. [#71827](https://github.com/ClickHouse/ClickHouse/pull/71827) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Update `HostResolver` 3 times in a `history` period. [#71863](https://github.com/ClickHouse/ClickHouse/pull/71863) ([Sema Checherinda](https://github.com/CheSema)). -* Queries like 'SELECT * FROM t LIMIT 1' used to load part indexes even though they were not used. [#71866](https://github.com/ClickHouse/ClickHouse/pull/71866) ([Alexander Gololobov](https://github.com/davenger)). -* Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). -* Do not increment the `ILLEGAL_TYPE_OF_ARGUMENT` counter in the `system.errors` table when the `bitmapTransform` function is used, and argument types are valid. [#71971](https://github.com/ClickHouse/ClickHouse/pull/71971) ([Dmitry Novik](https://github.com/novikd)). -* When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). - -#### Bug Fix (user-visible misbehavior in an official stable release) -* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). -* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). -* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). -* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* `GroupArraySortedData` uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. Then `GroupArraySortedData`'s destructor called destructors on uninitialized elements and crashed: ``` 2024.10.17 22:58:23.523790 [ 5233 ] {} BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} BaseDaemon: 4. DB::IAggregateFunctionDataHelper, DB::(anonymous namespace)::GroupArraySorted, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW::immutable_ptr&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr&, std::unordered_map::immutable_ptr, std::hash, std::equal_to, std::allocator::immutable_ptr>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW::immutable_ptr&, unsigned long, std::function const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector::immutable_ptr, std::allocator::immutable_ptr>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} BaseDaemon: 21. DB::MergeTreeBackgroundExecutor::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} BaseDaemon: 22. ThreadPoolImpl>::worker(std::__list_iterator, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} BaseDaemon: 23. void std::__function::__policy_invoker::__call_impl::ThreadFromGlobalPoolImpl>::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]>, void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort' ```. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). -* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). -* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). -* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). -* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). -* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). -* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). -* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). -* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). -* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). -* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). -* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). -* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). -* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). -* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). -* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). -* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). -* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). -* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). -* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). -* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). -* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). -* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). -* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). -* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). -* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). -* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). -* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). -* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). -* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). -* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). -* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). -* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). -* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). -* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). -* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). -* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). -* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). -* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). - -#### Build/Testing/Packaging Improvement -* Add the script to update sources of [docker official library](https://github.com/ClickHouse/docker-library). [#57203](https://github.com/ClickHouse/ClickHouse/pull/57203) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* The build system will prevent libraries with unexpected licenses. [#70988](https://github.com/ClickHouse/ClickHouse/pull/70988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Upgrade docker base image for clickhouse-server and keeper to `ubuntu:22.04`. **Breaking change**: the minimal supported docker version is `20.10.10`. [#71505](https://github.com/ClickHouse/ClickHouse/pull/71505) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Improve clickhouse-server Dockerfile.ubuntu. Deprecate `CLICKHOUSE_UID/CLICKHOUSE_GID` envs. Remove `CLICKHOUSE_DOCKER_RESTART_ON_EXIT` processing to complien requirements. Consistent `clickhouse/clickhouse-server/clickhouse-keeper` execution to not have it plain in one place and `/usr/bin/clickhouse*` in another. [#71573](https://github.com/ClickHouse/ClickHouse/pull/71573) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - -#### NO CL ENTRY - -* NO CL ENTRY: 'Revert "Miscellaneous"'. [#71083](https://github.com/ClickHouse/ClickHouse/pull/71083) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. [#71084](https://github.com/ClickHouse/ClickHouse/pull/71084) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. [#71220](https://github.com/ClickHouse/ClickHouse/pull/71220) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. [#71251](https://github.com/ClickHouse/ClickHouse/pull/71251) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. [#71266](https://github.com/ClickHouse/ClickHouse/pull/71266) ([Sergei Trifonov](https://github.com/serxa)). -* NO CL ENTRY: 'Revert "Selection of hash join inner table"'. [#71527](https://github.com/ClickHouse/ClickHouse/pull/71527) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. [#71619](https://github.com/ClickHouse/ClickHouse/pull/71619) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. [#71914](https://github.com/ClickHouse/ClickHouse/pull/71914) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. [#71945](https://github.com/ClickHouse/ClickHouse/pull/71945) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. [#71974](https://github.com/ClickHouse/ClickHouse/pull/71974) ([Max Kainov](https://github.com/maxknv)). -* NO CL ENTRY: 'CI: Functional Tests with praktika'. [#71976](https://github.com/ClickHouse/ClickHouse/pull/71976) ([Max Kainov](https://github.com/maxknv)). - -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Refactor TempDataOnDisk. [#66606](https://github.com/ClickHouse/ClickHouse/pull/66606) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Bump krb5 from v1.21.2 to v1.21.3. [#69360](https://github.com/ClickHouse/ClickHouse/pull/69360) ([Robert Schulze](https://github.com/rschu1ze)). -* USearch: Enable SimSIMD backend + enable dynamic dispatch. [#69387](https://github.com/ClickHouse/ClickHouse/pull/69387) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: enable libfuzzer. [#70112](https://github.com/ClickHouse/ClickHouse/pull/70112) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Use `clang-19`. [#70414](https://github.com/ClickHouse/ClickHouse/pull/70414) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Randomize Keeper feature flags in integration tests. [#70523](https://github.com/ClickHouse/ClickHouse/pull/70523) ([Antonio Andelic](https://github.com/antonio2368)). -* All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. [#70691](https://github.com/ClickHouse/ClickHouse/pull/70691) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Check number of arguments for function with Dynamic argument. [#70749](https://github.com/ClickHouse/ClickHouse/pull/70749) ([Nikita Taranov](https://github.com/nickitat)). -* Add a settings `filesystem_cache_enable_background_download_for_metadata_files` for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. [#70806](https://github.com/ClickHouse/ClickHouse/pull/70806) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Make ParquetMetadata say whether bloom filter is present. [#70947](https://github.com/ClickHouse/ClickHouse/pull/70947) ([Michael Kolupaev](https://github.com/al13n321)). -* test for reproducing that ReplacingMergeTree depends on the order of part attachment. [#71010](https://github.com/ClickHouse/ClickHouse/pull/71010) ([Konstantin Morozov](https://github.com/k-morozov)). -* CI: Build Job with praktika. [#71015](https://github.com/ClickHouse/ClickHouse/pull/71015) ([Max Kainov](https://github.com/maxknv)). -* Fix bad test `01524_do_not_merge_across_partitions_select_final.sql`. [#71035](https://github.com/ClickHouse/ClickHouse/pull/71035) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Enable enable_job_stack_trace by default. [#71039](https://github.com/ClickHouse/ClickHouse/pull/71039) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix two logical errors when reading from stdin in clickhouse local. [#71046](https://github.com/ClickHouse/ClickHouse/pull/71046) ([Michael Kolupaev](https://github.com/al13n321)). -* Sync changes to `ProtocolServerAdapter`. [#71058](https://github.com/ClickHouse/ClickHouse/pull/71058) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix a typo. [#71067](https://github.com/ClickHouse/ClickHouse/pull/71067) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Miscellaneous. [#71070](https://github.com/ClickHouse/ClickHouse/pull/71070) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove bad test `test_system_replicated_fetches`. [#71071](https://github.com/ClickHouse/ClickHouse/pull/71071) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update version after release. [#71076](https://github.com/ClickHouse/ClickHouse/pull/71076) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* CI: Functional tests for ARM + ASAN binary. [#71079](https://github.com/ClickHouse/ClickHouse/pull/71079) ([Max Kainov](https://github.com/maxknv)). -* CI: Functional Tests with praktika. [#71081](https://github.com/ClickHouse/ClickHouse/pull/71081) ([Max Kainov](https://github.com/maxknv)). -* Fixup of TrivialMergeSelector. [#71082](https://github.com/ClickHouse/ClickHouse/pull/71082) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Sync integration test with private. [#71096](https://github.com/ClickHouse/ClickHouse/pull/71096) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Unique symbols in the `system.coverage_log`. [#71099](https://github.com/ClickHouse/ClickHouse/pull/71099) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Better log messages. [#71102](https://github.com/ClickHouse/ClickHouse/pull/71102) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix error in Replicated database. [#71103](https://github.com/ClickHouse/ClickHouse/pull/71103) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update error message for JSONAsObject format. [#71123](https://github.com/ClickHouse/ClickHouse/pull/71123) ([Pavel Kruglov](https://github.com/Avogar)). -* Initial changelog for 24.10. [#71127](https://github.com/ClickHouse/ClickHouse/pull/71127) ([Raúl Marín](https://github.com/Algunenano)). -* Followup [#70520](https://github.com/ClickHouse/ClickHouse/issues/70520). [#71129](https://github.com/ClickHouse/ClickHouse/pull/71129) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Update compatibility setting for `hnsw_candidate_list_size_for_search`. [#71133](https://github.com/ClickHouse/ClickHouse/pull/71133) ([Robert Schulze](https://github.com/rschu1ze)). -* Try fix rabbitmq. [#71143](https://github.com/ClickHouse/ClickHouse/pull/71143) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Randomize setting `enable_vertical_final`. [#71144](https://github.com/ClickHouse/ClickHouse/pull/71144) ([Anton Popov](https://github.com/CurtizJ)). -* Fix bad test `02561_sorting_constants_and_distinct_crash`. [#71147](https://github.com/ClickHouse/ClickHouse/pull/71147) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add test to verify [#62308](https://github.com/ClickHouse/ClickHouse/issues/62308) works. [#71149](https://github.com/ClickHouse/ClickHouse/pull/71149) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). -* Fix 02932_refreshable_materialized_views_1 flakiness. [#71160](https://github.com/ClickHouse/ClickHouse/pull/71160) ([Michael Kolupaev](https://github.com/al13n321)). -* Use `_minmax_count_projection` instead of `Optimized trivial count` for `ReadFromPreparedSource` node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses [#70939](https://github.com/ClickHouse/ClickHouse/issues/70939). [#71166](https://github.com/ClickHouse/ClickHouse/pull/71166) ([Amos Bird](https://github.com/amosbird)). -* Close [#8687](https://github.com/ClickHouse/ClickHouse/issues/8687). [#71169](https://github.com/ClickHouse/ClickHouse/pull/71169) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fixes for interactive metrics. [#71173](https://github.com/ClickHouse/ClickHouse/pull/71173) ([Julia Kartseva](https://github.com/jkartseva)). -* Maybe not GWPAsan by default. [#71174](https://github.com/ClickHouse/ClickHouse/pull/71174) ([Antonio Andelic](https://github.com/antonio2368)). -* Print compression method in `clickhouse-compressor --stat`. Useful for inspecting random data files. [#71192](https://github.com/ClickHouse/ClickHouse/pull/71192) ([Amos Bird](https://github.com/amosbird)). -* Updating the events into the recent category and adding the new york event. [#71194](https://github.com/ClickHouse/ClickHouse/pull/71194) ([Zoe Steinkamp](https://github.com/zoesteinkamp)). -* Improve error and log messages around memory usage. [#71195](https://github.com/ClickHouse/ClickHouse/pull/71195) ([Raúl Marín](https://github.com/Algunenano)). -* Minor test adjustments. [#71199](https://github.com/ClickHouse/ClickHouse/pull/71199) ([Raúl Marín](https://github.com/Algunenano)). -* Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. [#71216](https://github.com/ClickHouse/ClickHouse/pull/71216) ([Raúl Marín](https://github.com/Algunenano)). -* Disable enable_named_columns_in_function_tuple for 24.10. [#71219](https://github.com/ClickHouse/ClickHouse/pull/71219) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Update meetups. [#71223](https://github.com/ClickHouse/ClickHouse/pull/71223) ([Tanya Bragin](https://github.com/tbragin)). -* Fix `WITH TOTALS` in subquery with parallel replicas. [#71224](https://github.com/ClickHouse/ClickHouse/pull/71224) ([Nikita Taranov](https://github.com/nickitat)). -* Ignore `No such key` exceptions in some cases. [#71236](https://github.com/ClickHouse/ClickHouse/pull/71236) ([Antonio Andelic](https://github.com/antonio2368)). -* Make cloud sync title shorter. [#71255](https://github.com/ClickHouse/ClickHouse/pull/71255) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Meetups update. [#71271](https://github.com/ClickHouse/ClickHouse/pull/71271) ([Tanya Bragin](https://github.com/tbragin)). -* Improve system.query_metric_log to remove flakiness. [#71295](https://github.com/ClickHouse/ClickHouse/pull/71295) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix debug log timestamp. [#71311](https://github.com/ClickHouse/ClickHouse/pull/71311) ([Pablo Marcos](https://github.com/pamarcos)). -* Expose one more simple merge selector setting. [#71313](https://github.com/ClickHouse/ClickHouse/pull/71313) ([alesapin](https://github.com/alesapin)). -* Better style for some sever-level settings. [#71319](https://github.com/ClickHouse/ClickHouse/pull/71319) ([alesapin](https://github.com/alesapin)). -* Sync some changes. [#71321](https://github.com/ClickHouse/ClickHouse/pull/71321) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add library to requirements for style-check and fix warning. [#71322](https://github.com/ClickHouse/ClickHouse/pull/71322) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix test `test_workload_entity_keeper_storage`: add more retries. [#71325](https://github.com/ClickHouse/ClickHouse/pull/71325) ([Sergei Trifonov](https://github.com/serxa)). -* Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. [#71335](https://github.com/ClickHouse/ClickHouse/pull/71335) ([Chang chen](https://github.com/baibaichen)). -* Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. [#71336](https://github.com/ClickHouse/ClickHouse/pull/71336) ([Michael Kolupaev](https://github.com/al13n321)). -* CI: Remove deprecated release script. [#71341](https://github.com/ClickHouse/ClickHouse/pull/71341) ([Max Kainov](https://github.com/maxknv)). -* Update version_date.tsv and changelog after v24.10.1.2812-stable. [#71343](https://github.com/ClickHouse/ClickHouse/pull/71343) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Bump USearch to 2.16.0 and add more tests. [#71344](https://github.com/ClickHouse/ClickHouse/pull/71344) ([Robert Schulze](https://github.com/rschu1ze)). -* check-doc-aspell: Print full path to script in CI report. [#71345](https://github.com/ClickHouse/ClickHouse/pull/71345) ([Vladimir Cherkasov](https://github.com/vdimir)). -* CI: Fix fedora version in create release workflow. [#71347](https://github.com/ClickHouse/ClickHouse/pull/71347) ([Max Kainov](https://github.com/maxknv)). -* fs cache: add assertions. [#71348](https://github.com/ClickHouse/ClickHouse/pull/71348) ([Kseniia Sumarokova](https://github.com/kssenii)). -* More info in TOO_SLOW exception. [#71365](https://github.com/ClickHouse/ClickHouse/pull/71365) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fix unused variables around WorkloadEntityStorageBase. [#71367](https://github.com/ClickHouse/ClickHouse/pull/71367) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Allow to prewarm mark cache by system command without enabled setting. [#71368](https://github.com/ClickHouse/ClickHouse/pull/71368) ([Anton Popov](https://github.com/CurtizJ)). -* Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. [#71380](https://github.com/ClickHouse/ClickHouse/pull/71380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Rename `compare8` to `compare16` for consistency. [#71416](https://github.com/ClickHouse/ClickHouse/pull/71416) ([Nikita Taranov](https://github.com/nickitat)). -* Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. [#71418](https://github.com/ClickHouse/ClickHouse/pull/71418) ([Azat Khuzhin](https://github.com/azat)). -* [Experiment] Analyzer: Check what happens after if-condition removal. [#71425](https://github.com/ClickHouse/ClickHouse/pull/71425) ([Dmitry Novik](https://github.com/novikd)). -* Update version_date.tsv and changelog after v24.8.6.70-lts. [#71428](https://github.com/ClickHouse/ClickHouse/pull/71428) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix upgrade check (24.11). [#71438](https://github.com/ClickHouse/ClickHouse/pull/71438) ([Raúl Marín](https://github.com/Algunenano)). -* Fix assert during insert into vector similarity index in presence of other skipping indexes. [#71457](https://github.com/ClickHouse/ClickHouse/pull/71457) ([Robert Schulze](https://github.com/rschu1ze)). -* Avoid `seconds left [-3]` in cache await. [#71468](https://github.com/ClickHouse/ClickHouse/pull/71468) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Added tests for corner cases for 24.10. [#71469](https://github.com/ClickHouse/ClickHouse/pull/71469) ([Max Vostrikov](https://github.com/max-vostrikov)). -* Expose base setting for merge selector. [#71497](https://github.com/ClickHouse/ClickHouse/pull/71497) ([alesapin](https://github.com/alesapin)). -* Fixed incorrect settings order `max_parser_depth` and `max_parser_backtracks`. [#71498](https://github.com/ClickHouse/ClickHouse/pull/71498) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Support the endpoint of oss accelerator. [#71502](https://github.com/ClickHouse/ClickHouse/pull/71502) ([Kai Zhu](https://github.com/nauu)). -* Fix flaky test_drop_complex_columns. [#71504](https://github.com/ClickHouse/ClickHouse/pull/71504) ([Ilya Golshtein](https://github.com/ilejn)). -* Move bitShift function changelog entries to backward incompatible. [#71510](https://github.com/ClickHouse/ClickHouse/pull/71510) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix copy/paste error. [#71513](https://github.com/ClickHouse/ClickHouse/pull/71513) ([Denny Crane](https://github.com/den-crane)). -* Allow specifying cmdline flags in integration test. It's needed by [#71452](https://github.com/ClickHouse/ClickHouse/issues/71452) to validate a bugfix. [#71523](https://github.com/ClickHouse/ClickHouse/pull/71523) ([Amos Bird](https://github.com/amosbird)). -* Add ProfileEvents for merge selector timings. [#71524](https://github.com/ClickHouse/ClickHouse/pull/71524) ([alesapin](https://github.com/alesapin)). -* Minor: Remove "experimental" mention of analyzer. [#71525](https://github.com/ClickHouse/ClickHouse/pull/71525) ([Robert Schulze](https://github.com/rschu1ze)). -* Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. [#71530](https://github.com/ClickHouse/ClickHouse/pull/71530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add test to check that accessing system.functions does not populate query_log used_functions. [#71535](https://github.com/ClickHouse/ClickHouse/pull/71535) ([Raúl Marín](https://github.com/Algunenano)). -* Improve `query_plan_merge_filters` optimization. Fixes [#71408](https://github.com/ClickHouse/ClickHouse/issues/71408). [#71539](https://github.com/ClickHouse/ClickHouse/pull/71539) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix compatibility with refreshable materialized views created by old clickhouse servers. [#71556](https://github.com/ClickHouse/ClickHouse/pull/71556) ([Michael Kolupaev](https://github.com/al13n321)). -* Vector similarity index: Re-introduce support for legacy index creation syntax. [#71572](https://github.com/ClickHouse/ClickHouse/pull/71572) ([Robert Schulze](https://github.com/rschu1ze)). -* Avoid port clash in CoordinationTest/0.TestSummingRaft1. [#71584](https://github.com/ClickHouse/ClickHouse/pull/71584) ([Raúl Marín](https://github.com/Algunenano)). -* Fix for `00180_no_seek_avoiding_when_reading_from_cache`. [#71596](https://github.com/ClickHouse/ClickHouse/pull/71596) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix for `test_storage_s3_queue::test_shards_distributed[ordered-2]`. [#71597](https://github.com/ClickHouse/ClickHouse/pull/71597) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Enable enable_job_stack_trace by default, second attempt. [#71625](https://github.com/ClickHouse/ClickHouse/pull/71625) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Update version_date.tsv and changelog after v24.3.13.40-lts. [#71627](https://github.com/ClickHouse/ClickHouse/pull/71627) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fixes the bug regarding max rows/bytes to read. [#71634](https://github.com/ClickHouse/ClickHouse/pull/71634) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* SimSIMD: Improve suppression for msan false positive. [#71635](https://github.com/ClickHouse/ClickHouse/pull/71635) ([Robert Schulze](https://github.com/rschu1ze)). -* Add `min_parts_to_merge_at_once` merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less than `e` (2.71...) data parts at once because it increases both write amplification and parts number. [#71637](https://github.com/ClickHouse/ClickHouse/pull/71637) ([alesapin](https://github.com/alesapin)). -* Fix broken 03247_ghdata_string_to_json_alter. [#71638](https://github.com/ClickHouse/ClickHouse/pull/71638) ([Pavel Kruglov](https://github.com/Avogar)). -* Update test. [#71654](https://github.com/ClickHouse/ClickHouse/pull/71654) ([Kseniia Sumarokova](https://github.com/kssenii)). -* CI: fix mysql containers using improper log directory. [#71655](https://github.com/ClickHouse/ClickHouse/pull/71655) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Update README.md - Update meetups. [#71657](https://github.com/ClickHouse/ClickHouse/pull/71657) ([Tanya Bragin](https://github.com/tbragin)). -* Add index granularity size column to system.parts. [#71658](https://github.com/ClickHouse/ClickHouse/pull/71658) ([alesapin](https://github.com/alesapin)). -* Update PULL_REQUEST_TEMPLATE.md. [#71687](https://github.com/ClickHouse/ClickHouse/pull/71687) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Historically we have a strange cherry-pick branches naming, e.g. `cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e`. The `cherrypick/24.3/62297` looks nicer and more straightforward. [#71698](https://github.com/ClickHouse/ClickHouse/pull/71698) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. [#71715](https://github.com/ClickHouse/ClickHouse/pull/71715) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix flaky test test_prometheus_protocols. [#71772](https://github.com/ClickHouse/ClickHouse/pull/71772) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. [#71782](https://github.com/ClickHouse/ClickHouse/pull/71782) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix processors profile is not collected for subqueries in some cases. [#71787](https://github.com/ClickHouse/ClickHouse/pull/71787) ([Nikita Taranov](https://github.com/nickitat)). -* Lint some stuff. [#71795](https://github.com/ClickHouse/ClickHouse/pull/71795) ([Konstantin Bogdanov](https://github.com/thevar1able)). -* Closes [#71780](https://github.com/ClickHouse/ClickHouse/issues/71780). [#71818](https://github.com/ClickHouse/ClickHouse/pull/71818) ([Kseniia Sumarokova](https://github.com/kssenii)). -* relax memory limit for 00755_avg_value_size_hint_passing.sql. [#71820](https://github.com/ClickHouse/ClickHouse/pull/71820) ([Sema Checherinda](https://github.com/CheSema)). -* The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper `Dockerfile.ubuntu` there. [#71825](https://github.com/ClickHouse/ClickHouse/pull/71825) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* tests for parseDateTime64InJodaSyntax. [#71829](https://github.com/ClickHouse/ClickHouse/pull/71829) ([Max Vostrikov](https://github.com/max-vostrikov)). -* Enable build profiling in pull requests. [#71847](https://github.com/ClickHouse/ClickHouse/pull/71847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add check and assertion. [#71856](https://github.com/ClickHouse/ClickHouse/pull/71856) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Some healthcheck is better than nothing. [#71865](https://github.com/ClickHouse/ClickHouse/pull/71865) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* More accurate `calculateCacheKey` implementation. [#71868](https://github.com/ClickHouse/ClickHouse/pull/71868) ([Nikita Taranov](https://github.com/nickitat)). -* add test 03248_max_parts_to_move. [#71869](https://github.com/ClickHouse/ClickHouse/pull/71869) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Update README.md - Update meetups. [#71872](https://github.com/ClickHouse/ClickHouse/pull/71872) ([Tanya Bragin](https://github.com/tbragin)). -* Prevents listing files from s3 while inserting. [#71889](https://github.com/ClickHouse/ClickHouse/pull/71889) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Keep query_plan_merge_filters disabled by default. [#71890](https://github.com/ClickHouse/ClickHouse/pull/71890) ([Raúl Marín](https://github.com/Algunenano)). -* Remove useless code. [#71900](https://github.com/ClickHouse/ClickHouse/pull/71900) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* TreeRewriter: fix typo: `parititon` => `partition`. [#71907](https://github.com/ClickHouse/ClickHouse/pull/71907) ([yun](https://github.com/yokofly)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. [#71912](https://github.com/ClickHouse/ClickHouse/pull/71912) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix test_storage_mongodb/test.py::test_secure_connection_uri. [#71924](https://github.com/ClickHouse/ClickHouse/pull/71924) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Adapt some test to run in non-CI configurations. [#71928](https://github.com/ClickHouse/ClickHouse/pull/71928) ([Raúl Marín](https://github.com/Algunenano)). -* Fix build after [#71179](https://github.com/ClickHouse/ClickHouse/issues/71179). Clang-19 checks for unused variables inside of if-conditions. [#71929](https://github.com/ClickHouse/ClickHouse/pull/71929) ([Dmitry Novik](https://github.com/novikd)). -* Fix flaky test 03262_column_sizes_with_dynamic_structure. [#71931](https://github.com/ClickHouse/ClickHouse/pull/71931) ([Pavel Kruglov](https://github.com/Avogar)). -* Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. [#71944](https://github.com/ClickHouse/ClickHouse/pull/71944) ([Nikita Taranov](https://github.com/nickitat)). -* Enabling `query_plan_merge_filters` again after [#71890](https://github.com/ClickHouse/ClickHouse/issues/71890). [#71964](https://github.com/ClickHouse/ClickHouse/pull/71964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Add comment about bf16 to CMake docs. [#71973](https://github.com/ClickHouse/ClickHouse/pull/71973) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix 00098_primary_key_memory_allocated. [#71977](https://github.com/ClickHouse/ClickHouse/pull/71977) ([Alexander Gololobov](https://github.com/davenger)). -* Add a test for [#71908](https://github.com/ClickHouse/ClickHouse/issues/71908). [#71986](https://github.com/ClickHouse/ClickHouse/pull/71986) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Miscellaneous. [#71987](https://github.com/ClickHouse/ClickHouse/pull/71987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Minor follow-up to [#71581](https://github.com/ClickHouse/ClickHouse/issues/71581). [#71993](https://github.com/ClickHouse/ClickHouse/pull/71993) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Stress test with praktika. [#71995](https://github.com/ClickHouse/ClickHouse/pull/71995) ([Max Kainov](https://github.com/maxknv)). -* Fix prewarm of mark cache after adding a new column. [#71996](https://github.com/ClickHouse/ClickHouse/pull/71996) ([Anton Popov](https://github.com/CurtizJ)). -* Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. [#72018](https://github.com/ClickHouse/ClickHouse/pull/72018) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix comments. [#72023](https://github.com/ClickHouse/ClickHouse/pull/72023) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add test for 33604. [#72026](https://github.com/ClickHouse/ClickHouse/pull/72026) ([Nikita Taranov](https://github.com/nickitat)). -* CI: Remove unsafe secret_envs input from yml workflows. [#72028](https://github.com/ClickHouse/ClickHouse/pull/72028) ([Max Kainov](https://github.com/maxknv)). -* Update version_date.tsv and changelog after v24.10.2.80-stable. [#72029](https://github.com/ClickHouse/ClickHouse/pull/72029) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.8.7.41-lts. [#72037](https://github.com/ClickHouse/ClickHouse/pull/72037) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.9.3.128-stable. [#72041](https://github.com/ClickHouse/ClickHouse/pull/72041) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelog after v24.3.14.35-lts. [#72042](https://github.com/ClickHouse/ClickHouse/pull/72042) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Save several minutes of build time. [#72046](https://github.com/ClickHouse/ClickHouse/pull/72046) ([Raúl Marín](https://github.com/Algunenano)). -* Update README.md - Update meetups. [#72048](https://github.com/ClickHouse/ClickHouse/pull/72048) ([Tanya Bragin](https://github.com/tbragin)). -* Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix test_disk_over_web_server/. [#72075](https://github.com/ClickHouse/ClickHouse/pull/72075) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Minor improvement for system.query_metric_log stateless test. [#72076](https://github.com/ClickHouse/ClickHouse/pull/72076) ([Pablo Marcos](https://github.com/pamarcos)). -* A follow-up for [#72057](https://github.com/ClickHouse/ClickHouse/issues/72057) and https://github.com/ClickHouse/ClickHouse/pull/71505. [#72079](https://github.com/ClickHouse/ClickHouse/pull/72079) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Add google-cloud-cpp submodule. [#72092](https://github.com/ClickHouse/ClickHouse/pull/72092) ([Pablo Marcos](https://github.com/pamarcos)). -* CI: Enable fuzzer job in Nightly workflow. [#72101](https://github.com/ClickHouse/ClickHouse/pull/72101) ([Max Kainov](https://github.com/maxknv)). -* Get rid of code duplication after adding `CHECK GRANT` in https://github.com/ClickHouse/ClickHouse/pull/68885. [#72103](https://github.com/ClickHouse/ClickHouse/pull/72103) ([Vitaly Baranov](https://github.com/vitlibar)). -* Add jwt-cpp submodule. [#72104](https://github.com/ClickHouse/ClickHouse/pull/72104) ([Pablo Marcos](https://github.com/pamarcos)). -* Fix list-licenses.sh with OSX. [#72107](https://github.com/ClickHouse/ClickHouse/pull/72107) ([Raúl Marín](https://github.com/Algunenano)). -* fix cancelation for PartitionedSink. [#72126](https://github.com/ClickHouse/ClickHouse/pull/72126) ([Sema Checherinda](https://github.com/CheSema)). -* FIx 02374_analyzer_join_using. [#72145](https://github.com/ClickHouse/ClickHouse/pull/72145) ([Vladimir Cherkasov](https://github.com/vdimir)). -* Fixed a test which was flaky-flaky. [#72147](https://github.com/ClickHouse/ClickHouse/pull/72147) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - ### ClickHouse release 24.10, 2024-10-31 #### Backward Incompatible Change From c4329bf2421a054161f998d99ba4498d3692546f Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Nov 2024 12:25:35 +0000 Subject: [PATCH 180/433] Fix #72174 --- src/Interpreters/Squashing.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 02d1ae528ac..04cd1698067 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -145,7 +145,20 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl auto columns = input_chunks[i].detachColumns(); for (size_t j = 0; j != num_columns; ++j) { - have_same_serialization[j] &= ISerialization::getKind(*columns[j]) == ISerialization::getKind(*mutable_columns[j]); + /// IColumn::structureEquals is not implemented for deprecated object type, ignore it and always convert to non-sparse. + bool has_object_deprecated = false; + columns[j]->forEachSubcolumnRecursively([&has_object_deprecated](const auto & subcolumn) + { + has_object_deprecated = has_object_deprecated || subcolumn.getDataType() == TypeIndex::ObjectDeprecated; + }); + mutable_columns[j]->forEachSubcolumnRecursively([&has_object_deprecated](const auto & subcolumn) + { + has_object_deprecated = has_object_deprecated || subcolumn.getDataType() == TypeIndex::ObjectDeprecated; + }); + + /// Need to check if there are any sparse columns in subcolumns, + /// since `IColumn::isSparse` is not recursive but sparse column can be inside a tuple, for example. + have_same_serialization[j] &= !has_object_deprecated && columns[j]->structureEquals(*mutable_columns[j]); source_columns_list[j].emplace_back(std::move(columns[j])); } } From 91d615812e809953142ad2854c768181e4a7b51c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 22 Nov 2024 12:11:14 +0000 Subject: [PATCH 181/433] fix ALTER DELETE with _block_number column --- src/Interpreters/MutationsInterpreter.cpp | 40 ++++++++++-------- .../MergeTree/StorageFromMergeTreeDataPart.h | 7 +++- .../03275_block_number_mutation.reference | 20 +++++++++ .../03275_block_number_mutation.sql | 41 +++++++++++++++++++ 4 files changed, 89 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/03275_block_number_mutation.reference create mode 100644 tests/queries/0_stateless/03275_block_number_mutation.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index a35353a6b2a..db61bd6f6ab 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -402,6 +402,8 @@ MutationsInterpreter::MutationsInterpreter( "Cannot execute mutation for {}. Mutation should be applied to every part separately.", source.getStorage()->getName()); } + + prepare(!settings.can_execute); } MutationsInterpreter::MutationsInterpreter( @@ -414,10 +416,21 @@ MutationsInterpreter::MutationsInterpreter( ContextPtr context_, Settings settings_) : MutationsInterpreter( - Source(storage_, std::move(source_part_), std::move(alter_conversions_)), + Source(storage_, source_part_, std::move(alter_conversions_)), std::move(metadata_snapshot_), std::move(commands_), std::move(available_columns_), std::move(context_), std::move(settings_)) { + const auto & part_columns = source_part_->getColumnsDescription(); + auto persistent_virtuals = storage_.getVirtualsPtr()->getNamesAndTypesList(VirtualsKind::Persistent); + NameSet available_columns_set(available_columns.begin(), available_columns.end()); + + for (const auto & column : persistent_virtuals) + { + if (part_columns.has(column.name) && !available_columns_set.contains(column.name)) + available_columns.push_back(column.name); + } + + prepare(!settings.can_execute); } MutationsInterpreter::MutationsInterpreter( @@ -442,8 +455,6 @@ MutationsInterpreter::MutationsInterpreter( LOG_TEST(logger, "Will use old analyzer to prepare mutation"); } context = std::move(new_context); - - prepare(!settings.can_execute); } static NameSet getKeyColumns(const MutationsInterpreter::Source & source, const StorageMetadataPtr & metadata_snapshot) @@ -579,13 +590,6 @@ void MutationsInterpreter::prepare(bool dry_run) auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns); NameSet available_columns_set(available_columns.begin(), available_columns.end()); - /// Add _row_exists column if it is physically present in the part - if (source.hasLightweightDeleteMask()) - { - all_columns.emplace_back(RowExistsColumn::name, RowExistsColumn::type); - available_columns_set.insert(RowExistsColumn::name); - } - NameSet updated_columns; bool materialize_ttl_recalculate_only = source.materializeTTLRecalculateOnly(); @@ -599,9 +603,15 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & [name, _] : command.column_to_update_expression) { - if (!available_columns_set.contains(name) && name != RowExistsColumn::name) - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, - "Column {} is updated but not requested to read", name); + if (name == RowExistsColumn::name) + { + if (available_columns_set.emplace(name).second) + available_columns.push_back(name); + } + else if (!available_columns_set.contains(name)) + { + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Column {} is updated but not requested to read", name); + } updated_columns.insert(name); } @@ -1070,10 +1080,6 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s auto all_columns = storage_snapshot->getColumnsByNames(options, available_columns); - /// Add _row_exists column if it is present in the part - if (source.hasLightweightDeleteMask() || deleted_mask_updated) - all_columns.emplace_back(RowExistsColumn::name, RowExistsColumn::type); - bool has_filters = false; /// Next, for each stage calculate columns changed by this and previous stages. for (size_t i = 0; i < prepared_stages.size(); ++i) diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 6825752acd7..ceaf1de0388 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -18,7 +18,7 @@ class StorageFromMergeTreeDataPart final : public IStorage { public: /// Used in part mutation. - explicit StorageFromMergeTreeDataPart( + StorageFromMergeTreeDataPart( const MergeTreeData::DataPartPtr & part_, const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot_) : IStorage(getIDFromPart(part_)) @@ -32,10 +32,13 @@ public: } /// Used in queries with projection. - StorageFromMergeTreeDataPart(const MergeTreeData & storage_, ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr_) + StorageFromMergeTreeDataPart( + const MergeTreeData & storage_, + ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr_) : IStorage(storage_.getStorageID()), storage(storage_), analysis_result_ptr(analysis_result_ptr_) { setInMemoryMetadata(storage.getInMemoryMetadata()); + setVirtuals(*storage.getVirtualsPtr()); } String getName() const override { return "FromMergeTreeDataPart"; } diff --git a/tests/queries/0_stateless/03275_block_number_mutation.reference b/tests/queries/0_stateless/03275_block_number_mutation.reference new file mode 100644 index 00000000000..cbd49a8c2cb --- /dev/null +++ b/tests/queries/0_stateless/03275_block_number_mutation.reference @@ -0,0 +1,20 @@ +8 44 +DELETE WHERE x < 2 1 +_block_number 1 +ts 1 +x 1 +8 44 +DELETE WHERE x < 2 1 +_block_number 1 +ts 1 +x 1 +8 44 +DELETE WHERE x < 2 1 +_block_number 1 +ts 1 +x 1 +8 44 +DELETE WHERE x < 2 1 +_block_number 1 +ts 1 +x 1 diff --git a/tests/queries/0_stateless/03275_block_number_mutation.sql b/tests/queries/0_stateless/03275_block_number_mutation.sql new file mode 100644 index 00000000000..698f177dff7 --- /dev/null +++ b/tests/queries/0_stateless/03275_block_number_mutation.sql @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS t_block_number_delete sync; + +SET mutations_sync = 2; + +CREATE TABLE t_block_number_delete (x UInt32, ts DateTime) ENGINE = MergeTree ORDER BY x SETTINGS enable_block_number_column = 1, min_bytes_for_wide_part = 1; + +INSERT INTO t_block_number_delete SELECT number, now() - INTERVAL number minute from numbers(10); +OPTIMIZE TABLE t_block_number_delete final; +ALTER TABLE t_block_number_delete DELETE WHERE x < 2; + +SELECT count(), sum(x) FROM t_block_number_delete; +SELECT command, is_done, latest_fail_reason FROM system.mutations WHERE database = currentDatabase() AND table = 't_block_number_delete'; +SELECT column, count() FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_block_number_delete' AND active GROUP BY column ORDER BY column; + +DETACH TABLE t_block_number_delete; +ATTACH TABLE t_block_number_delete; + +SELECT count(), sum(x) FROM t_block_number_delete; +SELECT command, is_done, latest_fail_reason FROM system.mutations WHERE database = currentDatabase() AND table = 't_block_number_delete'; +SELECT column, count() FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_block_number_delete' AND active GROUP BY column ORDER BY column; + +DROP TABLE t_block_number_delete; + +CREATE TABLE t_block_number_delete (x UInt32, ts DateTime) ENGINE = MergeTree ORDER BY x SETTINGS enable_block_number_column = 1, min_bytes_for_wide_part = '10G'; + +INSERT INTO t_block_number_delete SELECT number, now() - INTERVAL number minute from numbers(10); +OPTIMIZE TABLE t_block_number_delete final; +ALTER TABLE t_block_number_delete DELETE WHERE x < 2; + +SELECT count(), sum(x) FROM t_block_number_delete; +SELECT command, is_done, latest_fail_reason FROM system.mutations WHERE database = currentDatabase() AND table = 't_block_number_delete'; +SELECT column, count() FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_block_number_delete' AND active GROUP BY column ORDER BY column; + +DETACH TABLE t_block_number_delete; +ATTACH TABLE t_block_number_delete; + +SELECT count(), sum(x) FROM t_block_number_delete; +SELECT command, is_done, latest_fail_reason FROM system.mutations WHERE database = currentDatabase() AND table = 't_block_number_delete'; +SELECT column, count() FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_block_number_delete' AND active GROUP BY column ORDER BY column; + +DROP TABLE t_block_number_delete; From 07ecc200c32423ecdb062b12dbc187b5863c3f90 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 22 Nov 2024 13:08:21 +0000 Subject: [PATCH 182/433] Update version_date.tsv and changelogs after v24.8.8.17-lts --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.8.8.17-lts.md | 24 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 29 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.8.8.17-lts.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 0d75c0bd225..f3f25c1a247 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -38,7 +38,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.10.2.80" +ARG VERSION="24.10.3.21" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 5b3d86ca3e6..3bf23767150 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -35,7 +35,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.10.2.80" +ARG VERSION="24.10.3.21" 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 2e56f676cbc..31fdcb8a490 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -28,7 +28,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.10.2.80" +ARG VERSION="24.10.3.21" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" #docker-official-library:off diff --git a/docs/changelogs/v24.8.8.17-lts.md b/docs/changelogs/v24.8.8.17-lts.md new file mode 100644 index 00000000000..13ac1a5571f --- /dev/null +++ b/docs/changelogs/v24.8.8.17-lts.md @@ -0,0 +1,24 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.8.8.17-lts (81036bd118b) FIXME as compared to v24.8.7.41-lts (e28553d4f2b) + +#### Improvement +* Backported in [#72060](https://github.com/ClickHouse/ClickHouse/issues/72060): When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have `SELECT` permission or `dictGet` permission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. [#72051](https://github.com/ClickHouse/ClickHouse/pull/72051) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Backported in [#71981](https://github.com/ClickHouse/ClickHouse/issues/71981): After [this issue](https://github.com/ClickHouse/ClickHouse/pull/59946#issuecomment-1943653197) there are quite a few table replicas in production such that their `metadata_version` node value is both equal to `0` and is different from the respective table's `metadata` node version. This leads to `alter` queries failing on such replicas. [#69274](https://github.com/ClickHouse/ClickHouse/pull/69274) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). +* Backported in [#72142](https://github.com/ClickHouse/ClickHouse/issues/72142): Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#72038](https://github.com/ClickHouse/ClickHouse/issues/72038): Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#72032](https://github.com/ClickHouse/ClickHouse/issues/72032): Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#72155](https://github.com/ClickHouse/ClickHouse/issues/72155): Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). +* Backported in [#72114](https://github.com/ClickHouse/ClickHouse/issues/72114): Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#72067](https://github.com/ClickHouse/ClickHouse/issues/72067): Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. [#72049](https://github.com/ClickHouse/ClickHouse/pull/72049) ([Nikolay Degterinsky](https://github.com/evillique)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f69c72163c2..7591f7050cb 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,8 +1,10 @@ +v24.10.3.21-stable 2024-11-22 v24.10.2.80-stable 2024-11-18 v24.10.1.2812-stable 2024-11-01 v24.9.3.128-stable 2024-11-18 v24.9.2.42-stable 2024-10-03 v24.9.1.3278-stable 2024-09-26 +v24.8.8.17-lts 2024-11-22 v24.8.7.41-lts 2024-11-18 v24.8.6.70-lts 2024-11-04 v24.8.5.115-lts 2024-10-08 From f0b706ffdef2e2596108e19d0459f31a1528f504 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Nov 2024 14:10:40 +0100 Subject: [PATCH 183/433] Fix the test (take the test_ prefix into account). --- .../03167_improvement_table_name_too_long.sh | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index afcc3dee4b6..dd8d938d265 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -4,14 +4,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") -# let excess_length=allowed_name_length+1 +max_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") +let allowed_name_length=max_name_length-5 # We substract 5 from the name length because when we create files in the test environment, they have the 'test_' prefix +let excess_length=allowed_name_length+1 -getconf NAME_MAX /fasttest-workspace/db-fasttest/store -# long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) -# allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) +long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) +allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -# $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" -# $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" -# $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" -# ToHqNPfhZpLevFeVCquBvDPDdVSFxSTRaEkCIPQzhXCVAwbmHXNRJiEFqQGgNzLULAufPDpRIjXoTxYWcdnNStNLdWDIjNiHoKltMpdGbDzeVsXFPIynefYqJPLqCuuuckRpBjOFPGoAKndFzAOGwCcZaIsGFFkOOpPwTesCZfbjvtZWrGYeYqAWJsLPQPIRzFHfebEYLOguMMNjQ +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" +$CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" From 4c627aadd3654eb277bc5c18e860597a303b1bbf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Nov 2024 14:26:10 +0100 Subject: [PATCH 184/433] Update 03167_improvement_table_name_too_long.sh --- .../0_stateless/03167_improvement_table_name_too_long.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index dd8d938d265..1e2f784c4d4 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -4,9 +4,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -max_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") -let allowed_name_length=max_name_length-5 # We substract 5 from the name length because when we create files in the test environment, they have the 'test_' prefix -let excess_length=allowed_name_length+1 +allowed_name_length=$(( $($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") - 5 )) # Reserve 5 characters for 'test_' prefix +excess_length=$((allowed_name_length + 1)) # Ensure exceeding the limit long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) From a87dc7a3e24c599254503008110c6e3f11cb8ac1 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 22 Nov 2024 13:40:07 +0000 Subject: [PATCH 185/433] Add processing of incorrect icebrg 1 format --- .../DataLakes/IcebergMetadata.cpp | 154 ++++++++++++------ 1 file changed, 108 insertions(+), 46 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index f0a80a41d4e..bed3e78a0de 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -139,6 +139,7 @@ enum class DataFileContent : uint8_t * } */ + DataTypePtr getSimpleTypeByName(const String & type_name) { if (type_name == "boolean") @@ -248,7 +249,83 @@ DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & t } -std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version, bool ignore_schema_evolution) +std::pair +parseTableSchemaV2Method(const Poco::JSON::Object::Ptr & metadata_object, bool ignore_schema_evolution) +{ + Poco::JSON::Object::Ptr schema; + if (!metadata_object->has("current-schema-id")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'current-schema-id' field is missing in metadata"); + auto current_schema_id = metadata_object->getValue("current-schema-id"); + if (!metadata_object->has("schemas")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schemas' field is missing in metadata"); + auto schemas = metadata_object->get("schemas").extract(); + if (schemas->size() == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: schemas field is empty"); + + if (ignore_schema_evolution) + { + /// If we ignore schema evolution, we will just use latest schema for all data files. + /// Find schema with 'schema-id' equal to 'current_schema_id'. + for (uint32_t i = 0; i != schemas->size(); ++i) + { + auto current_schema = schemas->getObject(i); + if (!current_schema->has("schema-id")) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema-id' field is missing in schema"); + } + if (current_schema->getValue("schema-id") == current_schema_id) + { + schema = current_schema; + break; + } + } + + if (!schema) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, R"(There is no schema with "schema-id" that matches "current-schema-id" in metadata)"); + } + else + { + if (schemas->size() == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: schemas field is empty"); + if (schemas->size() > 1) + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is " + "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable " + "setting " + "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); + + /// Now we sure that there is only one schema. + schema = schemas->getObject(0); + if (schema->getValue("schema-id") != current_schema_id) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); + } + return {schema, current_schema_id}; +} + +std::pair +parseTableSchemaV1Method(const Poco::JSON::Object::Ptr & metadata_object, bool ignore_schema_evolution) +{ + if (!metadata_object->has("schema")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema' field is missing in metadata"); + Poco::JSON::Object::Ptr schema = metadata_object->getObject("schema"); + if (!metadata_object->has("schema")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema-id' field is missing in schema"); + auto current_schema_id = schema->getValue("schema-id"); + if (!ignore_schema_evolution && metadata_object->has("schemas") + && metadata_object->get("schemas").extract()->size() > 1) + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " + "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable " + "setting " + "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); + return {schema, current_schema_id}; +} + +std::pair parseTableSchema( + const Poco::JSON::Object::Ptr & metadata_object, LoggerPtr metadata_logger, int format_version, bool ignore_schema_evolution) { Poco::JSON::Object::Ptr schema; Int32 current_schema_id; @@ -259,55 +336,40 @@ std::pair parseTableSchema(const Poco::JSON::Object::P /// (we will compare schema id from manifest file and currently used schema). if (format_version == 2) { - current_schema_id = metadata_object->getValue("current-schema-id"); - auto schemas = metadata_object->get("schemas").extract(); - if (schemas->size() == 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: schemas field is empty"); - - if (ignore_schema_evolution) - { - /// If we ignore schema evolution, we will just use latest schema for all data files. - /// Find schema with 'schema-id' equal to 'current_schema_id'. - for (uint32_t i = 0; i != schemas->size(); ++i) - { - auto current_schema = schemas->getObject(i); - if (current_schema->getValue("schema-id") == current_schema_id) - { - schema = current_schema; - break; - } - } - - if (!schema) - throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(There is no schema with "schema-id" that matches "current-schema-id" in metadata)"); - } - else - { - if (schemas->size() != 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is " - "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable setting " - "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); - - /// Now we sure that there is only one schema. - schema = schemas->getObject(0); - if (schema->getValue("schema-id") != current_schema_id) - throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); - } + std::tie(schema, current_schema_id) = parseTableSchemaV2Method(metadata_object, ignore_schema_evolution); } else { - schema = metadata_object->getObject("schema"); - current_schema_id = schema->getValue("schema-id"); - /// Field "schemas" is optional for version 1, but after version 2 was introduced, - /// in most cases this field is added for new tables in version 1 as well. - if (!ignore_schema_evolution && metadata_object->has("schemas") && metadata_object->get("schemas").extract()->size() > 1) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " - "supported. If you want to ignore schema evolution and read all files using latest schema saved on table creation, enable setting " - "iceberg_engine_ignore_schema_evolution (Note: enabling this setting can lead to incorrect result)"); + try + { + std::tie(schema, current_schema_id) = parseTableSchemaV1Method(metadata_object, ignore_schema_evolution); + } + catch (const Exception & first_error) + { + try + { + std::tie(schema, current_schema_id) = parseTableSchemaV2Method(metadata_object, ignore_schema_evolution); + LOG_WARNING( + metadata_logger, + "Iceberg table schema was parsed using v2 specification, but it was impossible to parse it using v1 specification. Be " + "aware that you Iceberg writing Engine violates Iceberg specification. Error during parsing {}", + first_error.displayText()); + } + catch (const Exception & second_error) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot parse Iceberg table schema both with v1 and v2 methods. Old method error: {}. New method error: {}", + first_error.displayText(), + second_error.displayText()); + } + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, e.displayText()); + } + } } + NamesAndTypesList names_and_types; auto fields = schema->get("fields").extract(); for (size_t i = 0; i != fields->size(); ++i) @@ -404,7 +466,7 @@ IcebergMetadata::create(ObjectStoragePtr object_storage, ConfigurationObserverPt auto format_version = object->getValue("format-version"); auto [schema, schema_id] - = parseTableSchema(object, format_version, local_context->getSettingsRef()[Setting::iceberg_engine_ignore_schema_evolution]); + = parseTableSchema(object, log, format_version, local_context->getSettingsRef()[Setting::iceberg_engine_ignore_schema_evolution]); auto current_snapshot_id = object->getValue("current-snapshot-id"); auto snapshots = object->get("snapshots").extract(); From c5fffa7c934e882abad19c021c73be9ab1ce88a0 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 20 Nov 2024 17:32:20 +0100 Subject: [PATCH 186/433] docker build to use docker file from release tag --- .github/workflows/create_release.yml | 169 +++++++++++++++++++++++---- 1 file changed, 146 insertions(+), 23 deletions(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 9d2ee70365e..b53bf172096 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -25,6 +25,11 @@ env: required: false default: false type: boolean + only-docker: + description: 'Run only docker builds (repo-recovery, tests)' + required: false + default: false + type: boolean dry-run: description: 'Dry run' required: false @@ -45,6 +50,11 @@ env: required: false default: false type: boolean + only-docker: + description: 'Run only docker builds (repo-recovery, tests)' + required: false + default: false + type: boolean dry-run: description: 'Dry run' required: false @@ -69,13 +79,13 @@ jobs: - name: Prepare Release Info shell: bash run: | - if [ ${{ inputs.only-repo }} == "true" ]; then - git tag -l ${{ inputs.ref }} || { echo "With only-repo option ref must be a valid release tag"; exit 1; } + if [ ${{ inputs.only-repo }} == "true" ] || [ ${{ inputs.only-docker }} == "true" ]; then + git tag -l ${{ inputs.ref }} || { echo "With only-repo/docker option ref must be a valid release tag"; exit 1; } fi python3 ./tests/ci/create_release.py --prepare-release-info \ --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ ${{ inputs.dry-run == true && '--dry-run' || '' }} \ - ${{ inputs.only-repo == true && '--skip-tag-check' || '' }} + ${{ (inputs.only-repo == true || inputs.only-docker == true) && '--skip-tag-check' || '' }} echo "::group::Release Info" python3 -m json.tool /tmp/release_info.json echo "::endgroup::" @@ -87,31 +97,33 @@ jobs: echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" if [ "$is_latest" == "true" ]; then echo "DOCKER_TAG_TYPE=release-latest" >> "$GITHUB_ENV" + echo "IS_LATEST=1" >> "$GITHUB_ENV" else echo "DOCKER_TAG_TYPE=release" >> "$GITHUB_ENV" + echo "IS_LATEST=0" >> "$GITHUB_ENV" fi - name: Download All Release Artifacts - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push Git Tag for the Release - if: ${{ ! inputs.only-repo }} + if: ${{ ! inputs.only-repo && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Push New Release Branch - if: ${{ inputs.type == 'new' && ! inputs.only-repo }} + if: ${{ inputs.type == 'new' && ! inputs.only-repo && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump CH Version and Update Contributors' List - if: ${{ ! inputs.only-repo }} + if: ${{ ! inputs.only-repo && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/create_release.py --set-progress-started --progress "update changelog, docker version, security" @@ -135,7 +147,7 @@ jobs: python3 ./utils/security-generator/generate_security.py > SECURITY.md git diff HEAD - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run && ! inputs.only-repo }} + if: ${{ inputs.type == 'patch' && ! inputs.dry-run && ! inputs.only-repo && ! inputs.only-docker }} uses: peter-evans/create-pull-request@v6 with: author: "robot-clickhouse " @@ -153,65 +165,176 @@ jobs: ### Changelog category (leave one): - Not for changelog (changelog entry is not required) - name: Complete previous steps and Restore git state - if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo && ! inputs.only-docker }} shell: bash run: | git reset --hard HEAD git checkout "$GITHUB_REF_NAME" python3 ./tests/ci/create_release.py --set-progress-completed - name: Create GH Release - if: ${{ inputs.type == 'patch' && ! inputs.only-repo }} + if: ${{ inputs.type == 'patch' && ! inputs.only-repo && ! inputs.only-docker}} shell: bash run: | python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export TGZ Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test TGZ Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export RPM Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test RPM Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Export Debian Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Test Debian Packages - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && ! inputs.only-docker }} shell: bash run: | python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run == true && '--dry-run' || '' }} - name: Docker clickhouse/clickhouse-server building - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && inputs.dry-run != true }} shell: bash run: | cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker server release" - export CHECK_NAME="Docker server image" - python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + export DOCKER_IMAGE="clickhouse/clickhouse-server" + + # We must use docker file from the release commit + git checkout "${{ env.RELEASE_TAG }}" + python3 ./version_helper.py --export > /tmp/version.sh + . /tmp/version.sh + + if [[ $CLICKHOUSE_VERSION_STRING =~ ^[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+$ ]]; then + echo "ClickHouse version: $CLICKHOUSE_VERSION_STRING" + else + echo "Invalid version string: $CLICKHOUSE_VERSION_STRING" + exit 1 + fi + CLICKHOUSE_VERSION_MINOR=${CLICKHOUSE_VERSION_STRING%.*} + CLICKHOUSE_VERSION_MAJOR=${CLICKHOUSE_VERSION_MINOR%.*} + + # Define build configurations + configs=( + "ubuntu:../../docker/server/Dockerfile.ubuntu" + "alpine:../../docker/server/Dockerfile.alpine" + ) + + for config in "${configs[@]}"; do + # Split the config into variant and Dockerfile path + variant=${config%%:*} + dockerfile=${config##*:} + + VERSION_SUFFIX=$([ "$variant" = "ubuntu" ] && echo "" || echo "-$variant") + LABEL_VERSION="${CLICKHOUSE_VERSION_STRING}${VERSION_SUFFIX}" + TAGS=( + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_STRING}${VERSION_SUFFIX}" + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_MINOR}${VERSION_SUFFIX}" + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_MAJOR}${VERSION_SUFFIX}" + ) + + if [ "$IS_LATEST" = "1" ]; then + TAGS+=("--tag=${DOCKER_IMAGE}:latest${VERSION_SUFFIX}") + fi + + echo "Following tags will be created: ${TAGS[*]}" + + # shellcheck disable=SC2086,SC2048 + docker buildx build \ + --platform=linux/amd64,linux/arm64 \ + --output=type=registry \ + --label=com.clickhouse.build.version="$LABEL_VERSION" \ + ${TAGS[*]} \ + --build-arg=VERSION="$CLICKHOUSE_VERSION_STRING" \ + --progress=plain \ + --file="$dockerfile" \ + ../../docker/server + done + + git checkout - python3 ./create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building - if: ${{ inputs.type == 'patch' }} + if: ${{ inputs.type == 'patch' && inputs.dry-run != true }} shell: bash run: | cd "./tests/ci" python3 ./create_release.py --set-progress-started --progress "docker keeper release" - export CHECK_NAME="Docker keeper image" - python3 docker_server.py --tag-type ${{ env.DOCKER_TAG_TYPE }} --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + + export DOCKER_IMAGE="clickhouse/clickhouse-keeper" + + # We must use docker file from the release commit + git checkout "${{ env.RELEASE_TAG }}" + python3 ./version_helper.py --export > /tmp/version.sh + . /tmp/version.sh + + if [[ $CLICKHOUSE_VERSION_STRING =~ ^[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+$ ]]; then + echo "ClickHouse version: $CLICKHOUSE_VERSION_STRING" + else + echo "Invalid version string: $CLICKHOUSE_VERSION_STRING" + exit 1 + fi + CLICKHOUSE_VERSION_MINOR=${CLICKHOUSE_VERSION_STRING%.*} + CLICKHOUSE_VERSION_MAJOR=${CLICKHOUSE_VERSION_MINOR%.*} + + # Define build configurations + configs=( + "ubuntu:../../docker/keeper/Dockerfile.ubuntu" + "alpine:../../docker/keeper/Dockerfile.alpine" + ) + + for config in "${configs[@]}"; do + # Split the config into variant and Dockerfile path + variant=${config%%:*} + dockerfile=${config##*:} + + VERSION_SUFFIX=$([ "$variant" = "ubuntu" ] && echo "" || echo "-$variant") + LABEL_VERSION="${CLICKHOUSE_VERSION_STRING}${VERSION_SUFFIX}" + TAGS=( + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_STRING}${VERSION_SUFFIX}" + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_MINOR}${VERSION_SUFFIX}" + "--tag=${DOCKER_IMAGE}:${CLICKHOUSE_VERSION_MAJOR}${VERSION_SUFFIX}" + ) + + if [ "$IS_LATEST" = "1" ]; then + TAGS+=("--tag=${DOCKER_IMAGE}:latest${VERSION_SUFFIX}") + fi + + echo "Following tags will be created: ${TAGS[*]}" + + # shellcheck disable=SC2086,SC2048 + docker buildx build \ + --platform=linux/amd64,linux/arm64 \ + --output=type=registry \ + --label=com.clickhoghuse.build.version="$LABEL_VERSION" \ + ${TAGS[*]} \ + --build-arg=VERSION="$CLICKHOUSE_VERSION_STRING" \ + --progress=plain \ + --file="$dockerfile" \ + ../../docker/keeper + done + + git checkout - python3 ./create_release.py --set-progress-completed + # check out back if previous steps failed + - name: Checkout back + if: ${{ ! cancelled() }} + shell: bash + run: | + git checkout ${{ github.ref }} - name: Update release info. Merge created PRs shell: bash run: | From c3a32cc215f837decfeca933cd5c760879b2bb98 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 15:01:14 +0100 Subject: [PATCH 187/433] impl --- src/Columns/ColumnDynamic.h | 2 ++ ...4_dynamic_column_data_race_with_concurrent_hj.reference | 0 .../03274_dynamic_column_data_race_with_concurrent_hj.sql | 7 +++++++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.reference create mode 100644 tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.sql diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index fbab4d5da4c..bdbad99519f 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -319,6 +319,8 @@ public: variant_column_ptr = assert_cast(variant_column.get()); } + void forEachSubcolumn(ColumnCallback callback) const override { callback(variant_column); } + void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override { callback(*variant_column); diff --git a/tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.reference b/tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.sql b/tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.sql new file mode 100644 index 00000000000..c6f2e3b98b0 --- /dev/null +++ b/tests/queries/0_stateless/03274_dynamic_column_data_race_with_concurrent_hj.sql @@ -0,0 +1,7 @@ +SET join_algorithm = 'parallel_hash'; +SET allow_experimental_dynamic_type = 1; +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Tuple(c1 Int,c2 Dynamic)) ENGINE = Memory(); +SELECT 1 FROM t0 tx JOIN t0 ty ON tx.c0 = ty.c0; +DROP TABLE t0; + From 6c762a041864687efcdac8c93d1556c2b7c7af6a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Nov 2024 15:14:22 +0100 Subject: [PATCH 188/433] Update 03167_improvement_table_name_too_long.sh --- .../0_stateless/03167_improvement_table_name_too_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 1e2f784c4d4..4ba0b77b752 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -10,6 +10,6 @@ excess_length=$((allowed_name_length + 1)) # Ensure exceeding the limit long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 "ARGUMENT_OUT_OF_BOUND" +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o 'ARGUMENT_OUT_OF_BOUND' $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" From 93dc65ad8866e476a1216ca383936252e88884c7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 22 Nov 2024 10:24:05 +0000 Subject: [PATCH 189/433] Can we enable SimSIMD on ARM with Clang 19? --- cmake/tools.cmake | 2 +- contrib/SimSIMD | 2 +- contrib/SimSIMD-cmake/CMakeLists.txt | 4 +-- contrib/usearch | 2 +- contrib/usearch-cmake/CMakeLists.txt | 54 ++-------------------------- 5 files changed, 7 insertions(+), 57 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index d95f1fbb29b..dfd7ada0948 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -13,7 +13,7 @@ execute_process(COMMAND ${CMAKE_CXX_COMPILER} --version message (STATUS "Using compiler:\n${COMPILER_SELF_IDENTIFICATION}") # Require minimum compiler versions -set (CLANG_MINIMUM_VERSION 17) +set (CLANG_MINIMUM_VERSION 18) set (XCODE_MINIMUM_VERSION 12.0) set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index fa60f1b8e35..da2d3853729 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit fa60f1b8e3582c50978f0ae86c2ebb6c9af957f3 +Subproject commit da2d38537299ade247c2499131d936fb8db38f03 diff --git a/contrib/SimSIMD-cmake/CMakeLists.txt b/contrib/SimSIMD-cmake/CMakeLists.txt index 8350417479a..f819bd2d13a 100644 --- a/contrib/SimSIMD-cmake/CMakeLists.txt +++ b/contrib/SimSIMD-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ -# See contrib/usearch-cmake/CMakeLists.txt, why only enabled on x86 -if (ARCH_AMD64) +# See contrib/usearch-cmake/CMakeLists.txt, why only enabled on x86 and ARM +if (ARCH_AMD64 OR (ARCH_AARCH64 AND NOT NO_ARMV81_OR_HIGHER AND CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 19)) set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD") set(SIMSIMD_SRCS ${SIMSIMD_PROJECT_DIR}/c/lib.c) add_library(_simsimd ${SIMSIMD_SRCS}) diff --git a/contrib/usearch b/contrib/usearch index 7efe8b710c9..9561fcae124 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 7efe8b710c9831bfe06573b1df0fad001b04a2b5 +Subproject commit 9561fcae1249ea8effbf71250e8a7a7ea97e5dfe diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index fda061bf467..bb0f6983a23 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -6,9 +6,8 @@ target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/incl target_link_libraries(_usearch INTERFACE _fp16) target_compile_definitions(_usearch INTERFACE USEARCH_USE_FP16LIB) -# Only x86 for now. On ARM, the linker goes down in flames. To make SimSIMD compile, I had to remove a macro checks in SimSIMD -# for AVX512 (x86, worked nicely) and __ARM_BF16_FORMAT_ALTERNATIVE. It is probably because of that. -if (ARCH_AMD64) +# SimSIMD supports x86 and ARM platforms. The latter requires Clang 19 because older versions had a buggy bf16 implementation. +if (ARCH_AMD64 OR (ARCH_AARCH64 AND NOT NO_ARMV81_OR_HIGHER AND CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 19)) target_link_libraries(_usearch INTERFACE _simsimd) target_compile_definitions(_usearch INTERFACE USEARCH_USE_SIMSIMD) @@ -17,52 +16,3 @@ if (ARCH_AMD64) endif () add_library(ch_contrib::usearch ALIAS _usearch) - - -# Cf. https://github.com/llvm/llvm-project/issues/107810 (though it is not 100% the same stack) -# -# LLVM ERROR: Cannot select: 0x7996e7a73150: f32,ch = load<(load (s16) from %ir.22, !tbaa !54231), anyext from bf16> 0x79961cb737c0, 0x7996e7a1a500, undef:i64, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 -# 0x7996e7a1a500: i64 = add 0x79961e770d00, Constant:i64<-16>, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 -# 0x79961e770d00: i64,ch = CopyFromReg 0x79961cb737c0, Register:i64 %4, ./contrib/SimSIMD/include/simsimd/dot.h:215:1 -# 0x7996e7a1ae10: i64 = Register %4 -# 0x7996e7a1b5f0: i64 = Constant<-16> -# 0x7996e7a1a730: i64 = undef -# In function: _ZL23simsimd_dot_bf16_serialPKu6__bf16S0_yPd -# PLEASE submit a bug report to https://github.com/llvm/llvm-project/issues/ and include the crash backtrace. -# Stack dump: -# 0. Running pass 'Function Pass Manager' on module 'src/libdbms.a(MergeTreeIndexVectorSimilarity.cpp.o at 2312737440)'. -# 1. Running pass 'AArch64 Instruction Selection' on function '@_ZL23simsimd_dot_bf16_serialPKu6__bf16S0_yPd' -# #0 0x00007999e83a63bf llvm::sys::PrintStackTrace(llvm::raw_ostream&, int) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda63bf) -# #1 0x00007999e83a44f9 llvm::sys::RunSignalHandlers() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda44f9) -# #2 0x00007999e83a6b00 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xda6b00) -# #3 0x00007999e6e45320 (/lib/x86_64-linux-gnu/libc.so.6+0x45320) -# #4 0x00007999e6e9eb1c pthread_kill (/lib/x86_64-linux-gnu/libc.so.6+0x9eb1c) -# #5 0x00007999e6e4526e raise (/lib/x86_64-linux-gnu/libc.so.6+0x4526e) -# #6 0x00007999e6e288ff abort (/lib/x86_64-linux-gnu/libc.so.6+0x288ff) -# #7 0x00007999e82fe0c2 llvm::report_fatal_error(llvm::Twine const&, bool) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xcfe0c2) -# #8 0x00007999e8c2f8e3 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162f8e3) -# #9 0x00007999e8c2ed76 llvm::SelectionDAGISel::SelectCodeCommon(llvm::SDNode*, unsigned char const*, unsigned int) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162ed76) -# #10 0x00007999ea1adbcb (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x2badbcb) -# #11 0x00007999e8c2611f llvm::SelectionDAGISel::DoInstructionSelection() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x162611f) -# #12 0x00007999e8c25790 llvm::SelectionDAGISel::CodeGenAndEmitDAG() (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x1625790) -# #13 0x00007999e8c248de llvm::SelectionDAGISel::SelectAllBasicBlocks(llvm::Function const&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x16248de) -# #14 0x00007999e8c22934 llvm::SelectionDAGISel::runOnMachineFunction(llvm::MachineFunction&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x1622934) -# #15 0x00007999e87826b9 llvm::MachineFunctionPass::runOnFunction(llvm::Function&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x11826b9) -# #16 0x00007999e84f7772 llvm::FPPassManager::runOnFunction(llvm::Function&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xef7772) -# #17 0x00007999e84fd2f4 llvm::FPPassManager::runOnModule(llvm::Module&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xefd2f4) -# #18 0x00007999e84f7e9f llvm::legacy::PassManagerImpl::run(llvm::Module&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xef7e9f) -# #19 0x00007999e99f7d61 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f7d61) -# #20 0x00007999e99f8c91 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f8c91) -# #21 0x00007999e99f8b10 llvm::lto::thinBackend(llvm::lto::Config const&, unsigned int, std::function>> (unsigned int, llvm::Twine const&)>, llvm::Module&, llvm::ModuleSummaryIndex const&, llvm::DenseMap, std::equal_to, std::allocator>, llvm::DenseMapInfo, llvm::detail::DenseMapPair, std::equal_to, std::allocator>>> const&, llvm::DenseMap, llvm::detail::DenseMapPair> const&, llvm::MapVector, llvm::detail::DenseMapPair>, llvm::SmallVector, 0u>>*, std::vector> const&) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f8b10) -# #22 0x00007999e99f248d (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f248d) -# #23 0x00007999e99f1cd6 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0x23f1cd6) -# #24 0x00007999e82c9beb (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xcc9beb) -# #25 0x00007999e834ebe3 llvm::ThreadPool::processTasks(llvm::ThreadPoolTaskGroup*) (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xd4ebe3) -# #26 0x00007999e834f704 (/usr/lib/llvm-18/bin/../lib/libLLVM.so.18.1+0xd4f704) -# #27 0x00007999e6e9ca94 (/lib/x86_64-linux-gnu/libc.so.6+0x9ca94) -# #28 0x00007999e6f29c3c (/lib/x86_64-linux-gnu/libc.so.6+0x129c3c) -# clang++-18: error: unable to execute command: Aborted (core dumped) -# clang++-18: error: linker command failed due to signal (use -v to see invocation) -# ^[[A^Cninja: build stopped: interrupted by user. From f2e8237fe92f6aa7fccf7efd432cec8f9e94485d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Nov 2024 16:05:13 +0100 Subject: [PATCH 190/433] Fix the test --- src/Common/computeMaxTableNameLength.cpp | 4 +++- .../0_stateless/03167_improvement_table_name_too_long.sh | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Common/computeMaxTableNameLength.cpp b/src/Common/computeMaxTableNameLength.cpp index 0f5976569e5..eb11c43a149 100644 --- a/src/Common/computeMaxTableNameLength.cpp +++ b/src/Common/computeMaxTableNameLength.cpp @@ -22,11 +22,13 @@ size_t computeMaxTableNameLength(const String & database_name, ContextPtr contex size_t max_dropped_length = get_max_name_length(metadata_dropped_path); size_t escaped_db_name_length = escapeForFileName(database_name).length(); + const size_t dot = 1; const size_t uuid_length = 36; // Standard UUID length const size_t extension_length = strlen(".sql"); // Adjust for database name and UUID in dropped table filenames - size_t max_to_drop = max_dropped_length - escaped_db_name_length - uuid_length - extension_length; + // Max path will look like this: ./metadata_dropped/{db_name}.{table_name}.{uuid}.{extension} + size_t max_to_drop = max_dropped_length - dot - escaped_db_name_length - dot - uuid_length - extension_length; // Return the minimum of the two calculated lengths return std::min(max_create_length, max_to_drop); diff --git a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh index 4ba0b77b752..c599c44c2ff 100755 --- a/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh +++ b/tests/queries/0_stateless/03167_improvement_table_name_too_long.sh @@ -4,12 +4,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -allowed_name_length=$(( $($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") - 5 )) # Reserve 5 characters for 'test_' prefix -excess_length=$((allowed_name_length + 1)) # Ensure exceeding the limit +allowed_name_length=$($CLICKHOUSE_CLIENT -mn --query="SELECT getMaxTableNameLengthForDatabase('$CLICKHOUSE_DATABASE')") +excess_length=$((allowed_name_length + 1)) # Ensure exceeding the limit long_table_name=$(openssl rand -base64 $excess_length | tr -dc A-Za-z | head -c $excess_length) allowed_table_name=$(openssl rand -base64 $allowed_name_length | tr -dc A-Za-z | head -c $allowed_name_length) -$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o 'ARGUMENT_OUT_OF_BOUND' +$CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $long_table_name (id UInt32, long_table_name String) Engine=MergeTree() order by id;" 2>&1 | grep -o -m 1 'ARGUMENT_OUT_OF_BOUND' $CLICKHOUSE_CLIENT -mn --query="CREATE TABLE $allowed_table_name (id UInt32, allowed_table_name String) Engine=MergeTree() order by id;" $CLICKHOUSE_CLIENT -mn --query="DROP TABLE $allowed_table_name;" From fa94338f15c9bed5a1088b01f5cdf4fa24ffb69a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Nov 2024 15:10:42 +0000 Subject: [PATCH 191/433] add test --- ...74_squashing_transform_sparse_bug.reference | 0 .../03274_squashing_transform_sparse_bug.sql | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/03274_squashing_transform_sparse_bug.reference create mode 100644 tests/queries/0_stateless/03274_squashing_transform_sparse_bug.sql diff --git a/tests/queries/0_stateless/03274_squashing_transform_sparse_bug.reference b/tests/queries/0_stateless/03274_squashing_transform_sparse_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03274_squashing_transform_sparse_bug.sql b/tests/queries/0_stateless/03274_squashing_transform_sparse_bug.sql new file mode 100644 index 00000000000..4ea6dcc80e2 --- /dev/null +++ b/tests/queries/0_stateless/03274_squashing_transform_sparse_bug.sql @@ -0,0 +1,18 @@ + +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; + +SET max_insert_block_size = 1; +SET min_insert_block_size_rows = 1; +SET min_insert_block_size_bytes = 1; + +CREATE TABLE t0 (x UInt64, y Tuple(UInt64, UInt64) ) ENGINE = MergeTree ORDER BY x SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; +SYSTEM STOP MERGES t0; +INSERT INTO t0 SELECT if(number % 2 = 0, 0, number) as x, (x, 0) from numbers(200) SETTINGS max_block_size = 1; + +CREATE TABLE t1 (x UInt64, y Tuple(UInt64, UInt64) ) ENGINE = MergeTree ORDER BY x; + +SET min_joined_block_size_bytes = 100; + +SET join_algorithm = 'parallel_hash'; +SELECT sum(ignore(*)) FROM t0 a FULL JOIN t1 b ON a.x = b.x FORMAT Null; From 5abca6283ad14c84609c42fe876ef85b8cc23f20 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Nov 2024 15:12:04 +0000 Subject: [PATCH 192/433] up --- src/Interpreters/Squashing.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 04cd1698067..d8809265011 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -147,14 +147,12 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl { /// IColumn::structureEquals is not implemented for deprecated object type, ignore it and always convert to non-sparse. bool has_object_deprecated = false; - columns[j]->forEachSubcolumnRecursively([&has_object_deprecated](const auto & subcolumn) + auto has_object_deprecated_lambda = [&has_object_deprecated](const auto & subcolumn) { has_object_deprecated = has_object_deprecated || subcolumn.getDataType() == TypeIndex::ObjectDeprecated; - }); - mutable_columns[j]->forEachSubcolumnRecursively([&has_object_deprecated](const auto & subcolumn) - { - has_object_deprecated = has_object_deprecated || subcolumn.getDataType() == TypeIndex::ObjectDeprecated; - }); + }; + columns[j]->forEachSubcolumnRecursively(has_object_deprecated); + mutable_columns[j]->forEachSubcolumnRecursively(has_object_deprecated); /// Need to check if there are any sparse columns in subcolumns, /// since `IColumn::isSparse` is not recursive but sparse column can be inside a tuple, for example. From 844513b1d0e9faae844a2be3c0a4a9509e4f0635 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 22 Nov 2024 15:46:52 +0100 Subject: [PATCH 193/433] correctly cast write buffer --- src/IO/WriteHelpers.h | 2 +- .../03274_pretty_output_coloring.expect | 36 +++++++++++++++++++ .../03274_pretty_output_coloring.reference | 0 3 files changed, 37 insertions(+), 1 deletion(-) create mode 100755 tests/queries/0_stateless/03274_pretty_output_coloring.expect create mode 100644 tests/queries/0_stateless/03274_pretty_output_coloring.reference diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index f7b2504f664..b186ae20ebf 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1451,7 +1451,7 @@ String fourSpaceIndent(size_t indent); bool inline isWritingToTerminal(const WriteBuffer & buf) { - const auto * write_buffer_to_descriptor = typeid_cast(&buf); + const auto * write_buffer_to_descriptor = dynamic_cast(&buf); return write_buffer_to_descriptor && write_buffer_to_descriptor->getFD() == STDOUT_FILENO && isatty(STDOUT_FILENO); } diff --git a/tests/queries/0_stateless/03274_pretty_output_coloring.expect b/tests/queries/0_stateless/03274_pretty_output_coloring.expect new file mode 100755 index 00000000000..8e2985a6fa7 --- /dev/null +++ b/tests/queries/0_stateless/03274_pretty_output_coloring.expect @@ -0,0 +1,36 @@ +#!/usr/bin/expect -f + +set basedir [file dirname $argv0] +set basename [file tail $argv0] +if {[info exists env(CLICKHOUSE_TMP)]} { + set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) +} else { + set CLICKHOUSE_TMP "." +} +exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 +set history_file $CLICKHOUSE_TMP/$basename.history + +log_user 0 +set timeout 60 +match_max 100000 + +expect_after { + # Do not ignore eof from expect + -i $any_spawn_id eof { exp_continue } + # A default timeout action is to do nothing, change it to fail + -i $any_spawn_id timeout { exit 1 } +} + +# useful debugging configuration +# exp_internal 1 + +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --enable-progress-table-toggle=0 --highlight 0 --history_file=$history_file" +expect ":) " + +# Make a query +send -- "SELECT 1 as Hello\r" +expect -re "\\\[1mHello.*\\\[90m1\\\." +expect ":) " + +send -- "exit\r" +expect eof diff --git a/tests/queries/0_stateless/03274_pretty_output_coloring.reference b/tests/queries/0_stateless/03274_pretty_output_coloring.reference new file mode 100644 index 00000000000..e69de29bb2d From 532632c20306f0e0d3ddc8cb2c801aa4a638893b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Nov 2024 16:31:08 +0100 Subject: [PATCH 194/433] Update 03253_getMaxTableNameLength.reference --- tests/queries/0_stateless/03253_getMaxTableNameLength.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03253_getMaxTableNameLength.reference b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference index 0e1d238cd78..560a78db396 100644 --- a/tests/queries/0_stateless/03253_getMaxTableNameLength.reference +++ b/tests/queries/0_stateless/03253_getMaxTableNameLength.reference @@ -1,2 +1,2 @@ -208 206 +204 From 2bea36c19f3f7e8bf515495bcccea343e41dad59 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Nov 2024 15:35:45 +0000 Subject: [PATCH 195/433] fix --- src/Interpreters/Squashing.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index d8809265011..6f877f4d2db 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -146,13 +146,14 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl for (size_t j = 0; j != num_columns; ++j) { /// IColumn::structureEquals is not implemented for deprecated object type, ignore it and always convert to non-sparse. - bool has_object_deprecated = false; + bool has_object_deprecated = columns[j]->getDataType() == TypeIndex::ObjectDeprecated || + mutable_columns[j]->getDataType() == TypeIndex::ObjectDeprecated; auto has_object_deprecated_lambda = [&has_object_deprecated](const auto & subcolumn) { has_object_deprecated = has_object_deprecated || subcolumn.getDataType() == TypeIndex::ObjectDeprecated; }; - columns[j]->forEachSubcolumnRecursively(has_object_deprecated); - mutable_columns[j]->forEachSubcolumnRecursively(has_object_deprecated); + columns[j]->forEachSubcolumnRecursively(has_object_deprecated_lambda); + mutable_columns[j]->forEachSubcolumnRecursively(has_object_deprecated_lambda); /// Need to check if there are any sparse columns in subcolumns, /// since `IColumn::isSparse` is not recursive but sparse column can be inside a tuple, for example. From 0ca97d30365ea3d2b371205f851c1cf406d2620c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 22 Nov 2024 18:08:00 +0100 Subject: [PATCH 196/433] Use PreformattedMessage in S3Exception --- src/IO/S3Common.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 22b590dcb18..4e8f595d87f 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -33,8 +33,7 @@ public: // Format message with fmt::format, like the logging functions. template S3Exception(Aws::S3::S3Errors code_, fmt::format_string fmt, Args &&... args) - : Exception(fmt::format(fmt, std::forward(args)...), ErrorCodes::S3_ERROR) - , code(code_) + : Exception(ErrorCodes::S3_ERROR, PreformattedMessage::create(fmt, std::forward(args)...)), code(code_) { } From ff6f10843fc3bcf8455bb9c2d0bed2ad2c29bd54 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 22 Nov 2024 17:15:16 +0000 Subject: [PATCH 197/433] disable a cloud setting --- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 081e07ca2ce..63c06e88e79 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5751,7 +5751,7 @@ Allow JSON data type DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). )", EXPERIMENTAL) \ - DECLARE(Bool, allow_experimental_shared_set_join, true, R"( + DECLARE(Bool, allow_experimental_shared_set_join, false, R"( Only in ClickHouse Cloud. Allow to create ShareSet and SharedJoin )", EXPERIMENTAL) \ DECLARE(UInt64, max_limit_for_ann_queries, 1'000'000, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ed87fde8b7e..7ce502d232a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -75,6 +75,7 @@ static std::initializer_list Date: Fri, 22 Nov 2024 17:21:38 +0000 Subject: [PATCH 198/433] minor enhancements --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Core/Defines.h | 3 +++ src/Core/ServerSettings.cpp | 6 +++--- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterSystemQuery.h | 2 -- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp | 1 - src/Storages/MergeTree/IMergedBlockOutputStream.h | 1 - src/Storages/MergeTree/MergeTask.h | 4 ---- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 8 ++------ src/Storages/MergeTree/MergeTreeSettings.cpp | 4 ++-- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 4 ++-- .../MergeTree/MergedColumnOnlyOutputStream.cpp | 4 ++-- src/Storages/MergeTree/PrimaryIndexCache.h | 7 ++++--- src/Storages/MergeTree/new_index | 0 20 files changed, 30 insertions(+), 39 deletions(-) delete mode 100644 src/Storages/MergeTree/new_index diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index adfd3adcb2a..38cab468b29 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -787,7 +787,7 @@ void LocalServer::processConfig() if (primary_index_cache_size > max_cache_size) { primary_index_cache_size = max_cache_size; - LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(primary_index_cache_size)); } global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0b76adebb41..ebf69481f0a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1571,7 +1571,7 @@ try if (primary_index_cache_size > max_cache_size) { primary_index_cache_size = max_cache_size; - LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Lowered primary index cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(primary_index_cache_size)); } global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); diff --git a/src/Core/Defines.h b/src/Core/Defines.h index c6e65f34e90..faa5dc3a3fe 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -95,6 +95,9 @@ static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_SIZE_RATIO = 0.5l; static constexpr auto DEFAULT_MARK_CACHE_POLICY = "SLRU"; static constexpr auto DEFAULT_MARK_CACHE_MAX_SIZE = 5_GiB; static constexpr auto DEFAULT_MARK_CACHE_SIZE_RATIO = 0.5l; +static constexpr auto DEFAULT_PRIMARY_INDEX_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_PRIMARY_INDEX_CACHE_MAX_SIZE = 5_GiB; +static constexpr auto DEFAULT_PRIMARY_INDEX_CACHE_SIZE_RATIO = 0.5l; static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY = "SLRU"; static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE = 0; static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_SIZE_RATIO = 0.5; diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 78a3ad3c419..034b1392367 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -101,9 +101,9 @@ namespace DB DECLARE(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ DECLARE(Double, mark_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the mark cache relative to the cache's total size.", 0) \ DECLARE(Double, mark_cache_prewarm_ratio, 0.95, "The ratio of total size of mark cache to fill during prewarm.", 0) \ - DECLARE(String, primary_index_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Primary index cache policy name.", 0) \ - DECLARE(UInt64, primary_index_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for primary index (index of MergeTree family of tables).", 0) \ - DECLARE(Double, primary_index_cache_size_ratio, DEFAULT_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the primary index cache relative to the cache's total size.", 0) \ + DECLARE(String, primary_index_cache_policy, DEFAULT_PRIMARY_INDEX_CACHE_POLICY, "Primary index cache policy name.", 0) \ + DECLARE(UInt64, primary_index_cache_size, DEFAULT_PRIMARY_INDEX_CACHE_MAX_SIZE, "Size of cache for primary index (index of MergeTree family of tables).", 0) \ + DECLARE(Double, primary_index_cache_size_ratio, DEFAULT_PRIMARY_INDEX_CACHE_SIZE_RATIO, "The size of the protected queue in the primary index cache relative to the cache's total size.", 0) \ DECLARE(Double, primary_index_cache_prewarm_ratio, 0.95, "The ratio of total size of mark cache to fill during prewarm.", 0) \ DECLARE(String, index_uncompressed_cache_policy, DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY, "Secondary index uncompressed cache policy name.", 0) \ DECLARE(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of secondary indices. Zero means disabled.", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc8b68f72c2..6b571275f18 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3259,9 +3259,9 @@ void Context::updatePrimaryIndexCacheConfiguration(const Poco::Util::AbstractCon std::lock_guard lock(shared->mutex); if (!shared->primary_index_cache) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Mark cache was not created yet."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Primary index cache was not created yet."); - size_t max_size_in_bytes = config.getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + size_t max_size_in_bytes = config.getUInt64("primary_index_cache_size", DEFAULT_PRIMARY_INDEX_CACHE_MAX_SIZE); shared->primary_index_cache->setMaxSizeInBytes(max_size_in_bytes); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index e4db880da6a..8dd68b51ac4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -22,7 +22,6 @@ #include #include -#include "Storages/MergeTree/PrimaryIndexCache.h" #include "config.h" #include @@ -90,6 +89,7 @@ class RefreshSet; class Cluster; class Compiler; class MarkCache; +class PrimaryIndexCache; class PageCache; class MMappedFileCache; class UncompressedCache; diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 47fac8330a4..09cdeb72093 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -6,8 +6,6 @@ #include #include #include -#include "Storages/MarkCache.h" -#include "Storages/MergeTree/PrimaryIndexCache.h" #include diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 2aa9ac6a17c..4322f56c4cd 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index cf8989ee4df..2dff4fff80e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -25,7 +25,7 @@ #include #include #include -#include "Storages/MergeTree/PrimaryIndexCache.h" +#include namespace zkutil diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 41185447c7c..37a9cbffa4c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 53727d4578a..77f22711b8d 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 29b48f8d362..721a2a933e1 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -6,8 +6,6 @@ #include #include #include -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include "Storages/MergeTree/PrimaryIndexCache.h" #include #include @@ -223,9 +221,7 @@ private: std::promise promise{}; IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; - PlainMarksByName cached_marks; - std::unique_ptr cached_index; MergeTreeTransactionPtr txn; bool need_prefix; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 10b518822ac..14a0835d99c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -232,8 +232,8 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString storage_policy; extern const MergeTreeSettingsFloat zero_copy_concurrent_part_removal_max_postpone_ratio; extern const MergeTreeSettingsUInt64 zero_copy_concurrent_part_removal_max_split_times; - extern const MergeTreeSettingsBool use_primary_index_cache; - extern const MergeTreeSettingsBool prewarm_primary_index_cache; + extern const MergeTreeSettingsBool use_primary_key_cache; + extern const MergeTreeSettingsBool prewarm_primary_key_cache; extern const MergeTreeSettingsBool prewarm_mark_cache; extern const MergeTreeSettingsBool primary_key_lazy_load; } @@ -2350,10 +2350,10 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCache() const { - bool use_primary_index_cache = (*getSettings())[MergeTreeSetting::use_primary_index_cache]; + bool use_primary_key_cache = (*getSettings())[MergeTreeSetting::use_primary_key_cache]; bool primary_key_lazy_load = (*getSettings())[MergeTreeSetting::primary_key_lazy_load]; - if (!use_primary_index_cache || !primary_key_lazy_load) + if (!use_primary_key_cache || !primary_key_lazy_load) return nullptr; return getContext()->getPrimaryIndexCache(); @@ -2361,7 +2361,7 @@ PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCache() const PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCacheToPrewarm() const { - if (!(*getSettings())[MergeTreeSetting::prewarm_primary_index_cache]) + if (!(*getSettings())[MergeTreeSetting::prewarm_primary_key_cache]) return nullptr; return getPrimaryIndexCache(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 58a909e6a2d..240cfa71748 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -515,7 +515,7 @@ public: /// Returns a pointer to primary mark cache if it is required to be prewarmed. MarkCachePtr getMarkCacheToPrewarm() const; - /// Prewarm mark cache for the most recent data parts. + /// Prewarm mark cache and primary index cache for the most recent data parts. void prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache); String getLogName() const { return log.loadName(); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f6e42079d41..bd6feb99329 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -338,13 +338,9 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc */ MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; - if (settings.save_primary_index_in_memory) + if (settings.save_primary_index_in_memory && index_columns.empty()) { - if (index_columns.empty()) - index_columns = primary_index_block.cloneEmptyColumns(); - - for (const auto & column : index_columns) - column->reserve(column->size() + granules_to_write.size()); + index_columns = primary_index_block.cloneEmptyColumns(); } /// Write index. The index contains Primary Key value for each `index_granularity` row. diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index d37e2ec219e..fbb55d11416 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -238,8 +238,8 @@ namespace ErrorCodes DECLARE(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ DECLARE(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ DECLARE(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ - DECLARE(Bool, use_primary_index_cache, false, "Use cache for primary index instead of saving all indexes in memory. Can be useful for very large tables", 0) \ - DECLARE(Bool, prewarm_primary_index_cache, false, "If true primary index cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ + DECLARE(Bool, use_primary_key_cache, false, "Use cache for primary index instead of saving all indexes in memory. Can be useful for very large tables", 0) \ + DECLARE(Bool, prewarm_primary_key_cache, false, "If true primary index cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(Bool, prewarm_mark_cache, false, "If true mark cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(String, columns_to_prewarm_mark_cache, "", "List of columns to prewarm mark cache for (if enabled). Empty means all columns", 0) \ /** Projection settings. */ \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 01359af021e..db162bd82cb 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -37,9 +37,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( , default_codec(default_codec_) , write_settings(write_settings_) { - /// Save marks in memory if prewarm is enabled to avoid rereading marks file. + /// Save marks in memory if prewarm is enabled to avoid re-reading marks file. bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; - /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid rereading marks file. + /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid re-reading primary index file. bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); MergeTreeWriterSettings writer_settings( diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 46754d005ba..9f6ab952bc3 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -24,9 +24,9 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( WrittenOffsetColumns * offset_columns) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { - /// Save marks in memory if prewarm is enabled to avoid rereading marks file. + /// Save marks in memory if prewarm is enabled to avoid re-reading marks file. bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; - /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid rereading marks file. + /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid re-reading priamry index file. bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); /// Granularity is never recomputed while writing only columns. diff --git a/src/Storages/MergeTree/PrimaryIndexCache.h b/src/Storages/MergeTree/PrimaryIndexCache.h index 758f18dbed8..5ec185dcf58 100644 --- a/src/Storages/MergeTree/PrimaryIndexCache.h +++ b/src/Storages/MergeTree/PrimaryIndexCache.h @@ -16,7 +16,7 @@ namespace DB using PrimaryIndex = std::vector; -/// Estimate of number of bytes in cache for primaryindexs. +/// Estimate of number of bytes in cache for primary index. struct PrimaryIndexWeightFunction { /// We spent additional bytes on key in hashmap, linked lists, shared pointers, etc ... @@ -33,8 +33,9 @@ struct PrimaryIndexWeightFunction extern template class CacheBase; -/** Cache of primary index for StorageMergeTree. - * PrimaryIndex is an index structure that addresses ranges in column file, corresponding to ranges of primary key. +/** Cache of primary index for MergeTree tables. + * Primary index is a list of columns from primary key + * that store first row for each granule of data part. */ class PrimaryIndexCache : public CacheBase { diff --git a/src/Storages/MergeTree/new_index b/src/Storages/MergeTree/new_index deleted file mode 100644 index e69de29bb2d..00000000000 From bd3f300220b113fc08a0a11c68296513913bb5a9 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 22 Nov 2024 17:33:37 +0000 Subject: [PATCH 199/433] Move JSON/Dynamic/Variant types from experimental to beta --- docs/en/sql-reference/data-types/dynamic.md | 4 ++-- docs/en/sql-reference/data-types/newjson.md | 4 ++-- docs/en/sql-reference/data-types/variant.md | 2 +- src/Core/Settings.cpp | 19 ++++++++++--------- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/data-types/dynamic.md b/docs/en/sql-reference/data-types/dynamic.md index aa7455c8f68..08ccc4b8827 100644 --- a/docs/en/sql-reference/data-types/dynamic.md +++ b/docs/en/sql-reference/data-types/dynamic.md @@ -17,7 +17,7 @@ To declare a column of `Dynamic` type, use the following syntax: Where `N` is an optional parameter between `0` and `254` indicating how many different data types can be stored as separate subcolumns inside a column with type `Dynamic` across single block of data that is stored separately (for example across single data part for MergeTree table). If this limit is exceeded, all values with new types will be stored together in a special shared data structure in binary form. Default value of `max_types` is `32`. :::note -The Dynamic data type is an experimental feature. To use it, set `allow_experimental_dynamic_type = 1`. +The Dynamic data type is a beta feature. To use it, set `enable_dynamic_type = 1`. ::: ## Creating Dynamic @@ -54,7 +54,7 @@ SELECT 'Hello, World!'::Dynamic as d, dynamicType(d); Using CAST from `Variant` column: ```sql -SET allow_experimental_variant_type = 1, use_variant_as_common_type = 1; +SET enable_variant_type = 1, use_variant_as_common_type = 1; SELECT multiIf((number % 3) = 0, number, (number % 3) = 1, range(number + 1), NULL)::Dynamic AS d, dynamicType(d) FROM numbers(3) ``` diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 05197d9e692..bd65742d06f 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -10,8 +10,8 @@ keywords: [json, data type] Stores JavaScript Object Notation (JSON) documents in a single column. :::note -This feature is experimental and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-formats/json/overview) instead. -If you want to use JSON type, set `allow_experimental_json_type = 1`. +This feature is beta and is not production-ready. If you need to work with JSON documents, consider using [this guide](/docs/en/integrations/data-formats/json/overview) instead. +If you want to use JSON type, set `enable_json_type = 1`. ::: To declare a column of `JSON` type, use the following syntax: diff --git a/docs/en/sql-reference/data-types/variant.md b/docs/en/sql-reference/data-types/variant.md index 7cb0f4ad4ea..37a7bf1894e 100644 --- a/docs/en/sql-reference/data-types/variant.md +++ b/docs/en/sql-reference/data-types/variant.md @@ -18,7 +18,7 @@ because working with values of such types can lead to ambiguity. By default, cre ::: :::note -The Variant data type is an experimental feature. To use it, set `allow_experimental_variant_type = 1`. +The Variant data type is a beta feature. To use it, set `enable_variant_type = 1`. ::: ## Creating Variant diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 734523a208b..f58e854d50a 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5741,6 +5741,16 @@ In `clickhouse-local` it is enabled by default and can be explicitly disabled. DECLARE(Bool, push_external_roles_in_interserver_queries, true, R"( Enable pushing user roles from originator to other nodes while performing a query. )", 0) \ + \ + DECLARE(Bool, allow_experimental_variant_type, false, R"( +Allows creation of [Variant](../../sql-reference/data-types/variant.md) data type. +)", BETA) ALIAS(enable_variant_type) \ + DECLARE(Bool, allow_experimental_dynamic_type, false, R"( +Allows creation of [Dynamic](../../sql-reference/data-types/dynamic.md) data type. +)", BETA) ALIAS(enable_dynamic_type) \ + DECLARE(Bool, allow_experimental_json_type, false, R"( +Allows creation of [JSON](../../sql-reference/data-types/newjson.md) data type. +)", BETA) ALIAS(enable_json_type) \ \ \ /* ####################################################### */ \ @@ -5776,15 +5786,6 @@ Possible values: )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_vector_similarity_index, false, R"( Allow experimental vector similarity index -)", EXPERIMENTAL) \ - DECLARE(Bool, allow_experimental_variant_type, false, R"( -Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). -)", EXPERIMENTAL) \ - DECLARE(Bool, allow_experimental_dynamic_type, false, R"( -Allow Dynamic data type -)", EXPERIMENTAL) \ - DECLARE(Bool, allow_experimental_json_type, false, R"( -Allow JSON data type )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). From f0888d5b69213bf70fb03ad01e3e4d0820aa8dad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 22 Nov 2024 18:16:06 +0000 Subject: [PATCH 200/433] remove unused method --- src/Interpreters/MutationsInterpreter.cpp | 6 ------ src/Interpreters/MutationsInterpreter.h | 1 - 2 files changed, 7 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index db61bd6f6ab..5662f069bad 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -343,12 +343,6 @@ bool MutationsInterpreter::Source::supportsLightweightDelete() const return storage->supportsLightweightDelete(); } - -bool MutationsInterpreter::Source::hasLightweightDeleteMask() const -{ - return part && part->hasLightweightDelete(); -} - bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const { return data && (*data->getSettings())[MergeTreeSetting::materialize_ttl_recalculate_only]; diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 901cd13cd2f..3f07d57362d 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -122,7 +122,6 @@ public: const MergeTreeData * getMergeTreeData() const; bool supportsLightweightDelete() const; - bool hasLightweightDeleteMask() const; bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; From 6c7cde57028ca11e49b6f695e37734ba50b98531 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 22 Nov 2024 19:56:07 +0100 Subject: [PATCH 201/433] Better --- src/IO/S3Common.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 4e8f595d87f..eedc3109ded 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -32,8 +32,8 @@ public: // Format message with fmt::format, like the logging functions. template - S3Exception(Aws::S3::S3Errors code_, fmt::format_string fmt, Args &&... args) - : Exception(ErrorCodes::S3_ERROR, PreformattedMessage::create(fmt, std::forward(args)...)), code(code_) + S3Exception(Aws::S3::S3Errors code_, FormatStringHelper fmt, Args &&... args) + : Exception(PreformattedMessage{fmt.format(std::forward(args)...)}, ErrorCodes::S3_ERROR), code(code_) { } From b3344558c1d708907c9b518dbd7b16700c83e014 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 21:01:05 +0100 Subject: [PATCH 202/433] fix --- tests/integration/helpers/postgres_utility.py | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index ba6ec1fd66e..c6c8fc01211 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -343,16 +343,20 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") + + # Check based on `system.tables` is not enough, because tables appear there before they are loaded. + # It may lead to error `Unknown table expression identifier...` + while True: + try: + instance.query(f"SELECT * FROM `{materialized_database}`.`{table}` LIMIT 1 FORMAT Null") + break + except Exception: + time.sleep(0.2) + continue + database_tables = instance.query( f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" ) - - while table not in database_tables: - time.sleep(0.2) - database_tables = instance.query( - f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" - ) - assert table in database_tables From 7dead112d17fe9b0fd42640e9c33060967cd3eb1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 22 Nov 2024 20:11:08 +0000 Subject: [PATCH 203/433] Automatic style fix --- tests/integration/helpers/postgres_utility.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index c6c8fc01211..6a91c41154a 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -348,7 +348,9 @@ def assert_nested_table_is_created( # It may lead to error `Unknown table expression identifier...` while True: try: - instance.query(f"SELECT * FROM `{materialized_database}`.`{table}` LIMIT 1 FORMAT Null") + instance.query( + f"SELECT * FROM `{materialized_database}`.`{table}` LIMIT 1 FORMAT Null" + ) break except Exception: time.sleep(0.2) From 52391a82719f62bd733d9c2b5ca8c7874c14fafb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 22 Nov 2024 22:51:28 +0100 Subject: [PATCH 204/433] Revert "Set enable_http_compression default value to 1" --- docs/ru/operations/settings/settings.md | 2 +- docs/zh/operations/settings/settings.md | 2 +- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 1 - 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 222acc7aa4c..a6b45f4a792 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -136,7 +136,7 @@ ClickHouse применяет настройку в тех случаях, ко - 0 — выключена. - 1 — включена. -Значение по умолчанию: 1. +Значение по умолчанию: 0. ## http_zlib_compression_level {#settings-http_zlib_compression_level} diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index baa4fcb0754..5e59196f56c 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -97,7 +97,7 @@ ClickHouse从表的过时副本中选择最相关的副本。 - 0 — Disabled. - 1 — Enabled. -默认值:1。 +默认值:0。 ## http_zlib_compression_level {#settings-http_zlib_compression_level} diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index d63ab6feb96..9348d67a771 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1800,7 +1800,7 @@ Possible values: - 0 — Disabled. - 1 — Enabled. -)", 1) \ +)", 0) \ DECLARE(Int64, http_zlib_compression_level, 3, R"( Sets the level of data compression in the response to an HTTP request if [enable_http_compression = 1](#enable_http_compression). diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e01554049c7..95f4f573985 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,7 +64,6 @@ static std::initializer_list Date: Fri, 22 Nov 2024 22:03:53 +0000 Subject: [PATCH 205/433] fix tests --- tests/queries/0_stateless/03273_primary_index_cache.sql | 2 +- .../queries/0_stateless/03274_prewarm_primary_index_cache.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03273_primary_index_cache.sql b/tests/queries/0_stateless/03273_primary_index_cache.sql index 18c2703689b..04a03797bcb 100644 --- a/tests/queries/0_stateless/03273_primary_index_cache.sql +++ b/tests/queries/0_stateless/03273_primary_index_cache.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS t_primary_index_cache; CREATE TABLE t_primary_index_cache (a UInt64, b UInt64) ENGINE = MergeTree ORDER BY a PARTITION BY a % 2 -SETTINGS use_primary_index_cache = 1, prewarm_primary_index_cache = 0, index_granularity = 64, index_granularity_bytes = '10M', min_bytes_for_wide_part = 0; +SETTINGS use_primary_key_cache = 1, prewarm_primary_key_cache = 0, index_granularity = 64, index_granularity_bytes = '10M', min_bytes_for_wide_part = 0; SYSTEM DROP PRIMARY INDEX CACHE; diff --git a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql index 08c41d0fc72..16e895a7798 100644 --- a/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql +++ b/tests/queries/0_stateless/03274_prewarm_primary_index_cache.sql @@ -6,12 +6,12 @@ DROP TABLE IF EXISTS t_prewarm_cache_rmt_2; CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03274_prewarm_mark_cache_smt/t_prewarm_cache', '1') ORDER BY a PARTITION BY a % 2 -SETTINGS prewarm_primary_index_cache = 1, use_primary_index_cache = 1; +SETTINGS prewarm_primary_key_cache = 1, use_primary_key_cache = 1; CREATE TABLE t_prewarm_cache_rmt_2 (a UInt64, b UInt64, c UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03274_prewarm_mark_cache_smt/t_prewarm_cache', '2') ORDER BY a PARTITION BY a % 2 -SETTINGS prewarm_primary_index_cache = 1, use_primary_index_cache = 1; +SETTINGS prewarm_primary_key_cache = 1, use_primary_key_cache = 1; SYSTEM DROP PRIMARY INDEX CACHE; SYSTEM STOP FETCHES t_prewarm_cache_rmt_2; From f3cd9d4c72e62a584b03fbf1377211bce393b286 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 4 Nov 2024 23:09:37 +0000 Subject: [PATCH 206/433] fix transaction rollback when file write finalize fails --- ...dataStorageFromPlainObjectStorageOperations.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 62015631aa5..3dac1df2ff8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -97,20 +97,22 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un { auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); - if (write_finalized) + if (write_finalized || write_created) { const auto base_path = path.parent_path(); + size_t erase_count = 0; { std::lock_guard lock(path_map.mutex); - path_map.map.erase(base_path); + erase_count = path_map.map.erase(base_path); + } + if (erase_count) + { + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::sub(metric, erase_count); } - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, 1); object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } - else if (write_created) - object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( From 9602e338ada4af6d28671f12fba41a1b78a44324 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 8 Nov 2024 06:29:00 +0000 Subject: [PATCH 207/433] address feedback Introduce removePathIfExists method. --- .../ObjectStorages/InMemoryDirectoryPathMap.h | 6 ++++ ...torageFromPlainObjectStorageOperations.cpp | 28 +++++++------------ 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h index 117cbad6203..d8e72d253f3 100644 --- a/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h +++ b/src/Disks/ObjectStorages/InMemoryDirectoryPathMap.h @@ -57,6 +57,12 @@ struct InMemoryDirectoryPathMap return it->second; } + bool removePathIfExists(const std::filesystem::path & path) + { + std::lock_guard lock(mutex); + return map.erase(path) != 0; + } + mutable SharedMutex mutex; #ifdef OS_LINUX diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 3dac1df2ff8..3c7473b50c9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -100,15 +100,10 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un if (write_finalized || write_created) { const auto base_path = path.parent_path(); - size_t erase_count = 0; - { - std::lock_guard lock(path_map.mutex); - erase_count = path_map.map.erase(base_path); - } - if (erase_count) + if (path_map.removePathIfExists(base_path)) { auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, erase_count); + CurrentMetrics::sub(metric, 1); } object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); @@ -251,19 +246,16 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: auto metadata_object = StoredObject(/*remote_path*/ metadata_object_key.serialize(), /*local_path*/ path / PREFIX_PATH_FILE_NAME); object_storage->removeObjectIfExists(metadata_object); + if (path_map.removePathIfExists(base_path)) { - std::lock_guard lock(path_map.mutex); - auto & map = path_map.map; - map.erase(base_path); + removed = true; + + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::sub(metric, 1); + + auto event = object_storage->getMetadataStorageMetrics().directory_removed; + ProfileEvents::increment(event); } - - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, 1); - - removed = true; - - auto event = object_storage->getMetadataStorageMetrics().directory_removed; - ProfileEvents::increment(event); } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::unique_lock &) From 8de4ecf5131f9ed57f011be7334fa2ebcc1ed22f Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Nov 2024 01:44:23 +0000 Subject: [PATCH 208/433] add failpoint and test --- src/Common/FailPoint.cpp | 3 +- ...torageFromPlainObjectStorageOperations.cpp | 15 +++++-- ...aStorageFromPlainObjectStorageOperations.h | 1 - .../03008_s3_plain_rewritable_fault.reference | 1 + .../03008_s3_plain_rewritable_fault.sh | 43 +++++++++++++++++++ 5 files changed, 58 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference create mode 100755 tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index ef9e6bc96a9..a640523ac52 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -41,7 +41,7 @@ static struct InitFiu REGULAR(use_delayed_remote_source) \ REGULAR(cluster_discovery_faults) \ REGULAR(replicated_sends_failpoint) \ - REGULAR(stripe_log_sink_write_fallpoint)\ + REGULAR(stripe_log_sink_write_fallpoint) \ ONCE(smt_commit_merge_mutate_zk_fail_after_op) \ ONCE(smt_commit_merge_mutate_zk_fail_before_op) \ ONCE(smt_commit_write_zk_fail_after_op) \ @@ -77,6 +77,7 @@ static struct InitFiu REGULAR(replicated_merge_tree_all_replicas_stale) \ REGULAR(zero_copy_lock_zk_fail_before_op) \ REGULAR(zero_copy_lock_zk_fail_after_op) \ + REGULAR(plain_object_storage_write_fail_on_directory_create) \ namespace FailPoints diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 3c7473b50c9..f2b9e5161b8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -18,8 +19,14 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int FILE_ALREADY_EXISTS; extern const int INCORRECT_DATA; +extern const int FAULT_INJECTED; }; +namespace FailPoints +{ +extern const char plain_object_storage_write_fail_on_directory_create[]; +} + namespace { @@ -72,8 +79,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, /* settings */ {}); - write_created = true; - { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; @@ -85,6 +90,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: CurrentMetrics::add(metric, 1); writeString(path.string(), *buf); + fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_create, { + throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when creating '{}' directory", path); + }); buf->finalize(); write_finalized = true; @@ -97,8 +105,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un { auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); - if (write_finalized || write_created) + if (write_finalized) { + LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Undoing '{}' directory creation", path); const auto base_path = path.parent_path(); if (path_map.removePathIfExists(base_path)) { diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 565d4429548..0a26dfed3e2 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -19,7 +19,6 @@ private: const std::string metadata_key_prefix; const std::string object_key_prefix; - bool write_created = false; bool write_finalized = false; public: diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh new file mode 100755 index 00000000000..274e7c148c2 --- /dev/null +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-shared-merge-tree +# Tag no-fasttest: requires S3 +# Tag no-shared-merge-tree: does not support replication + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +on_exit() { + ${CLICKHOUSE_CLIENT} --query " + SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create +" +} + +trap on_exit EXIT + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test_s3_mt_fault" + +${CLICKHOUSE_CLIENT} --query " +CREATE TABLE test_s3_mt_fault (a Int32, b Int64) engine = MergeTree() ORDER BY tuple(a, b) +SETTINGS disk = disk( + name = 03008_s3_plain_rewritable_fault, + type = s3_plain_rewritable, + endpoint = 'http://localhost:11111/test/03008_test_s3_mt_fault/', + access_key_id = clickhouse, + secret_access_key = clickhouse); +" + +${CLICKHOUSE_CLIENT} --query " +SYSTEM ENABLE FAILPOINT plain_object_storage_write_fail_on_directory_create +" + +${CLICKHOUSE_CLIENT} --query " +INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100)'" 2>&1 | grep -Fq "FAULT_INJECTED" + +${CLICKHOUSE_CLIENT} --query "SELECT count(*) from test_s3_mt_fault;" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault;" + +${CLICKHOUSE_CLIENT} --query " +SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create +" From 208e96f360990af60b30c1284b2465ce0bb0d400 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Nov 2024 03:29:07 +0000 Subject: [PATCH 209/433] add plain_object_storage_write_fail_on_directory_move fault injection --- src/Common/FailPoint.cpp | 1 + ...torageFromPlainObjectStorageOperations.cpp | 8 ++--- ...aStorageFromPlainObjectStorageOperations.h | 1 - .../03008_s3_plain_rewritable_fault.reference | 13 ++++++- .../03008_s3_plain_rewritable_fault.sh | 34 ++++++++++++++----- 5 files changed, 43 insertions(+), 14 deletions(-) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index a640523ac52..027cc347386 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -78,6 +78,7 @@ static struct InitFiu REGULAR(zero_copy_lock_zk_fail_before_op) \ REGULAR(zero_copy_lock_zk_fail_after_op) \ REGULAR(plain_object_storage_write_fail_on_directory_create) \ + REGULAR(plain_object_storage_write_fail_on_directory_move) \ namespace FailPoints diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index f2b9e5161b8..96832c4dfb1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -25,6 +25,7 @@ extern const int FAULT_INJECTED; namespace FailPoints { extern const char plain_object_storage_write_fail_on_directory_create[]; +extern const char plain_object_storage_write_fail_on_directory_move[]; } namespace @@ -190,8 +191,10 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u getLogger("MetadataStorageFromPlainObjectStorageMoveDirectoryOperation"), "Moving directory '{}' to '{}'", path_from, path_to); auto write_buf = createWriteBuf(path_from, path_to, /* validate_content */ true); - write_created = true; writeString(path_to.string(), *write_buf); + fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_move, { + throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when moving from '{}' to '{}'", path_from, path_to); + }); write_buf->finalize(); /// parent_path() removes the trailing '/'. @@ -216,10 +219,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq std::lock_guard lock(path_map.mutex); auto & map = path_map.map; map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); - } - if (write_created) - { auto write_buf = createWriteBuf(path_to, path_from, /* verify_content */ false); writeString(path_from.string(), *write_buf); write_buf->finalize(); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 0a26dfed3e2..393fe21a4c3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -42,7 +42,6 @@ private: ObjectStoragePtr object_storage; const std::string metadata_key_prefix; - bool write_created = false; bool write_finalized = false; std::unique_ptr diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference index 573541ac970..2f87357245f 100644 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.reference @@ -1 +1,12 @@ -0 +1 2 +2 2 +3 1 +4 7 +5 10 +6 12 +1 2 +2 2 +3 1 +4 7 +5 10 +6 12 diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh index 274e7c148c2..89ed24e2b43 100755 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh @@ -1,15 +1,17 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-shared-merge-tree +# Tags: no-fasttest, no-shared-merge-tree, no-parallel # Tag no-fasttest: requires S3 # Tag no-shared-merge-tree: does not support replication +# Tag no-parallel: uses failpoints CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh on_exit() { - ${CLICKHOUSE_CLIENT} --query " - SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create + ${CLICKHOUSE_CLIENT} -m --query " + SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create; + SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_move; " } @@ -27,17 +29,33 @@ SETTINGS disk = disk( secret_access_key = clickhouse); " +${CLICKHOUSE_CLIENT} --query " +INSERT INTO test_s3_mt_fault (*) VALUES (1, 2), (2, 2), (3, 1), (4, 7), (5, 10), (6, 12); +OPTIMIZE TABLE test_s3_mt_fault FINAL; +" + ${CLICKHOUSE_CLIENT} --query " SYSTEM ENABLE FAILPOINT plain_object_storage_write_fail_on_directory_create " ${CLICKHOUSE_CLIENT} --query " -INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100)'" 2>&1 | grep -Fq "FAULT_INJECTED" +INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100)" 2>&1 | grep -Fq "FAULT_INJECTED" -${CLICKHOUSE_CLIENT} --query "SELECT count(*) from test_s3_mt_fault;" - -${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault;" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_s3_mt_fault;" ${CLICKHOUSE_CLIENT} --query " -SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create +SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_create; +SYSTEM ENABLE FAILPOINT plain_object_storage_write_fail_on_directory_move; " + +${CLICKHOUSE_CLIENT} --query " +INSERT INTO test_s3_mt_fault (*) select number, number from numbers_mt(100); +" 2>&1 | grep -Fq "FAULT_INJECTED" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_s3_mt_fault;" + +${CLICKHOUSE_CLIENT} --query " +SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_move; +" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault" From 16f3447ef94e5b4d0d2a9805a9251c013c4cf629 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Nov 2024 08:20:45 +0000 Subject: [PATCH 210/433] fix and simplify --- ...torageFromPlainObjectStorageOperations.cpp | 47 +++++++++---------- ...aStorageFromPlainObjectStorageOperations.h | 2 - 2 files changed, 21 insertions(+), 28 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 96832c4dfb1..8ed3d78895b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -80,6 +80,14 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE, /* settings */ {}); + writeString(path.string(), *buf); + fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_create, { + throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when creating '{}' directory", path); + }); + buf->finalize(); + + auto event = object_storage->getMetadataStorageMetrics().directory_created; + ProfileEvents::increment(event); { std::lock_guard lock(path_map.mutex); auto & map = path_map.map; @@ -89,35 +97,20 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std: } auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::add(metric, 1); - - writeString(path.string(), *buf); - fiu_do_on(FailPoints::plain_object_storage_write_fail_on_directory_create, { - throw Exception(ErrorCodes::FAULT_INJECTED, "Injecting fault when creating '{}' directory", path); - }); - buf->finalize(); - - write_finalized = true; - - auto event = object_storage->getMetadataStorageMetrics().directory_created; - ProfileEvents::increment(event); } void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock &) { - auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); - - if (write_finalized) + LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Undoing '{}' directory creation", path); + const auto base_path = path.parent_path(); + if (path_map.removePathIfExists(base_path)) { - LOG_TRACE(getLogger("MetadataStorageFromPlainObjectStorageCreateDirectoryOperation"), "Undoing '{}' directory creation", path); - const auto base_path = path.parent_path(); - if (path_map.removePathIfExists(base_path)) - { - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::sub(metric, 1); - } - - object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::sub(metric, 1); } + + auto metadata_object_key = createMetadataObjectKey(object_key_prefix, metadata_key_prefix); + object_storage->removeObjectIfExists(StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME)); } MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFromPlainObjectStorageMoveDirectoryOperation( @@ -216,9 +209,11 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq { if (write_finalized) { - std::lock_guard lock(path_map.mutex); - auto & map = path_map.map; - map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); + { + std::lock_guard lock(path_map.mutex); + auto & map = path_map.map; + map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped()); + } auto write_buf = createWriteBuf(path_to, path_from, /* verify_content */ false); writeString(path_from.string(), *write_buf); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 393fe21a4c3..35efe3d7b36 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -19,8 +19,6 @@ private: const std::string metadata_key_prefix; const std::string object_key_prefix; - bool write_finalized = false; - public: MetadataStorageFromPlainObjectStorageCreateDirectoryOperation( /// path_ must end with a trailing '/'. From e192d6c558afffa93bfb7bf8f53ad9c6794adacc Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Nov 2024 09:33:29 +0000 Subject: [PATCH 211/433] better --- ...torageFromPlainObjectStorageOperations.cpp | 22 +++++++++---------- ...aStorageFromPlainObjectStorageOperations.h | 2 +- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp index 8ed3d78895b..0e66975f239 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.cpp @@ -252,21 +252,29 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std: if (path_map.removePathIfExists(base_path)) { - removed = true; - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; CurrentMetrics::sub(metric, 1); auto event = object_storage->getMetadataStorageMetrics().directory_removed; ProfileEvents::increment(event); } + + remove_attempted = true; } void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::unique_lock &) { - if (!removed) + if (!remove_attempted) return; + { + std::lock_guard lock(path_map.mutex); + auto & map = path_map.map; + map.emplace(path.parent_path(), key_prefix); + } + auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; + CurrentMetrics::add(metric, 1); + auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); auto metadata_object = StoredObject(metadata_object_key.serialize(), path / PREFIX_PATH_FILE_NAME); auto buf = object_storage->writeObject( @@ -277,14 +285,6 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un /* settings */ {}); writeString(path.string(), *buf); buf->finalize(); - - { - std::lock_guard lock(path_map.mutex); - auto & map = path_map.map; - map.emplace(path.parent_path(), std::move(key_prefix)); - } - auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; - CurrentMetrics::add(metric, 1); } MetadataStorageFromPlainObjectStorageWriteFileOperation::MetadataStorageFromPlainObjectStorageWriteFileOperation( diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h index 35efe3d7b36..2b4e19bd639 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorageOperations.h @@ -69,7 +69,7 @@ private: const std::string metadata_key_prefix; std::string key_prefix; - bool removed = false; + bool remove_attempted = false; public: MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation( From 559ea73045e887a6f8380eaee4bc23ecd2d1905f Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Nov 2024 21:37:08 +0000 Subject: [PATCH 212/433] fix flaky test --- tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh index 89ed24e2b43..24297e69d31 100755 --- a/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh +++ b/tests/queries/0_stateless/03008_s3_plain_rewritable_fault.sh @@ -58,4 +58,5 @@ ${CLICKHOUSE_CLIENT} --query " SYSTEM DISABLE FAILPOINT plain_object_storage_write_fail_on_directory_move; " -${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault" +# Filter out 'Removing temporary directory' because the fault injection prevents directory rename. +${CLICKHOUSE_CLIENT} --query "DROP TABLE test_s3_mt_fault SYNC" 2>&1 | grep -v 'Removing temporary directory' ||: From 989c85ef52cadf5c1ddafa55ff76c0353438af0f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 10:12:11 +0100 Subject: [PATCH 213/433] De-virtualize compareAt and getUInt calls in comparator --- src/Columns/ColumnLowCardinality.cpp | 68 +++++++++++++++++++++++++--- src/Columns/IColumnImpl.h | 22 +++++++-- 2 files changed, 79 insertions(+), 11 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 8f94416459c..ec82f4249bd 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -426,14 +426,24 @@ void ColumnLowCardinality::getPermutation(IColumn::PermutationSortDirection dire getPermutationImpl(direction, stability, limit, nan_direction_hint, res); } -void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +namespace +{ + +template +void updatePermutationWithTypedColumns( + const ColumnLowCardinality & column, + IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) { bool ascending = direction == IColumn::PermutationSortDirection::Ascending; - auto comparator = [this, ascending, stability, nan_direction_hint](size_t lhs, size_t rhs) + /// Cast indexes and dictionary columns to their real types so that compareAt and getUInt methods can be inlined. + const IndexColumn & indexes = assert_cast(column.getIndexes()); + const DictinaryColumn & dictionary = assert_cast(*column.getDictionary().getNestedColumn()); + + auto comparator = [&indexes, &dictionary, ascending, stability, nan_direction_hint](size_t lhs, size_t rhs) { - int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint); + int ret = dictionary.compareAt(indexes.getUInt(lhs), indexes.getUInt(rhs), dictionary, nan_direction_hint); if (unlikely(stability == IColumn::PermutationSortStability::Stable && ret == 0)) return lhs < rhs; @@ -442,13 +452,57 @@ void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection d return ret > 0; }; - auto equal_comparator = [this, nan_direction_hint](size_t lhs, size_t rhs) + auto equal_comparator = [&indexes, &dictionary, nan_direction_hint](size_t lhs, size_t rhs) { - int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint); + int ret = dictionary.compareAt(indexes.getUInt(lhs), indexes.getUInt(rhs), dictionary, nan_direction_hint); return ret == 0; }; - updatePermutationImpl(limit, res, equal_ranges, comparator, equal_comparator, DefaultSort(), DefaultPartialSort()); + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, comparator, equal_comparator, DefaultSort(), DefaultPartialSort()); +} + +template +void updatePermutationWithIndexType( + const ColumnLowCardinality & column, + IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) +{ + /// Dispatch by dictionary column type. + if (typeid_cast(column.getDictionary().getNestedColumn().get())) + { + updatePermutationWithTypedColumns(column, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + } + else + { + /// Use default implementation for other types. + updatePermutationWithTypedColumns(column, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + } +} + +} + +void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + /// Dispatch by index column type. + switch (idx.getSizeOfIndexType()) + { + case sizeof(UInt8): + updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + case sizeof(UInt16): + updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + case sizeof(UInt32): + updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + case sizeof(UInt64): + updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + return; + default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of index type for low cardinality column."); + } } void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator, IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 80c08f51346..c50338cc5da 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -127,19 +127,20 @@ void IColumn::getPermutationImpl( } template -void IColumn::updatePermutationImpl( +void updateColumnPermutationImpl( size_t limit, - Permutation & res, + size_t size, + IColumn::Permutation & res, EqualRanges & equal_ranges, Compare compare, Equals equals, Sort full_sort, - PartialSort partial_sort) const + PartialSort partial_sort) { if (equal_ranges.empty()) return; - if (limit >= size() || limit > equal_ranges.back().to) + if (limit >= size || limit > equal_ranges.back().to) limit = 0; EqualRanges new_ranges; @@ -210,4 +211,17 @@ void IColumn::updatePermutationImpl( equal_ranges = std::move(new_ranges); } +template +void IColumn::updatePermutationImpl( + size_t limit, + Permutation & res, + EqualRanges & equal_ranges, + Compare compare, + Equals equals, + Sort full_sort, + PartialSort partial_sort) const +{ + updateColumnPermutationImpl(limit, size(), res, equal_ranges, compare, equals, full_sort, partial_sort); +} + } From 19556968bc5b440310ee63094b91cdf426baa4cc Mon Sep 17 00:00:00 2001 From: Zaynulla <36727185+Zaynulla@users.noreply.github.com> Date: Sat, 23 Nov 2024 14:43:02 +0300 Subject: [PATCH 214/433] window-functions doc typos fix - A forgotten closing parenthesis ] has been added. - The alias "teamMax" is less confusing for max (salary) than "teamAvg". This is probably the result of copying & pasting from the previous part of the document. --- docs/en/sql-reference/window-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 27d4bd763c7..3f97bad558d 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -47,7 +47,7 @@ aggregate_function (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name -WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) +WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]]) ``` - `PARTITION BY` - defines how to break a resultset into groups. @@ -168,7 +168,7 @@ Compare each player's salary to the maximum for their team. ```sql SELECT player, salary, team, - max(salary) OVER (PARTITION BY team) AS teamAvg, + max(salary) OVER (PARTITION BY team) AS teamMax, salary - teamAvg AS diff FROM salaries; ``` From a64ed742973e8544eb512b9c6f142330e3d65bdb Mon Sep 17 00:00:00 2001 From: Zaynulla <36727185+Zaynulla@users.noreply.github.com> Date: Sat, 23 Nov 2024 15:52:48 +0300 Subject: [PATCH 215/433] varPop doc latex formula added - There is a latex formula in the documentation of the [covarPop](https://clickhouse.com/docs/en/sql-reference/aggregate-functions/reference/covarpop) method, which is quite similar to varPop method. - In Russian docs there are formulas both for [varPop](https://clickhouse.com/docs/ru/sql-reference/aggregate-functions/reference/varpop), and [covarPop](https://clickhouse.com/docs/ru/sql-reference/aggregate-functions/reference/covarpop). Therefore, for consistency, it is suggested to add formula here too. --- .../sql-reference/aggregate-functions/reference/varpop.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index 182e830f19f..2b275540863 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -6,7 +6,11 @@ sidebar_position: 210 ## varPop -Calculates the population variance. +Calculates the population variance: + +$$ +\frac{\Sigma{(x - \bar{x})^2}}{n} +$$ **Syntax** From 18a881437d5dc7b80ff86a4157faa21c53f68c69 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 15:40:06 +0100 Subject: [PATCH 216/433] Use sorted dictionary to further speed up the comparison --- src/Columns/ColumnLowCardinality.cpp | 76 +++++++++++++++++----------- 1 file changed, 46 insertions(+), 30 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index ec82f4249bd..3d767cbbf45 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -429,37 +429,34 @@ void ColumnLowCardinality::getPermutation(IColumn::PermutationSortDirection dire namespace { -template -void updatePermutationWithTypedColumns( - const ColumnLowCardinality & column, - IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) +/// Comapator for sorting LowCardinality column with the help of sorted dictionary. +template +struct LowCardinalityComparator { - bool ascending = direction == IColumn::PermutationSortDirection::Ascending; + const IndexColumn & real_indexes; /// Indexes column + const PaddedPODArray & position_by_index; /// Maps original dictionary index to position in sorted dictionary - /// Cast indexes and dictionary columns to their real types so that compareAt and getUInt methods can be inlined. - const IndexColumn & indexes = assert_cast(column.getIndexes()); - const DictinaryColumn & dictionary = assert_cast(*column.getDictionary().getNestedColumn()); - - auto comparator = [&indexes, &dictionary, ascending, stability, nan_direction_hint](size_t lhs, size_t rhs) + inline bool operator () (size_t lhs, size_t rhs) const { - int ret = dictionary.compareAt(indexes.getUInt(lhs), indexes.getUInt(rhs), dictionary, nan_direction_hint); - if (unlikely(stability == IColumn::PermutationSortStability::Stable && ret == 0)) + int ret; + + const UInt64 lhs_index = real_indexes.getUInt(lhs); + const UInt64 rhs_index = real_indexes.getUInt(rhs); + + if (lhs_index == rhs_index) + ret = 0; + else + ret = CompareHelper::compare(position_by_index[lhs_index], position_by_index[rhs_index], 0); + + if (stable && ret == 0) return lhs < rhs; if (ascending) return ret < 0; + return ret > 0; - }; - - auto equal_comparator = [&indexes, &dictionary, nan_direction_hint](size_t lhs, size_t rhs) - { - int ret = dictionary.compareAt(indexes.getUInt(lhs), indexes.getUInt(rhs), dictionary, nan_direction_hint); - return ret == 0; - }; - - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, comparator, equal_comparator, DefaultSort(), DefaultPartialSort()); -} + } +}; template void updatePermutationWithIndexType( @@ -467,17 +464,36 @@ void updatePermutationWithIndexType( IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) { - /// Dispatch by dictionary column type. - if (typeid_cast(column.getDictionary().getNestedColumn().get())) + /// Cast indexes column to the real type so that compareAt and getUInt methods can be inlined. + const IndexColumn * real_indexes = assert_cast(&column.getIndexes()); + + IColumn::Permutation dict_perm; + column.getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); + + PaddedPODArray position_by_index(dict_perm.size()); + for (size_t i = 0; i < dict_perm.size(); ++i) + position_by_index[dict_perm[i]] = i; + + auto equal_comparator = [real_indexes](size_t lhs, size_t rhs) { - updatePermutationWithTypedColumns(column, direction, stability, limit, nan_direction_hint, res, equal_ranges); - return; + return real_indexes->getUInt(lhs) == real_indexes->getUInt(rhs); + }; + + const bool ascending = (direction == IColumn::PermutationSortDirection::Ascending); + const bool stable = (stability == IColumn::PermutationSortStability::Stable); + if (ascending) + { + if (stable) + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); + else + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); } else { - /// Use default implementation for other types. - updatePermutationWithTypedColumns(column, direction, stability, limit, nan_direction_hint, res, equal_ranges); - return; + if (stable) + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); + else + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); } } From ce82c19c2c4c954f5685453bd48b89f1a17ef6d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 18:38:10 +0100 Subject: [PATCH 217/433] Add some random final modifier to some classes --- src/Processors/Transforms/AggregatingTransform.cpp | 10 +++++----- src/Processors/Transforms/AggregatingTransform.h | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 21eec6c305a..332c7aab40b 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -51,7 +51,7 @@ namespace } /// Reads chunks from file in native format. Provide chunks with aggregation info. - class SourceFromNativeStream : public ISource + class SourceFromNativeStream final : public ISource { public: explicit SourceFromNativeStream(const Block & header, TemporaryBlockStreamReaderHolder tmp_stream_) @@ -84,7 +84,7 @@ namespace /// Worker which merges buckets for two-level aggregation. /// Atomically increments bucket counter and returns merged result. -class ConvertingAggregatedToChunksWithMergingSource : public ISource +class ConvertingAggregatedToChunksWithMergingSource final : public ISource { public: static constexpr UInt32 NUM_BUCKETS = 256; @@ -143,7 +143,7 @@ private: }; /// Asks Aggregator to convert accumulated aggregation state into blocks (without merging) and pushes them to later steps. -class ConvertingAggregatedToChunksSource : public ISource +class ConvertingAggregatedToChunksSource final : public ISource { public: ConvertingAggregatedToChunksSource(AggregatingTransformParamsPtr params_, AggregatedDataVariantsPtr variant_) @@ -188,7 +188,7 @@ private: }; /// Reads chunks from GroupingAggregatedTransform (stored in ChunksToMerge structure) and outputs them. -class FlattenChunksToMergeTransform : public IProcessor +class FlattenChunksToMergeTransform final : public IProcessor { public: explicit FlattenChunksToMergeTransform(const Block & input_header, const Block & output_header) @@ -272,7 +272,7 @@ private: /// ConvertingAggregatedToChunksWithMergingSource -> /// /// Result chunks guaranteed to be sorted by bucket number. -class ConvertingAggregatedToChunksTransform : public IProcessor +class ConvertingAggregatedToChunksTransform final : public IProcessor { public: ConvertingAggregatedToChunksTransform(AggregatingTransformParamsPtr params_, ManyAggregatedDataVariantsPtr data_, size_t num_threads_) diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index d1f4290a5b2..bf423398811 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -22,7 +22,7 @@ namespace CurrentMetrics namespace DB { -class AggregatedChunkInfo : public ChunkInfoCloneable +class AggregatedChunkInfo final : public ChunkInfoCloneable { public: bool is_overflows = false; @@ -149,7 +149,7 @@ using ManyAggregatedDataPtr = std::shared_ptr; * At aggregation step, every transform uses it's own AggregatedDataVariants structure. * At merging step, all structures pass to ConvertingAggregatedToChunksTransform. */ -class AggregatingTransform : public IProcessor +class AggregatingTransform final : public IProcessor { public: AggregatingTransform(Block header, AggregatingTransformParamsPtr params_); From 7e730bba9eacb9eb9646368427bd71adfb50d660 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 18:56:54 +0100 Subject: [PATCH 218/433] Fix: dictionary is already sorted in requested direction --- src/Columns/ColumnLowCardinality.cpp | 25 ++++++------------------- 1 file changed, 6 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 3d767cbbf45..f038f682f36 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -430,7 +430,8 @@ namespace { /// Comapator for sorting LowCardinality column with the help of sorted dictionary. -template +/// NOTE: Dictionary itself must be sorted in ASC or DESC order depending on the requested direction. +template struct LowCardinalityComparator { const IndexColumn & real_indexes; /// Indexes column @@ -451,10 +452,7 @@ struct LowCardinalityComparator if (stable && ret == 0) return lhs < rhs; - if (ascending) - return ret < 0; - - return ret > 0; + return ret < 0; } }; @@ -479,22 +477,11 @@ void updatePermutationWithIndexType( return real_indexes->getUInt(lhs) == real_indexes->getUInt(rhs); }; - const bool ascending = (direction == IColumn::PermutationSortDirection::Ascending); const bool stable = (stability == IColumn::PermutationSortStability::Stable); - if (ascending) - { - if (stable) - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); - else - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); - } + if (stable) + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); else - { - if (stable) - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); - else - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); - } + updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); } } From 9c0683e15786727b343d58a6fbd83464f4f9f7e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 19:16:25 +0100 Subject: [PATCH 219/433] Allow UNION in matviews --- .../Transforms/buildPushingToViewsChain.cpp | 4 +- src/Storages/SelectQueryDescription.cpp | 66 +++++---------- .../03275_matview_with_union.reference | 6 ++ .../0_stateless/03275_matview_with_union.sql | 82 +++++++++++++++++++ 4 files changed, 111 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/03275_matview_with_union.reference create mode 100644 tests/queries/0_stateless/03275_matview_with_union.sql diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 9d914149b4a..a028a79c917 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -148,7 +148,7 @@ private: struct State { QueryPipeline pipeline; - PullingPipelineExecutor executor; + PullingAsyncPipelineExecutor executor; explicit State(QueryPipeline pipeline_) : pipeline(std::move(pipeline_)) diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index 6ec4f86a2a1..b21711965e0 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -64,8 +64,6 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "StorageMaterializedView cannot be created from table functions ({})", serializeAST(*subquery)); - if (ast_select->list_of_selects->children.size() != 1) - throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW"); auto & inner_query = ast_select->list_of_selects->children.at(0); @@ -75,69 +73,47 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt } -void checkAllowedQueries(const ASTSelectQuery & query) +void checkAllowedQueries(const ASTSelectWithUnionQuery & select) { - if (query.prewhere() || query.final() || query.sampleSize()) - throw Exception(DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL."); - - ASTPtr subquery = extractTableExpression(query, 0); - if (!subquery) - return; - - if (const auto * ast_select = subquery->as()) + for (const auto & children : select.list_of_selects->children) { - if (ast_select->list_of_selects->children.size() != 1) - throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW"); + auto * query = children->as(); - const auto & inner_query = ast_select->list_of_selects->children.at(0); + if (query->prewhere() || query->final() || query->sampleSize()) + throw Exception(DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL."); - checkAllowedQueries(inner_query->as()); + ASTPtr subquery = extractTableExpression(*query, 0); + if (!subquery) + return; + + if (const auto * ast_select_with_union = subquery->as()) + { + checkAllowedQueries(*ast_select_with_union); + } } } } -/// check if only one single select query in SelectWithUnionQuery -static bool isSingleSelect(const ASTPtr & select, ASTPtr & res) -{ - auto * new_select = select->as(); - if (new_select == nullptr) - return false; - - if (new_select->list_of_selects->children.size() != 1) - return false; - - auto & new_inner_query = new_select->list_of_selects->children.at(0); - if (new_inner_query->as()) - { - res = new_inner_query; - return true; - } - - return isSingleSelect(new_inner_query, res); -} - SelectQueryDescription SelectQueryDescription::getSelectQueryFromASTForMatView(const ASTPtr & select, bool refreshable, ContextPtr context) { SelectQueryDescription result; result.select_query = select->as().clone(); + ASTSelectWithUnionQuery & query = result.select_query->as(); + /// Skip all the checks, none of them apply to refreshable views. /// Don't assign select_table_id. This way no materialized view dependency gets registered, /// so data doesn't get pushed to the refreshable view on source table inserts. if (refreshable) return result; - ASTPtr new_inner_query; - - if (!isSingleSelect(select, new_inner_query)) - throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "UNION is not supported for MATERIALIZED VIEW"); - - auto & select_query = new_inner_query->as(); - checkAllowedQueries(select_query); - - result.select_table_id = extractDependentTableFromSelectQuery(select_query, context); - result.inner_query = new_inner_query->clone(); + checkAllowedQueries(query); + /// We trigger only for the first found table + ASTSelectQuery & new_inner_query = query.list_of_selects->children.at(0)->as(); + /// Extracting first found table ID + result.select_table_id = extractDependentTableFromSelectQuery(new_inner_query, context); + result.inner_query = new_inner_query.clone(); return result; } diff --git a/tests/queries/0_stateless/03275_matview_with_union.reference b/tests/queries/0_stateless/03275_matview_with_union.reference new file mode 100644 index 00000000000..b5849ce77a4 --- /dev/null +++ b/tests/queries/0_stateless/03275_matview_with_union.reference @@ -0,0 +1,6 @@ +4000 +10 0 +10 1 +10 2 +10 3 +10 4 diff --git a/tests/queries/0_stateless/03275_matview_with_union.sql b/tests/queries/0_stateless/03275_matview_with_union.sql new file mode 100644 index 00000000000..caf4a6ac5fc --- /dev/null +++ b/tests/queries/0_stateless/03275_matview_with_union.sql @@ -0,0 +1,82 @@ +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS matview; + +CREATE TABLE src ( + event_time DateTime, + key UInt64, + value Int64 +) +ENGINE = MergeTree() +ORDER BY (event_time, key); + +CREATE TABLE dst ( + step UInt16, + rounded_event_time DateTime, + key UInt64, + value AggregateFunction(max, Int64) +) +ENGINE = AggregatingMergeTree() +ORDER BY (step, rounded_event_time, key); + +CREATE MATERIALIZED VIEW matview TO dst +( + step UInt16, + rounded_event_time DateTime, + key UInt64, + value AggregateFunction(max, Int64) +) AS +SELECT * FROM ( + SELECT + 1 AS step, + key, + intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time, + initializeAggregation('maxState', value) AS value + FROM src + ORDER BY + rounded_event_time, + key + UNION ALL + SELECT + 5 AS step, + key, + intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time, + initializeAggregation('maxState', value) AS value + FROM src + ORDER BY + rounded_event_time, + key + UNION ALL + SELECT + 15 AS step, + key, + intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time, + initializeAggregation('maxState', value) AS value + FROM src + ORDER BY + rounded_event_time, + key + UNION ALL + SELECT + 30 AS step, + key, + intDiv(toUnixTimestamp(event_time), step) * step AS rounded_event_time, + initializeAggregation('maxState', value) AS value + FROM src + ORDER BY + rounded_event_time, + key +) +ORDER BY step, rounded_event_time, key SETTINGS query_plan_remove_redundant_sorting = 0; + +set optimize_on_insert = 1; + +INSERT INTO src SELECT toDateTime('2020-10-01 00:00:00') + number, number % 100, number from numbers(1000); + +SELECT count() FROM dst; + +SELECT count(), key FROM dst WHERE step = 30 group by key ORDER BY key LIMIT 5; + +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS matview; From ee6ff8bbd6885d590e810a9672cb813b5470189e Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 19:48:35 +0100 Subject: [PATCH 220/433] Speedup partition key detection if block has only one partition --- .../MergeTree/MergeTreeDataWriter.cpp | 87 ++++++++++--------- 1 file changed, 46 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 5e78ab49010..059175d90bd 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -298,62 +298,67 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( ColumnRawPtrs partition_columns; partition_columns.reserve(partition_key_names_and_types.size()); + bool all_partition_columns_are_equal = true; for (const auto & element : partition_key_names_and_types) + { partition_columns.emplace_back(block_copy.getByName(element.name).column.get()); - - PODArray partition_num_to_first_row; - IColumn::Selector selector; - buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts, context); - - auto async_insert_info_with_partition = scatterAsyncInsertInfoBySelector(async_insert_info, selector, partition_num_to_first_row.size()); - - size_t partitions_count = partition_num_to_first_row.size(); - result.reserve(partitions_count); - - auto get_partition = [&](size_t num) + if (!partition_columns.back()->hasEqualValues()) + all_partition_columns_are_equal = false; + } + auto get_partition = [&](size_t row_num) { Row partition(partition_columns.size()); for (size_t i = 0; i < partition_columns.size(); ++i) - partition[i] = (*partition_columns[i])[partition_num_to_first_row[num]]; + partition[i] = (*partition_columns[i])[row_num]; return partition; }; - if (partitions_count == 1) + if (!all_partition_columns_are_equal) + { + PODArray partition_num_to_first_row; + IColumn::Selector selector; + buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts, context); + + auto async_insert_info_with_partition = scatterAsyncInsertInfoBySelector(async_insert_info, selector, partition_num_to_first_row.size()); + + size_t partitions_count = partition_num_to_first_row.size(); + result.reserve(partitions_count); + + for (size_t i = 0; i < partitions_count; ++i) + result.emplace_back(block.cloneEmpty(), get_partition(i)); + + for (size_t col = 0; col < block.columns(); ++col) + { + MutableColumns scattered = block.getByPosition(col).column->scatter(partitions_count, selector); + for (size_t i = 0; i < partitions_count; ++i) + result[i].block.getByPosition(col).column = std::move(scattered[i]); + } + + for (size_t i = 0; i < async_insert_info_with_partition.size(); ++i) + { + if (async_insert_info_with_partition[i] == nullptr) + { + LOG_ERROR( + getLogger("MergeTreeDataWriter"), + "The {}th element in async_insert_info_with_partition is nullptr. There are totally {} partitions in the insert. Selector content is ({}). Offsets content is ({})", + i, partitions_count, fmt::join(selector.begin(), selector.end(), ","), fmt::join(async_insert_info->offsets.begin(), async_insert_info->offsets.end(), ",")); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error for async deduplicated insert, please check error logs"); + } + result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); + result[i].tokens = std::move(async_insert_info_with_partition[i]->tokens); + } + } + else { /// A typical case is when there is one partition (you do not need to split anything). /// NOTE: returning a copy of the original block so that calculated partition key columns /// do not interfere with possible calculated primary key columns of the same name. result.emplace_back(Block(block), get_partition(0)); - if (!async_insert_info_with_partition.empty()) + if (async_insert_info != nullptr) { - result[0].offsets = std::move(async_insert_info_with_partition[0]->offsets); - result[0].tokens = std::move(async_insert_info_with_partition[0]->tokens); + result[0].offsets = std::move(async_insert_info->offsets); + result[0].tokens = std::move(async_insert_info->tokens); } - return result; - } - - for (size_t i = 0; i < partitions_count; ++i) - result.emplace_back(block.cloneEmpty(), get_partition(i)); - - for (size_t col = 0; col < block.columns(); ++col) - { - MutableColumns scattered = block.getByPosition(col).column->scatter(partitions_count, selector); - for (size_t i = 0; i < partitions_count; ++i) - result[i].block.getByPosition(col).column = std::move(scattered[i]); - } - - for (size_t i = 0; i < async_insert_info_with_partition.size(); ++i) - { - if (async_insert_info_with_partition[i] == nullptr) - { - LOG_ERROR( - getLogger("MergeTreeDataWriter"), - "The {}th element in async_insert_info_with_partition is nullptr. There are totally {} partitions in the insert. Selector content is ({}). Offsets content is ({})", - i, partitions_count, fmt::join(selector.begin(), selector.end(), ","), fmt::join(async_insert_info->offsets.begin(), async_insert_info->offsets.end(), ",")); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error for async deduplicated insert, please check error logs"); - } - result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); - result[i].tokens = std::move(async_insert_info_with_partition[i]->tokens); } return result; From f14f794e7dcf31225b079a39cd6a42edebb86125 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 20:32:37 +0100 Subject: [PATCH 221/433] Undo changes to IColumnImpl.h --- src/Columns/ColumnLowCardinality.cpp | 27 +++++++++++++-------------- src/Columns/ColumnLowCardinality.h | 5 +++++ src/Columns/IColumnImpl.h | 20 +++----------------- 3 files changed, 21 insertions(+), 31 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index f038f682f36..cbc02570381 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -429,7 +429,7 @@ void ColumnLowCardinality::getPermutation(IColumn::PermutationSortDirection dire namespace { -/// Comapator for sorting LowCardinality column with the help of sorted dictionary. +/// Compator for sorting LowCardinality column with the help of sorted dictionary. /// NOTE: Dictionary itself must be sorted in ASC or DESC order depending on the requested direction. template struct LowCardinalityComparator @@ -456,17 +456,18 @@ struct LowCardinalityComparator } }; +} + template -void updatePermutationWithIndexType( - const ColumnLowCardinality & column, +void ColumnLowCardinality::updatePermutationWithIndexType( IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { /// Cast indexes column to the real type so that compareAt and getUInt methods can be inlined. - const IndexColumn * real_indexes = assert_cast(&column.getIndexes()); + const IndexColumn * real_indexes = assert_cast(&getIndexes()); IColumn::Permutation dict_perm; - column.getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); + getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); PaddedPODArray position_by_index(dict_perm.size()); for (size_t i = 0; i < dict_perm.size(); ++i) @@ -479,11 +480,9 @@ void updatePermutationWithIndexType( const bool stable = (stability == IColumn::PermutationSortStability::Stable); if (stable) - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); + updatePermutationImpl(limit, res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); else - updateColumnPermutationImpl(limit, column.size(), res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); -} - + updatePermutationImpl(limit, res, equal_ranges, LowCardinalityComparator{*real_indexes, position_by_index}, equal_comparator, DefaultSort(), DefaultPartialSort()); } void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, @@ -493,16 +492,16 @@ void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection d switch (idx.getSizeOfIndexType()) { case sizeof(UInt8): - updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); return; case sizeof(UInt16): - updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); return; case sizeof(UInt32): - updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); return; case sizeof(UInt64): - updatePermutationWithIndexType(*this, direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); return; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of index type for low cardinality column."); } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 3cc1c8919c0..7380b5eff07 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -389,6 +389,11 @@ private: int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; void getPermutationImpl(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + template + void updatePermutationWithIndexType( + IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, + size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const; }; bool isColumnLowCardinalityNullable(const IColumn & column); diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index c50338cc5da..4515e7a48e8 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -127,20 +127,19 @@ void IColumn::getPermutationImpl( } template -void updateColumnPermutationImpl( +void IColumn::updatePermutationImpl( size_t limit, - size_t size, IColumn::Permutation & res, EqualRanges & equal_ranges, Compare compare, Equals equals, Sort full_sort, - PartialSort partial_sort) + PartialSort partial_sort) const { if (equal_ranges.empty()) return; - if (limit >= size || limit > equal_ranges.back().to) + if (limit >= size() || limit > equal_ranges.back().to) limit = 0; EqualRanges new_ranges; @@ -211,17 +210,4 @@ void updateColumnPermutationImpl( equal_ranges = std::move(new_ranges); } -template -void IColumn::updatePermutationImpl( - size_t limit, - Permutation & res, - EqualRanges & equal_ranges, - Compare compare, - Equals equals, - Sort full_sort, - PartialSort partial_sort) const -{ - updateColumnPermutationImpl(limit, size(), res, equal_ranges, compare, equals, full_sort, partial_sort); -} - } From 833b137390e388571c5b8847a4f04d628b05e233 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 20:33:43 +0100 Subject: [PATCH 222/433] Undo changes to IColumnImpl.h --- src/Columns/IColumnImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 4515e7a48e8..80c08f51346 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -129,7 +129,7 @@ void IColumn::getPermutationImpl( template void IColumn::updatePermutationImpl( size_t limit, - IColumn::Permutation & res, + Permutation & res, EqualRanges & equal_ranges, Compare compare, Equals equals, From 8d8d1e85030f751fdd3fe45db840413ca3ece484 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sat, 23 Nov 2024 20:37:14 +0100 Subject: [PATCH 223/433] typo --- src/Columns/ColumnLowCardinality.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index cbc02570381..2317cca01fa 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -429,7 +429,7 @@ void ColumnLowCardinality::getPermutation(IColumn::PermutationSortDirection dire namespace { -/// Compator for sorting LowCardinality column with the help of sorted dictionary. +/// Comparator for sorting LowCardinality column with the help of sorted dictionary. /// NOTE: Dictionary itself must be sorted in ASC or DESC order depending on the requested direction. template struct LowCardinalityComparator From 40be220f8b74003ffbaa858dc521e03dcb497889 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 21:50:45 +0100 Subject: [PATCH 224/433] Update src/Storages/MergeTree/MergeTreeDataWriter.cpp --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 059175d90bd..eaf77136810 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -325,7 +325,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( result.reserve(partitions_count); for (size_t i = 0; i < partitions_count; ++i) - result.emplace_back(block.cloneEmpty(), get_partition(i)); + result.emplace_back(block.cloneEmpty(), get_partition(partition_num_to_first_row[i])); for (size_t col = 0; col < block.columns(); ++col) { From 7816346403539a62ed20ebd8cc2196100fc38146 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 23 Nov 2024 21:53:47 +0000 Subject: [PATCH 225/433] revert build limits --- CMakeLists.txt | 1 - docker/packager/binary-builder/build.sh | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 03c20325f1e..a165be799c0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -66,7 +66,6 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # Twice as large set (RLIMIT_DATA 10000000000) set (RLIMIT_AS 20000000000) - set (RLIMIT_CPU 2000) endif() # For some files currently building RISCV64/LOONGARCH64 might be too slow. diff --git a/docker/packager/binary-builder/build.sh b/docker/packager/binary-builder/build.sh index b38d204d123..f18a6c2a798 100755 --- a/docker/packager/binary-builder/build.sh +++ b/docker/packager/binary-builder/build.sh @@ -91,7 +91,7 @@ then fi # Build everything -cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=0 "${CMAKE_FLAGS_ARRAY[@]}" .. +cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS_ARRAY[@]}" .. # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. From 3d8378a0c5d037680d38cb2277cf429423ca689e Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 23:17:14 +0100 Subject: [PATCH 226/433] Optimize function argMin/Max for some data types --- .../AggregateFunctionsArgMinArgMax.cpp | 127 ++++++++++++++---- 1 file changed, 103 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index cc72a26af16..d44a5d13b98 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -13,18 +13,36 @@ struct Settings; namespace ErrorCodes { - -extern const int INCORRECT_DATA; -extern const int ILLEGAL_TYPE_OF_ARGUMENT; -extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; } namespace { -template +template struct AggregateFunctionArgMinMaxData { +private: + ResultType result_data; + ValueType value_data; + +public: + ResultType & result() { return result_data; } + const ResultType & result() const { return result_data; } + ValueType & value() { return value_data; } + const ValueType & value() const { return value_data; } + + static bool allocatesMemoryInArena(TypeIndex) + { + return ResultType::allocatesMemoryInArena() || ValueType::allocatesMemoryInArena(); + } +}; + +template +struct AggregateFunctionArgMinMaxDataGeneric +{ private: SingleValueDataBaseMemoryBlock result_data; ValueType value_data; @@ -35,27 +53,22 @@ public: ValueType & value() { return value_data; } const ValueType & value() const { return value_data; } - [[noreturn]] explicit AggregateFunctionArgMinMaxData() + static bool allocatesMemoryInArena(TypeIndex result_type_index) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionArgMinMaxData initialized empty"); + return singleValueTypeAllocatesMemoryInArena(result_type_index) || ValueType::allocatesMemoryInArena(); } - explicit AggregateFunctionArgMinMaxData(TypeIndex result_type) : value_data() - { - generateSingleValueFromTypeIndex(result_type, result_data); - } - - ~AggregateFunctionArgMinMaxData() { result().~SingleValueDataBase(); } + ~AggregateFunctionArgMinMaxDataGeneric() { result().~SingleValueDataBase(); } }; static_assert( - sizeof(AggregateFunctionArgMinMaxData) <= 2 * SingleValueDataBase::MAX_STORAGE_SIZE, + sizeof(AggregateFunctionArgMinMaxDataGeneric) <= 2 * SingleValueDataBase::MAX_STORAGE_SIZE, "Incorrect size of AggregateFunctionArgMinMaxData struct"); /// Returns the first arg value found for the minimum/maximum value. Example: argMin(arg, value). -template +template class AggregateFunctionArgMinMax final - : public IAggregateFunctionDataHelper, AggregateFunctionArgMinMax> + : public IAggregateFunctionDataHelper> { private: const DataTypePtr & type_val; @@ -63,7 +76,8 @@ private: const SerializationPtr serialization_val; const TypeIndex result_type_index; - using Base = IAggregateFunctionDataHelper, AggregateFunctionArgMinMax>; + + using Base = IAggregateFunctionDataHelper>; public: explicit AggregateFunctionArgMinMax(const DataTypes & argument_types_) @@ -91,7 +105,7 @@ public: void create(AggregateDataPtr __restrict place) const override /// NOLINT { - new (place) AggregateFunctionArgMinMaxData(result_type_index); + new (place) Data(); } String getName() const override @@ -215,7 +229,7 @@ public: bool allocatesMemoryInArena() const override { - return singleValueTypeAllocatesMemoryInArena(result_type_index) || ValueData::allocatesMemoryInArena(); + return Data::allocatesMemoryInArena(result_type_index); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -224,12 +238,77 @@ public: } }; -template -AggregateFunctionPtr createAggregateFunctionArgMinMax( - const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings) +using AllTypes = std::tuple; + + +template +IAggregateFunction * createForPair(const TypeIndex & result_type, const TypeIndex & value_type, const DataTypes & argument_types) { - return AggregateFunctionPtr(createAggregateFunctionSingleValue( - name, argument_types, parameters, settings)); + if (TypeToTypeIndex == result_type && value_type == TypeToTypeIndex) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + return nullptr; +} + +template +IAggregateFunction * tryValueTypes(const DataTypes & argument_types, const TypeIndex & result_type, const TypeIndex & value_type, std::tuple) +{ + IAggregateFunction * result = nullptr; + ((result = result ? result : createForPair(result_type, value_type, argument_types)), ...); // Fold expression + return result; +} + +template +IAggregateFunction * tryResultTypes(const DataTypes & argument_types, const TypeIndex result_idx, const TypeIndex value_idx, std::tuple, std::tuple value_tuple) +{ + IAggregateFunction * result = nullptr; + ((result = result ? result : tryValueTypes(argument_types, result_idx, value_idx, value_tuple)), ...); // Fold expression + return result; +} + +template +AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const DataTypes & argument_types, const Array &, const Settings *) { + using AllTypesTuple = AllTypes; + + const DataTypePtr & result_type = argument_types[0]; + const DataTypePtr & value_type = argument_types[1]; + + WhichDataType which_result(result_type); + WhichDataType which_value(value_type); + + auto convert_date_type = [] (TypeIndex type_index) + { + if (type_index == TypeIndex::Date) + return TypeToTypeIndex; + else if (type_index == TypeIndex::DateTime) + return TypeToTypeIndex; + else + return type_index; + }; + + AggregateFunctionPtr result = AggregateFunctionPtr(tryResultTypes(argument_types, convert_date_type(which_result.idx), convert_date_type(which_value.idx), AllTypesTuple{}, AllTypesTuple{})); + if (!result) + { + WhichDataType which(value_type); +#define DISPATCH(TYPE) \ + if (which_value.idx == TypeIndex::TYPE) \ + return AggregateFunctionPtr(new AggregateFunctionArgMinMax>, isMin>(argument_types)); /// NOLINT + FOR_SINGLE_VALUE_NUMERIC_TYPES(DISPATCH) +#undef DISPATCH + + if (which.idx == TypeIndex::Date) + return AggregateFunctionPtr(new AggregateFunctionArgMinMax>, isMin>(argument_types)); + if (which.idx == TypeIndex::DateTime) + return AggregateFunctionPtr(new AggregateFunctionArgMinMax>, isMin>(argument_types)); + if (which.idx == TypeIndex::String) + return AggregateFunctionPtr(new AggregateFunctionArgMinMax, isMin>(argument_types)); + + return AggregateFunctionPtr(new AggregateFunctionArgMinMax, isMin>(argument_types)); + } + return result; } } From 71b1ba5009233e14ba59fbd626a87f03a8aff24a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 23:23:45 +0100 Subject: [PATCH 227/433] Followup --- .../AggregateFunctionsArgMinArgMax.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index d44a5d13b98..1038030eca6 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -34,6 +34,9 @@ public: ValueType & value() { return value_data; } const ValueType & value() const { return value_data; } + AggregateFunctionArgMinMaxData() = default; + explicit AggregateFunctionArgMinMaxData(TypeIndex) {} + static bool allocatesMemoryInArena(TypeIndex) { return ResultType::allocatesMemoryInArena() || ValueType::allocatesMemoryInArena(); @@ -53,6 +56,16 @@ public: ValueType & value() { return value_data; } const ValueType & value() const { return value_data; } + [[noreturn]] AggregateFunctionArgMinMaxDataGeneric() + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionArgMinMaxData initialized empty"); + } + + explicit AggregateFunctionArgMinMaxDataGeneric(TypeIndex result_type) : value_data() + { + generateSingleValueFromTypeIndex(result_type, result_data); + } + static bool allocatesMemoryInArena(TypeIndex result_type_index) { return singleValueTypeAllocatesMemoryInArena(result_type_index) || ValueType::allocatesMemoryInArena(); @@ -105,7 +118,7 @@ public: void create(AggregateDataPtr __restrict place) const override /// NOLINT { - new (place) Data(); + new (place) Data(result_type_index); } String getName() const override From 02bacd71fd2b4cdd5d45edcc7b4052c630e7cc59 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 23:26:29 +0100 Subject: [PATCH 228/433] Style --- src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index 1038030eca6..afa654000b3 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -270,7 +270,7 @@ template IAggregateFunction * tryValueTypes(const DataTypes & argument_types, const TypeIndex & result_type, const TypeIndex & value_type, std::tuple) { IAggregateFunction * result = nullptr; - ((result = result ? result : createForPair(result_type, value_type, argument_types)), ...); // Fold expression + ((result = result ? result : createForPair(result_type, value_type, argument_types)), ...); return result; } @@ -278,12 +278,13 @@ template IAggregateFunction * tryResultTypes(const DataTypes & argument_types, const TypeIndex result_idx, const TypeIndex value_idx, std::tuple, std::tuple value_tuple) { IAggregateFunction * result = nullptr; - ((result = result ? result : tryValueTypes(argument_types, result_idx, value_idx, value_tuple)), ...); // Fold expression + ((result = result ? result : tryValueTypes(argument_types, result_idx, value_idx, value_tuple)), ...); return result; } template -AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const DataTypes & argument_types, const Array &, const Settings *) { +AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const DataTypes & argument_types, const Array &, const Settings *) +{ using AllTypesTuple = AllTypes; const DataTypePtr & result_type = argument_types[0]; From a9336afbc7419525b732938d4a6b832204fe663b Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 23 Nov 2024 23:53:59 +0100 Subject: [PATCH 229/433] Followup --- src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index afa654000b3..3045bc8c930 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -251,7 +251,8 @@ public: } }; -using AllTypes = std::tuple; @@ -285,7 +286,7 @@ IAggregateFunction * tryResultTypes(const DataTypes & argument_types, const Type template AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const DataTypes & argument_types, const Array &, const Settings *) { - using AllTypesTuple = AllTypes; + using TypesToCreateSpecializedDataTuple = TypesToCreateSpecializedData; const DataTypePtr & result_type = argument_types[0]; const DataTypePtr & value_type = argument_types[1]; @@ -303,7 +304,7 @@ AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const return type_index; }; - AggregateFunctionPtr result = AggregateFunctionPtr(tryResultTypes(argument_types, convert_date_type(which_result.idx), convert_date_type(which_value.idx), AllTypesTuple{}, AllTypesTuple{})); + AggregateFunctionPtr result = AggregateFunctionPtr(tryResultTypes(argument_types, convert_date_type(which_result.idx), convert_date_type(which_value.idx), TypesToCreateSpecializedDataTuple{}, TypesToCreateSpecializedDataTuple{})); if (!result) { WhichDataType which(value_type); From db10fcf9504889b2afc46375ddb30974817a1968 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 03:40:04 +0000 Subject: [PATCH 230/433] split FunctionsConversion --- src/Functions/FunctionsConversion.cpp | 5626 ------------------- src/Functions/FunctionsConversion.h | 5567 ++++++++++++++++++ src/Functions/FunctionsConversion_impl0.cpp | 47 + src/Functions/FunctionsConversion_impl1.cpp | 38 + src/Functions/FunctionsConversion_impl2.cpp | 38 + src/Functions/FunctionsConversion_impl3.cpp | 46 + src/Functions/FunctionsConversion_reg.cpp | 192 + 7 files changed, 5928 insertions(+), 5626 deletions(-) create mode 100644 src/Functions/FunctionsConversion.h create mode 100644 src/Functions/FunctionsConversion_impl0.cpp create mode 100644 src/Functions/FunctionsConversion_impl1.cpp create mode 100644 src/Functions/FunctionsConversion_impl2.cpp create mode 100644 src/Functions/FunctionsConversion_impl3.cpp create mode 100644 src/Functions/FunctionsConversion_reg.cpp diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index bbe87b600a9..e69de29bb2d 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -1,5626 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace Setting -{ - extern const SettingsBool cast_ipv4_ipv6_default_on_conversion_error; - extern const SettingsBool cast_string_to_dynamic_use_inference; - extern const SettingsDateTimeOverflowBehavior date_time_overflow_behavior; - extern const SettingsBool input_format_ipv4_default_on_conversion_error; - extern const SettingsBool input_format_ipv6_default_on_conversion_error; - extern const SettingsBool precise_float_parsing; - extern const SettingsBool date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands; -} - -namespace ErrorCodes -{ - extern const int ATTEMPT_TO_READ_AFTER_EOF; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_READ_ARRAY_FROM_TEXT; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_PARSE_TEXT; - extern const int CANNOT_PARSE_UUID; - extern const int CANNOT_PARSE_IPV4; - extern const int CANNOT_PARSE_IPV6; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; - extern const int LOGICAL_ERROR; - extern const int TYPE_MISMATCH; - extern const int CANNOT_CONVERT_TYPE; - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NOT_IMPLEMENTED; - extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; -} - -namespace -{ - -/** Type conversion functions. - * toType - conversion in "natural way"; - */ - -UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) -{ - const auto * arg_type = named_column.type.get(); - bool ok = checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type); - if (!ok) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of toDecimal() scale {}", named_column.type->getName()); - - Field field; - named_column.column->get(0, field); - return static_cast(field.safeGet()); -} - - -/** Conversion of Date to DateTime: adding 00:00:00 time component. - */ -template -struct ToDateTimeImpl -{ - static constexpr auto name = "toDateTime"; - - static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (d > MAX_DATETIME_DAY_NUM) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Day number {} is out of bounds of type DateTime", d); - } - else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) - { - d = std::min(d, MAX_DATETIME_DAY_NUM); - } - return static_cast(time_zone.fromDayNum(DayNum(d))); - } - - static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) - { - if (d < 0) - return 0; - else if (d > MAX_DATETIME_DAY_NUM) - d = MAX_DATETIME_DAY_NUM; - } - else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (d < 0 || d > MAX_DATETIME_DAY_NUM) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", d); - } - return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); - } - - static UInt32 execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) - { - return dt; - } - - static UInt32 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Ignore) - return static_cast(dt64); - else - { - if (dt64 < 0 || dt64 >= MAX_DATETIME_TIMESTAMP) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) - return dt64 < 0 ? 0 : std::numeric_limits::max(); - else - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); - } - else - return static_cast(dt64); - } - } -}; - - -/// Implementation of toDate function. - -template -struct ToDateTransform32Or64 -{ - static constexpr auto name = "toDate"; - - static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl & time_zone) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); - } - /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, - /// otherwise treat it as unix timestamp. This is a bit weird, but we leave this behavior. - if (from <= DATE_LUT_MAX_DAY_NUM) - return from; - else - return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); - } -}; - - -template -struct ToDateTransform32Or64Signed -{ - static constexpr auto name = "toDate"; - - static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl & time_zone) - { - // TODO: decide narrow or extended range based on FromType - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from < 0 || from > MAX_DATE_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); - } - else - { - if (from < 0) - return 0; - } - return (from <= DATE_LUT_MAX_DAY_NUM) - ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATE_TIMESTAMP))); - } -}; - -template -struct ToDateTransform8Or16Signed -{ - static constexpr auto name = "toDate"; - - static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl &) - { - if (from < 0) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); - else - return 0; - } - return from; - } -}; - -/// Implementation of toDate32 function. - -template -struct ToDate32Transform32Or64 -{ - static constexpr auto name = "toDate32"; - - static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone) - { - if (from < DATE_LUT_MAX_EXTEND_DAY_NUM) - { - return static_cast(from); - } - else - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); - } - return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME64_TIMESTAMP))); - } - } -}; - -template -struct ToDate32Transform32Or64Signed -{ - static constexpr auto name = "toDate32"; - - static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone) - { - static const Int32 daynum_min_offset = -static_cast(DateLUTImpl::getDayNumOffsetEpoch()); - - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from < daynum_min_offset || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); - } - - if (from < daynum_min_offset) - return daynum_min_offset; - - return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) - ? static_cast(from) - : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(MAX_DATETIME64_TIMESTAMP))); - } -}; - -template -struct ToDate32Transform8Or16Signed -{ - static constexpr auto name = "toDate32"; - - static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl &) - { - return from; - } -}; - -template -struct ToDateTimeTransform64 -{ - static constexpr auto name = "toDateTime"; - - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); - } - return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); - } -}; - -template -struct ToDateTimeTransformSigned -{ - static constexpr auto name = "toDateTime"; - - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) - { - if (from < 0) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); - else - return 0; - } - return from; - } -}; - -template -struct ToDateTimeTransform64Signed -{ - static constexpr auto name = "toDateTime"; - - static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from < 0 || from > MAX_DATETIME_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); - } - - if (from < 0) - return 0; - return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); - } -}; - -/** Conversion of numeric to DateTime64 - */ - -template -struct ToDateTime64TransformUnsigned -{ - static constexpr auto name = "toDateTime64"; - - const DateTime64::NativeType scale_multiplier; - - ToDateTime64TransformUnsigned(UInt32 scale) /// NOLINT - : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) - {} - - NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); - else - return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); - } - else - return DecimalUtils::decimalFromComponentsWithMultiplier(std::min(from, MAX_DATETIME64_TIMESTAMP), 0, scale_multiplier); - } -}; - -template -struct ToDateTime64TransformSigned -{ - static constexpr auto name = "toDateTime64"; - - const DateTime64::NativeType scale_multiplier; - - ToDateTime64TransformSigned(UInt32 scale) /// NOLINT - : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) - {} - - NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); - } - from = static_cast(std::max(from, MIN_DATETIME64_TIMESTAMP)); - from = static_cast(std::min(from, MAX_DATETIME64_TIMESTAMP)); - - return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); - } -}; - -template -struct ToDateTime64TransformFloat -{ - static constexpr auto name = "toDateTime64"; - - const UInt32 scale; - - ToDateTime64TransformFloat(UInt32 scale_) /// NOLINT - : scale(scale_) - {} - - NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const - { - if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) - { - if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] - throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); - } - - from = std::max(from, static_cast(MIN_DATETIME64_TIMESTAMP)); - from = std::min(from, static_cast(MAX_DATETIME64_TIMESTAMP)); - return convertToDecimal(from, scale); - } -}; - -struct ToDateTime64Transform -{ - static constexpr auto name = "toDateTime64"; - - const DateTime64::NativeType scale_multiplier; - - ToDateTime64Transform(UInt32 scale) /// NOLINT - : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) - {} - - DateTime64::NativeType execute(UInt16 d, const DateLUTImpl & time_zone) const - { - const auto dt = ToDateTimeImpl<>::execute(d, time_zone); - return execute(dt, time_zone); - } - - DateTime64::NativeType execute(Int32 d, const DateLUTImpl & time_zone) const - { - Int64 dt = static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); - return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); - } - - DateTime64::NativeType execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) const - { - return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); - } -}; - -/** Transformation of numbers, dates, datetimes to strings: through formatting. - */ -template -struct FormatImpl -{ - template - static ReturnType execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType *, const DateLUTImpl *) - { - writeText(x, wb); - return ReturnType(true); - } -}; - -template <> -struct FormatImpl -{ - template - static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl * time_zone) - { - writeDateText(DayNum(x), wb, *time_zone); - return ReturnType(true); - } -}; - -template <> -struct FormatImpl -{ - template - static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) - { - writeDateText(ExtendedDayNum(x), wb, *time_zone); - return ReturnType(true); - } -}; - -template <> -struct FormatImpl -{ - template - static ReturnType execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime *, const DateLUTImpl * time_zone) - { - writeDateTimeText(x, wb, *time_zone); - return ReturnType(true); - } -}; - -template <> -struct FormatImpl -{ - template - static ReturnType execute(const DataTypeDateTime64::FieldType x, WriteBuffer & wb, const DataTypeDateTime64 * type, const DateLUTImpl * time_zone) - { - writeDateTimeText(DateTime64(x), type->getScale(), wb, *time_zone); - return ReturnType(true); - } -}; - - -template -struct FormatImpl> -{ - template - static ReturnType execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum * type, const DateLUTImpl *) - { - static constexpr bool throw_exception = std::is_same_v; - - if constexpr (throw_exception) - { - writeString(type->getNameForValue(x), wb); - } - else - { - StringRef res; - bool is_ok = type->getNameForValue(x, res); - if (is_ok) - writeString(res, wb); - return ReturnType(is_ok); - } - } -}; - -template -struct FormatImpl> -{ - template - static ReturnType execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal * type, const DateLUTImpl *) - { - writeText(x, type->getScale(), wb, false); - return ReturnType(true); - } -}; - -ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col) -{ - ColumnUInt8::MutablePtr null_map = nullptr; - if (const auto * col_nullable = checkAndGetColumn(col.get())) - { - null_map = ColumnUInt8::create(); - null_map->insertRangeFrom(col_nullable->getNullMapColumn(), 0, col_nullable->size()); - } - return null_map; -} - - -/// Generic conversion of any type to String or FixedString via serialization to text. -template -struct ConvertImplGenericToString -{ - static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const ContextPtr & context) - { - static_assert(std::is_same_v || std::is_same_v, - "Can be used only to serialize to ColumnString or ColumnFixedString"); - - ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); - - const auto & col_with_type_and_name = columnGetNested(arguments[0]); - const IDataType & type = *col_with_type_and_name.type; - const IColumn & col_from = *col_with_type_and_name.column; - - size_t size = col_from.size(); - auto col_to = removeNullable(result_type)->createColumn(); - - { - ColumnStringHelpers::WriteHelper write_helper( - assert_cast(*col_to), - size); - - auto & write_buffer = write_helper.getWriteBuffer(); - - FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; - auto serialization = type.getDefaultSerialization(); - for (size_t row = 0; row < size; ++row) - { - serialization->serializeText(col_from, row, write_buffer, format_settings); - write_helper.rowWritten(); - } - - write_helper.finalize(); - } - - if (result_type->isNullable() && null_map) - return ColumnNullable::create(std::move(col_to), std::move(null_map)); - return col_to; - } -}; - -/** Conversion of time_t to UInt16, Int32, UInt32 - */ -template -void convertFromTime(typename DataType::FieldType & x, time_t & time) -{ - x = time; -} - -template <> -inline void convertFromTime(DataTypeDateTime::FieldType & x, time_t & time) -{ - if (unlikely(time < 0)) - x = 0; - else if (unlikely(time > MAX_DATETIME_TIMESTAMP)) - x = MAX_DATETIME_TIMESTAMP; - else - x = static_cast(time); -} - -/** Conversion of strings to numbers, dates, datetimes: through parsing. - */ -template -void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) -{ - if constexpr (is_floating_point) - { - if (precise_float_parsing) - readFloatTextPrecise(x, rb); - else - readFloatTextFast(x, rb); - } - else - readText(x, rb); -} - -template <> -inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - DayNum tmp(0); - readDateText(tmp, rb, *time_zone); - x = tmp; -} - -template <> -inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - ExtendedDayNum tmp(0); - readDateText(tmp, rb, *time_zone); - x = tmp; -} - - -// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code. -template <> -inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - time_t time = 0; - readDateTimeText(time, rb, *time_zone); - convertFromTime(x, time); -} - -template <> -inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - UUID tmp; - readUUIDText(tmp, rb); - x = tmp.toUnderType(); -} - -template <> -inline void parseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - IPv4 tmp; - readIPv4Text(tmp, rb); - x = tmp.toUnderType(); -} - -template <> -inline void parseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - IPv6 tmp; - readIPv6Text(tmp, rb); - x = tmp; -} - -template -bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) -{ - if constexpr (is_floating_point) - { - if (precise_float_parsing) - return tryReadFloatTextPrecise(x, rb); - else - return tryReadFloatTextFast(x, rb); - } - else /*if constexpr (is_integral_v)*/ - return tryReadIntText(x, rb); -} - -template <> -inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - DayNum tmp(0); - if (!tryReadDateText(tmp, rb, *time_zone)) - return false; - x = tmp; - return true; -} - -template <> -inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - ExtendedDayNum tmp(0); - if (!tryReadDateText(tmp, rb, *time_zone)) - return false; - x = tmp; - return true; -} - -template <> -inline bool tryParseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) -{ - time_t time = 0; - if (!tryReadDateTimeText(time, rb, *time_zone)) - return false; - convertFromTime(x, time); - return true; -} - -template <> -inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - UUID tmp; - if (!tryReadUUIDText(tmp, rb)) - return false; - - x = tmp.toUnderType(); - return true; -} - -template <> -inline bool tryParseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - IPv4 tmp; - if (!tryReadIPv4Text(tmp, rb)) - return false; - - x = tmp.toUnderType(); - return true; -} - -template <> -inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) -{ - IPv6 tmp; - if (!tryReadIPv6Text(tmp, rb)) - return false; - - x = tmp; - return true; -} - - -/** Throw exception with verbose message when string value is not parsed completely. - */ -[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, const IDataType & result_type) -{ - WriteBufferFromOwnString message_buf; - message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size()) - << " as " << result_type.getName() - << ": syntax error"; - - if (read_buffer.offset()) - message_buf << " at position " << read_buffer.offset() - << " (parsed just " << quote << String(read_buffer.buffer().begin(), read_buffer.offset()) << ")"; - else - message_buf << " at begin of string"; - - // Currently there are no functions toIPv{4,6}Or{Null,Zero} - if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6")) - message_buf << ". Note: there are to" << result_type.getName() << "OrZero and to" << result_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; - - throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}", {String(read_buffer.buffer().begin(), read_buffer.buffer().size()), result_type.getName()}}, ErrorCodes::CANNOT_PARSE_TEXT); -} - - -enum class ConvertFromStringExceptionMode : uint8_t -{ - Throw, /// Throw exception if value cannot be parsed. - Zero, /// Fill with zero or default if value cannot be parsed. - Null /// Return ColumnNullable with NULLs when value cannot be parsed. -}; - -enum class ConvertFromStringParsingMode : uint8_t -{ - Normal, - BestEffort, /// Only applicable for DateTime. Will use sophisticated method, that is slower. - BestEffortUS -}; - -struct AccurateConvertStrategyAdditions -{ - UInt32 scale { 0 }; -}; - -struct AccurateOrNullConvertStrategyAdditions -{ - UInt32 scale { 0 }; -}; - -template -struct ConvertThroughParsing -{ - static_assert(std::is_same_v || std::is_same_v, - "ConvertThroughParsing is only applicable for String or FixedString data types"); - - static constexpr bool to_datetime64 = std::is_same_v; - - static bool isAllRead(ReadBuffer & in) - { - /// In case of FixedString, skip zero bytes at end. - if constexpr (std::is_same_v) - while (!in.eof() && *in.position() == 0) - ++in.position(); - - if (in.eof()) - return true; - - /// Special case, that allows to parse string with DateTime or DateTime64 as Date or Date32. - if constexpr (std::is_same_v || std::is_same_v) - { - if (!in.eof() && (*in.position() == ' ' || *in.position() == 'T')) - { - if (in.buffer().size() == strlen("YYYY-MM-DD hh:mm:ss")) - return true; - - if (in.buffer().size() >= strlen("YYYY-MM-DD hh:mm:ss.x") - && in.buffer().begin()[19] == '.') - { - in.position() = in.buffer().begin() + 20; - - while (!in.eof() && isNumericASCII(*in.position())) - ++in.position(); - - if (in.eof()) - return true; - } - } - } - - return false; - } - - template - static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, size_t input_rows_count, - Additions additions [[maybe_unused]] = Additions()) - { - using ColVecTo = typename ToDataType::ColumnType; - - const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr; - const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr; - - /// For conversion to Date or DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || to_datetime64) - { - const auto result_type = removeNullable(res_type); - // Time zone is already figured out during result type resolution, no need to do it here. - if (const auto dt_col = checkAndGetDataType(result_type.get())) - local_time_zone = &dt_col->getTimeZone(); - else - local_time_zone = &extractTimeZoneFromFunctionArguments(arguments, 1, 0); - - if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS) - utc_time_zone = &DateLUT::instance("UTC"); - } - else if constexpr (std::is_same_v || std::is_same_v) - { - // Timezone is more or less dummy when parsing Date/Date32 from string. - local_time_zone = &DateLUT::instance(); - utc_time_zone = &DateLUT::instance("UTC"); - } - - const IColumn * col_from = arguments[0].column.get(); - const ColumnString * col_from_string = checkAndGetColumn(col_from); - const ColumnFixedString * col_from_fixed_string = checkAndGetColumn(col_from); - - if (std::is_same_v && !col_from_string) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - col_from->getName(), Name::name); - - if (std::is_same_v && !col_from_fixed_string) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - col_from->getName(), Name::name); - - size_t size = input_rows_count; - typename ColVecTo::MutablePtr col_to = nullptr; - - if constexpr (IsDataTypeDecimal) - { - UInt32 scale = additions; - if constexpr (to_datetime64) - { - ToDataType check_bounds_in_ctor(scale, local_time_zone ? local_time_zone->getTimeZone() : String{}); - } - else - { - ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale); - } - col_to = ColVecTo::create(size, scale); - } - else - col_to = ColVecTo::create(size); - - typename ColVecTo::Container & vec_to = col_to->getData(); - - ColumnUInt8::MutablePtr col_null_map_to; - ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; - if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) - { - col_null_map_to = ColumnUInt8::create(size); - vec_null_map_to = &col_null_map_to->getData(); - } - - const ColumnString::Chars * chars = nullptr; - const IColumn::Offsets * offsets = nullptr; - size_t fixed_string_size = 0; - - if constexpr (std::is_same_v) - { - chars = &col_from_string->getChars(); - offsets = &col_from_string->getOffsets(); - } - else - { - chars = &col_from_fixed_string->getChars(); - fixed_string_size = col_from_fixed_string->getN(); - } - - size_t current_offset = 0; - - bool precise_float_parsing = false; - - if (DB::CurrentThread::isInitialized()) - { - const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - precise_float_parsing = query_context->getSettingsRef()[Setting::precise_float_parsing]; - } - - for (size_t i = 0; i < size; ++i) - { - size_t next_offset = std::is_same_v ? (*offsets)[i] : (current_offset + fixed_string_size); - size_t string_size = std::is_same_v ? next_offset - current_offset - 1 : fixed_string_size; - - ReadBufferFromMemory read_buffer(chars->data() + current_offset, string_size); - - if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw) - { - if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort) - { - if constexpr (to_datetime64) - { - DateTime64 res = 0; - parseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; - } - else - { - time_t res; - parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); - convertFromTime(vec_to[i], res); - } - } - else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) - { - if constexpr (to_datetime64) - { - DateTime64 res = 0; - parseDateTime64BestEffortUS(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; - } - else - { - time_t res; - parseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); - convertFromTime(vec_to[i], res); - } - } - else - { - if constexpr (to_datetime64) - { - DateTime64 value = 0; - readDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone); - vec_to[i] = value; - } - else if constexpr (IsDataTypeDecimal) - { - SerializationDecimal::readText( - vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); - } - else - { - /// we want to utilize constexpr condition here, which is not mixable with value comparison - do - { - if constexpr (std::is_same_v && std::is_same_v) - { - if (fixed_string_size == IPV6_BINARY_LENGTH) - { - readBinary(vec_to[i], read_buffer); - break; - } - } - if constexpr (std::is_same_v) - { - if (!tryParseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing)) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to type {}", TypeName); - } - else - parseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); - } while (false); - } - } - - if (!isAllRead(read_buffer)) - throwExceptionForIncompletelyParsedValue(read_buffer, *res_type); - } - else - { - bool parsed; - - if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort) - { - if constexpr (to_datetime64) - { - DateTime64 res = 0; - parsed = tryParseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; - } - else - { - time_t res; - parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); - convertFromTime(vec_to[i],res); - } - } - else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) - { - if constexpr (to_datetime64) - { - DateTime64 res = 0; - parsed = tryParseDateTime64BestEffortUS(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; - } - else - { - time_t res; - parsed = tryParseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); - convertFromTime(vec_to[i],res); - } - } - else - { - if constexpr (to_datetime64) - { - DateTime64 value = 0; - parsed = tryReadDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone); - vec_to[i] = value; - } - else if constexpr (IsDataTypeDecimal) - { - parsed = SerializationDecimal::tryReadText( - vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); - } - else if (std::is_same_v && std::is_same_v - && fixed_string_size == IPV6_BINARY_LENGTH) - { - readBinary(vec_to[i], read_buffer); - parsed = true; - } - else - { - parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); - } - } - - if (!isAllRead(read_buffer)) - parsed = false; - - if (!parsed) - { - if constexpr (std::is_same_v) - { - vec_to[i] = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance) - } - else - { - vec_to[i] = static_cast(0); - } - } - - if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) - (*vec_null_map_to)[i] = !parsed; - } - - current_offset = next_offset; - } - - if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) - return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); - else - return col_to; - } -}; - - -/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type. -struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; - -enum class BehaviourOnErrorFromString : uint8_t -{ - ConvertDefaultBehaviorTag, - ConvertReturnNullOnErrorTag, - ConvertReturnZeroOnErrorTag -}; - -/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment. - * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) - */ -template -struct ConvertImpl -{ - template - static ColumnPtr NO_SANITIZE_UNDEFINED execute( - const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type [[maybe_unused]], size_t input_rows_count, - BehaviourOnErrorFromString from_string_tag [[maybe_unused]], Additions additions = Additions()) - { - const ColumnWithTypeAndName & named_from = arguments[0]; - - if constexpr ((std::is_same_v && !FromDataType::is_parametric) - || (std::is_same_v && std::is_same_v) - || (std::is_same_v && std::is_same_v)) - { - /// If types are the same, reuse the columns. - /// Conversions between Enum and the underlying type are also free. - return named_from.column; - } - else if constexpr ((std::is_same_v || std::is_same_v) - && std::is_same_v) - { - /// Conversion of DateTime to Date: throw off time component. - /// Conversion of Date32 to Date. - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (std::is_same_v && std::is_same_v) - { - /// Conversion of DateTime to Date: throw off time component. - return DateTimeTransformImpl::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr ((std::is_same_v || std::is_same_v) - && std::is_same_v) - { - /// Conversion from Date/Date32 to DateTime. - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (std::is_same_v && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - /** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience, - * Float32, Float64) to Date. If the - * number is less than 65536, then it is treated as DayNum, and if it's greater or equals to 65536, - * then treated as unix timestamp. If the number exceeds UInt32, saturate to MAX_UINT32 then as DayNum. - * It's a bit illogical, as we actually have two functions in one. - * But allows to support frequent case, - * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. - * (otherwise such usage would be frequent mistake). - */ - else if constexpr (( - std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - /// Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to DateTime. - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (std::is_same_v - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count); - } - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - else if constexpr (std::is_same_v - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - else if constexpr (( - std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - /// Conversion of DateTime64 to Date or DateTime: discards fractional part. - else if constexpr (std::is_same_v - && std::is_same_v) - { - return DateTimeTransformImpl>, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - else if constexpr (std::is_same_v - && std::is_same_v) - { - return DateTimeTransformImpl>, false>::template execute( - arguments, result_type, input_rows_count, additions); - } - /// Conversion of Date or DateTime to DateTime64: add zero sub-second part. - else if constexpr (( - std::is_same_v - || std::is_same_v - || std::is_same_v) - && std::is_same_v) - { - return DateTimeTransformImpl::template execute( - arguments, result_type, input_rows_count, additions); - } - else if constexpr (IsDataTypeDateOrDateTime - && std::is_same_v) - { - /// Date or DateTime to String - - using FromFieldType = typename FromDataType::FieldType; - using ColVecType = ColumnVectorOrDecimal; - - auto datetime_arg = arguments[0]; - - const DateLUTImpl * time_zone = nullptr; - const ColumnConst * time_zone_column = nullptr; - - if (arguments.size() == 1) - { - auto non_null_args = createBlockWithNestedColumns(arguments); - time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); - } - else /// When we have a column for timezone - { - datetime_arg.column = datetime_arg.column->convertToFullColumnIfConst(); - - if constexpr (std::is_same_v || std::is_same_v) - time_zone = &DateLUT::instance(); - /// For argument of Date or DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v) - { - if ((time_zone_column = checkAndGetColumnConst(arguments[1].column.get()))) - { - auto non_null_args = createBlockWithNestedColumns(arguments); - time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); - } - } - } - const auto & col_with_type_and_name = columnGetNested(datetime_arg); - - if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) - { - auto col_to = ColumnString::create(); - - const typename ColVecType::Container & vec_from = col_from->getData(); - ColumnString::Chars & data_to = col_to->getChars(); - ColumnString::Offsets & offsets_to = col_to->getOffsets(); - size_t size = vec_from.size(); - - if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1)); - else - data_to.resize(size * 3); /// Arbitrary - - offsets_to.resize(size); - - WriteBufferFromVector write_buffer(data_to); - const FromDataType & type = static_cast(*col_with_type_and_name.type); - - ColumnUInt8::MutablePtr null_map = copyNullMap(datetime_arg.column); - - bool cut_trailing_zeros_align_to_groups_of_thousands = false; - if (DB::CurrentThread::isInitialized()) - { - const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - cut_trailing_zeros_align_to_groups_of_thousands = query_context->getSettingsRef()[Setting::date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands]; - } - - if (!null_map && arguments.size() > 1) - null_map = copyNullMap(arguments[1].column->convertToFullColumnIfConst()); - - if (null_map) - { - for (size_t i = 0; i < size; ++i) - { - if (!time_zone_column && arguments.size() > 1) - { - if (!arguments[1].column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); - } - bool is_ok = true; - if constexpr (std::is_same_v) - { - if (cut_trailing_zeros_align_to_groups_of_thousands) - writeDateTimeTextCutTrailingZerosAlignToGroupOfThousands(DateTime64(vec_from[i]), type.getScale(), write_buffer, *time_zone); - else - is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); - } - else - { - is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); - } - null_map->getData()[i] |= !is_ok; - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); - } - } - else - { - for (size_t i = 0; i < size; ++i) - { - if (!time_zone_column && arguments.size() > 1) - { - if (!arguments[1].column.get()->getDataAt(i).toString().empty()) - time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); - } - if constexpr (std::is_same_v) - { - if (cut_trailing_zeros_align_to_groups_of_thousands) - writeDateTimeTextCutTrailingZerosAlignToGroupOfThousands(DateTime64(vec_from[i]), type.getScale(), write_buffer, *time_zone); - else - FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); - } - else - { - FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); - } - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); - } - } - - write_buffer.finalize(); - - if (null_map) - return ColumnNullable::create(std::move(col_to), std::move(null_map)); - return col_to; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), Name::name); - } - /// Conversion from FixedString to String. - /// Cutting sequences of zero bytes from end of strings. - else if constexpr (std::is_same_v - && std::is_same_v) - { - ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); - const auto & nested = columnGetNested(arguments[0]); - if (const ColumnFixedString * col_from = checkAndGetColumn(nested.column.get())) - { - auto col_to = ColumnString::create(); - - const ColumnFixedString::Chars & data_from = col_from->getChars(); - ColumnString::Chars & data_to = col_to->getChars(); - ColumnString::Offsets & offsets_to = col_to->getOffsets(); - size_t size = col_from->size(); - size_t n = col_from->getN(); - data_to.resize(size * (n + 1)); /// + 1 - zero terminator - offsets_to.resize(size); - - size_t offset_from = 0; - size_t offset_to = 0; - for (size_t i = 0; i < size; ++i) - { - if (!null_map || !null_map->getData()[i]) - { - size_t bytes_to_copy = n; - while (bytes_to_copy > 0 && data_from[offset_from + bytes_to_copy - 1] == 0) - --bytes_to_copy; - - memcpy(&data_to[offset_to], &data_from[offset_from], bytes_to_copy); - offset_to += bytes_to_copy; - } - data_to[offset_to] = 0; - ++offset_to; - offsets_to[i] = offset_to; - offset_from += n; - } - - data_to.resize(offset_to); - if (result_type->isNullable() && null_map) - return ColumnNullable::create(std::move(col_to), std::move(null_map)); - return col_to; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), Name::name); - } - else if constexpr (std::is_same_v) - { - /// Anything else to String. - - using FromFieldType = typename FromDataType::FieldType; - using ColVecType = ColumnVectorOrDecimal; - - ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); - - const auto & col_with_type_and_name = columnGetNested(arguments[0]); - const auto & type = static_cast(*col_with_type_and_name.type); - - if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) - { - auto col_to = ColumnString::create(); - - const typename ColVecType::Container & vec_from = col_from->getData(); - ColumnString::Chars & data_to = col_to->getChars(); - ColumnString::Offsets & offsets_to = col_to->getOffsets(); - size_t size = vec_from.size(); - - data_to.resize(size * 3); - offsets_to.resize(size); - - WriteBufferFromVector write_buffer(data_to); - - if (null_map) - { - for (size_t i = 0; i < size; ++i) - { - bool is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); - /// We don't use timezones in this branch - null_map->getData()[i] |= !is_ok; - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); - } - } - else - { - for (size_t i = 0; i < size; ++i) - { - FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); - writeChar(0, write_buffer); - offsets_to[i] = write_buffer.count(); - } - } - - write_buffer.finalize(); - - if (null_map) - return ColumnNullable::create(std::move(col_to), std::move(null_map)); - return col_to; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - arguments[0].column->getName(), Name::name); - } - else if constexpr (std::is_same_v - && std::is_same_v - && std::is_same_v) - { - return ConvertImpl::template execute( - arguments, result_type, input_rows_count, from_string_tag); - } - else if constexpr ((std::is_same_v || std::is_same_v)) - { - switch (from_string_tag) - { - case BehaviourOnErrorFromString::ConvertDefaultBehaviorTag: - return ConvertThroughParsing::execute( - arguments, result_type, input_rows_count, additions); - case BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag: - return ConvertThroughParsing::execute( - arguments, result_type, input_rows_count, additions); - case BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag: - return ConvertThroughParsing::execute( - arguments, result_type, input_rows_count, additions); - } - } - else if constexpr (std::is_same_v && std::is_same_v) - { - IntervalKind to = typeid_cast(result_type.get())->getKind(); - IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); - - if (from == to || arguments[0].column->empty()) - return arguments[0].column; - - Int64 conversion_factor = 1; - Int64 result_value; - - int from_position = static_cast(from.kind); - int to_position = static_cast(to.kind); /// Positions of each interval according to granularity map - - if (from_position < to_position) - { - for (int i = from_position; i < to_position; ++i) - conversion_factor *= interval_conversions[i]; - result_value = arguments[0].column->getInt(0) / conversion_factor; - } - else - { - for (int i = from_position; i > to_position; --i) - conversion_factor *= interval_conversions[i]; - result_value = arguments[0].column->getInt(0) * conversion_factor; - } - - return ColumnConst::create(ColumnInt64::create(1, result_value), input_rows_count); - } - else - { - using FromFieldType = typename FromDataType::FieldType; - using ToFieldType = typename ToDataType::FieldType; - using ColVecFrom = typename FromDataType::ColumnType; - using ColVecTo = typename ToDataType::ColumnType; - - if constexpr ((IsDataTypeDecimal || IsDataTypeDecimal) - && !(std::is_same_v || std::is_same_v) - && (!IsDataTypeDecimalOrNumber || !IsDataTypeDecimalOrNumber)) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - named_from.column->getName(), Name::name); - } - - const ColVecFrom * col_from = checkAndGetColumn(named_from.column.get()); - if (!col_from) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", - named_from.column->getName(), Name::name); - - typename ColVecTo::MutablePtr col_to = nullptr; - - if constexpr (IsDataTypeDecimal) - { - UInt32 scale; - - if constexpr (std::is_same_v - || std::is_same_v) - { - scale = additions.scale; - } - else - { - scale = additions; - } - - col_to = ColVecTo::create(0, scale); - } - else - col_to = ColVecTo::create(); - - const auto & vec_from = col_from->getData(); - auto & vec_to = col_to->getData(); - vec_to.resize(input_rows_count); - - ColumnUInt8::MutablePtr col_null_map_to; - ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; - if constexpr (std::is_same_v) - { - col_null_map_to = ColumnUInt8::create(input_rows_count, false); - vec_null_map_to = &col_null_map_to->getData(); - } - - bool result_is_bool = isBool(result_type); - for (size_t i = 0; i < input_rows_count; ++i) - { - if constexpr (std::is_same_v) - { - if (result_is_bool) - { - vec_to[i] = vec_from[i] != FromFieldType(0); - continue; - } - } - - if constexpr (std::is_same_v && std::is_same_v) - { - static_assert( - std::is_same_v, - "UInt128 and UUID types must be same"); - - vec_to[i].items[1] = vec_from[i].toUnderType().items[0]; - vec_to[i].items[0] = vec_from[i].toUnderType().items[1]; - } - else if constexpr (std::is_same_v && std::is_same_v) - { - static_assert( - std::is_same_v, - "UInt128 and IPv6 types must be same"); - - vec_to[i].items[1] = std::byteswap(vec_from[i].toUnderType().items[0]); - vec_to[i].items[0] = std::byteswap(vec_from[i].toUnderType().items[1]); - } - else if constexpr (std::is_same_v != std::is_same_v) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Conversion between numeric types and UUID is not supported. " - "Probably the passed UUID is unquoted"); - } - else if constexpr ( - (std::is_same_v != std::is_same_v) - && !(is_any_of - || is_any_of)) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", - TypeName, TypeName); - } - else if constexpr (std::is_same_v != std::is_same_v - && !(std::is_same_v || std::is_same_v)) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Conversion between numeric types and IPv6 is not supported. " - "Probably the passed IPv6 is unquoted"); - } - else if constexpr (IsDataTypeDecimal || IsDataTypeDecimal) - { - if constexpr (std::is_same_v) - { - ToFieldType result; - bool convert_result = false; - - if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) - convert_result = tryConvertDecimals(vec_from[i], col_from->getScale(), col_to->getScale(), result); - else if constexpr (IsDataTypeDecimal && IsDataTypeNumber) - convert_result = tryConvertFromDecimal(vec_from[i], col_from->getScale(), result); - else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) - convert_result = tryConvertToDecimal(vec_from[i], col_to->getScale(), result); - - if (convert_result) - vec_to[i] = result; - else - { - vec_to[i] = static_cast(0); - (*vec_null_map_to)[i] = true; - } - } - else - { - if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) - vec_to[i] = convertDecimals(vec_from[i], col_from->getScale(), col_to->getScale()); - else if constexpr (IsDataTypeDecimal && IsDataTypeNumber) - vec_to[i] = convertFromDecimal(vec_from[i], col_from->getScale()); - else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) - vec_to[i] = convertToDecimal(vec_from[i], col_to->getScale()); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Unsupported data type in conversion function"); - } - } - else if constexpr (std::is_same_v && std::is_same_v) - { - const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; - const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); - if (!matchIPv6Subnet(src, ip4_cidr, 96)) - { - char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; - char * paddr = addr; - formatIPv6(src, paddr); - - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, named_from.column->getName()); - } - - uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); - if constexpr (std::endian::native == std::endian::little) - { - dst[0] = src[15]; - dst[1] = src[14]; - dst[2] = src[13]; - dst[3] = src[12]; - } - else - { - dst[0] = src[12]; - dst[1] = src[13]; - dst[2] = src[14]; - dst[3] = src[15]; - } - } - else if constexpr (std::is_same_v && std::is_same_v) - { - const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); - uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); - std::memset(dst, '\0', IPV6_BINARY_LENGTH); - dst[10] = dst[11] = 0xff; - - if constexpr (std::endian::native == std::endian::little) - { - dst[12] = src[3]; - dst[13] = src[2]; - dst[14] = src[1]; - dst[15] = src[0]; - } - else - { - dst[12] = src[0]; - dst[13] = src[1]; - dst[14] = src[2]; - dst[15] = src[3]; - } - } - else if constexpr (std::is_same_v && std::is_same_v) - { - vec_to[i] = static_cast(static_cast(vec_from[i])); - } - else if constexpr (std::is_same_v - && (std::is_same_v || std::is_same_v)) - { - vec_to[i] = static_cast(vec_from[i] * DATE_SECONDS_PER_DAY); - } - else - { - /// If From Data is Nan or Inf and we convert to integer type, throw exception - if constexpr (is_floating_point && !is_floating_point) - { - if (!isFinite(vec_from[i])) - { - if constexpr (std::is_same_v) - { - vec_to[i] = 0; - (*vec_null_map_to)[i] = true; - continue; - } - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Unexpected inf or nan to integer conversion"); - } - } - - if constexpr (std::is_same_v - || std::is_same_v) - { - bool convert_result = accurate::convertNumeric(vec_from[i], vec_to[i]); - - if (!convert_result) - { - if (std::is_same_v) - { - vec_to[i] = 0; - (*vec_null_map_to)[i] = true; - } - else - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", - named_from.column->getName(), result_type->getName()); - } - } - } - else - { - vec_to[i] = static_cast(vec_from[i]); - } - } - } - - if constexpr (std::is_same_v) - return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); - else - return col_to; - } - } -}; - - -/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization. -template -struct ConvertImplGenericFromString -{ - static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count, const ContextPtr & context) - { - const IColumn & column_from = *arguments[0].column; - const IDataType & data_type_to = *result_type; - auto res = data_type_to.createColumn(); - auto serialization = data_type_to.getDefaultSerialization(); - const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - - executeImpl(column_from, *res, *serialization, input_rows_count, null_map, result_type.get(), context); - return res; - } - - static void executeImpl( - const IColumn & column_from, - IColumn & column_to, - const ISerialization & serialization_from, - size_t input_rows_count, - const PaddedPODArray * null_map, - const IDataType * result_type, - const ContextPtr & context) - { - column_to.reserve(input_rows_count); - - FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; - for (size_t i = 0; i < input_rows_count; ++i) - { - if (null_map && (*null_map)[i]) - { - column_to.insertDefault(); - continue; - } - - const auto & val = column_from.getDataAt(i); - ReadBufferFromMemory read_buffer(val.data, val.size); - try - { - serialization_from.deserializeWholeText(column_to, read_buffer, format_settings); - } - catch (const Exception &) - { - if constexpr (throw_on_error) - throw; - /// Check if exception happened after we inserted the value - /// (deserializeWholeText should not do it, but let's check anyway). - if (column_to.size() > i) - column_to.popBack(column_to.size() - i); - column_to.insertDefault(); - } - - /// Usually deserializeWholeText checks for eof after parsing, but let's check one more time just in case. - if (!read_buffer.eof()) - { - if constexpr (throw_on_error) - { - if (result_type) - throwExceptionForIncompletelyParsedValue(read_buffer, *result_type); - else - throw Exception( - ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to column {}. Expected eof", column_to.getName()); - } - else - { - if (column_to.size() > i) - column_to.popBack(column_to.size() - i); - column_to.insertDefault(); - } - } - } - } -}; - - -/// Declared early because used below. -struct NameToDate { static constexpr auto name = "toDate"; }; -struct NameToDate32 { static constexpr auto name = "toDate32"; }; -struct NameToDateTime { static constexpr auto name = "toDateTime"; }; -struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; }; -struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; }; -struct NameToString { static constexpr auto name = "toString"; }; -struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; }; -struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; }; -struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; }; -struct NameToDecimal256 { static constexpr auto name = "toDecimal256"; }; - - -#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \ - struct NameToInterval ## INTERVAL_KIND \ - { \ - static constexpr auto name = "toInterval" #INTERVAL_KIND; \ - static constexpr auto kind = IntervalKind::Kind::INTERVAL_KIND; \ - }; - -DEFINE_NAME_TO_INTERVAL(Nanosecond) -DEFINE_NAME_TO_INTERVAL(Microsecond) -DEFINE_NAME_TO_INTERVAL(Millisecond) -DEFINE_NAME_TO_INTERVAL(Second) -DEFINE_NAME_TO_INTERVAL(Minute) -DEFINE_NAME_TO_INTERVAL(Hour) -DEFINE_NAME_TO_INTERVAL(Day) -DEFINE_NAME_TO_INTERVAL(Week) -DEFINE_NAME_TO_INTERVAL(Month) -DEFINE_NAME_TO_INTERVAL(Quarter) -DEFINE_NAME_TO_INTERVAL(Year) - -#undef DEFINE_NAME_TO_INTERVAL - -struct NameParseDateTimeBestEffort; -struct NameParseDateTimeBestEffortOrZero; -struct NameParseDateTimeBestEffortOrNull; - -template -constexpr bool mightBeDateTime() -{ - if constexpr (std::is_same_v) - return true; - else if constexpr ( - std::is_same_v || std::is_same_v - || std::is_same_v || std::is_same_v) - return true; - - return false; -} - -template -inline bool isDateTime64(const ColumnsWithTypeAndName & arguments) -{ - if constexpr (std::is_same_v) - return true; - else if constexpr (std::is_same_v || std::is_same_v - || std::is_same_v || std::is_same_v) - { - return (arguments.size() == 2 && isUInt(arguments[1].type)) || arguments.size() == 3; - } - - return false; -} - -template -class FunctionConvert : public IFunction -{ -public: - using Monotonic = MonotonicityImpl; - - static constexpr auto name = Name::name; - static constexpr bool to_datetime64 = std::is_same_v; - static constexpr bool to_decimal = IsDataTypeDecimal && !to_datetime64; - - static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionConvert(ContextPtr context_) : context(context_) {} - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool isInjective(const ColumnsWithTypeAndName &) const override { return std::is_same_v; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override - { - return !(IsDataTypeDateOrDateTime && isNumber(*arguments[0].type)); - } - - using DefaultReturnTypeGetter = std::function; - static DataTypePtr getReturnTypeDefaultImplementationForNulls(const ColumnsWithTypeAndName & arguments, const DefaultReturnTypeGetter & getter) - { - NullPresence null_presence = getNullPresense(arguments); - - if (null_presence.has_null_constant) - { - return makeNullable(std::make_shared()); - } - if (null_presence.has_nullable) - { - auto nested_columns = Block(createBlockWithNestedColumns(arguments)); - auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end())); - return makeNullable(return_type); - } - - return getter(arguments); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - auto getter = [&] (const auto & args) { return getReturnTypeImplRemovedNullable(args); }; - auto res = getReturnTypeDefaultImplementationForNulls(arguments, getter); - to_nullable = res->isNullable(); - checked_return_type = true; - return res; - } - - DataTypePtr getReturnTypeImplRemovedNullable(const ColumnsWithTypeAndName & arguments) const - { - FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, "any type"}}; - FunctionArgumentDescriptors optional_args; - - if constexpr (to_decimal) - { - mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); - } - - if (!to_decimal && isDateTime64(arguments)) - { - mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); - } - - // toString(DateTime or DateTime64, [timezone: String]) - if ((std::is_same_v && !arguments.empty() && (isDateTime64(arguments[0].type) || isDateTime(arguments[0].type))) - // toUnixTimestamp(value[, timezone : String]) - || std::is_same_v - // toDate(value[, timezone : String]) - || std::is_same_v // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below. - // toDate32(value[, timezone : String]) - || std::is_same_v - // toDateTime(value[, timezone: String]) - || std::is_same_v - // toDateTime64(value, scale : Integer[, timezone: String]) - || std::is_same_v) - { - optional_args.push_back({"timezone", static_cast(&isString), nullptr, "String"}); - } - - validateFunctionArguments(*this, arguments, mandatory_args, optional_args); - - if constexpr (std::is_same_v) - { - return std::make_shared(Name::kind); - } - else if constexpr (to_decimal) - { - UInt64 scale = extractToDecimalScale(arguments[1]); - - if constexpr (std::is_same_v) - return createDecimalMaxPrecision(scale); - else if constexpr (std::is_same_v) - return createDecimalMaxPrecision(scale); - else if constexpr (std::is_same_v) - return createDecimalMaxPrecision(scale); - else if constexpr (std::is_same_v) - return createDecimalMaxPrecision(scale); - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected branch in code of conversion function: it is a bug."); - } - else - { - // Optional second argument with time zone for DateTime. - UInt8 timezone_arg_position = 1; - UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale; - - // DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first. - if (isDateTime64(arguments)) - { - timezone_arg_position += 1; - scale = static_cast(arguments[1].column->get64(0)); - - if (to_datetime64 || scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime - return std::make_shared(scale, - extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); - - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); - } - - if constexpr (std::is_same_v) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); - else if constexpr (std::is_same_v) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected branch in code of conversion function: it is a bug."); - else - return std::make_shared(); - } - } - - /// Function actually uses default implementation for nulls, - /// but we need to know if return type is Nullable or not, - /// so we use checked_return_type only to intercept the first call to getReturnTypeImpl(...). - bool useDefaultImplementationForNulls() const override - { - bool to_nullable_string = to_nullable && std::is_same_v; - return checked_return_type && !to_nullable_string; - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override - { - if constexpr (std::is_same_v) - return {}; - else if constexpr (std::is_same_v) - return {2}; - return {1}; - } - bool canBeExecutedOnDefaultArguments() const override { return false; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - try - { - return executeInternal(arguments, result_type, input_rows_count); - } - catch (Exception & e) - { - /// More convenient error message. - if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - { - e.addMessage("Cannot parse " - + result_type->getName() + " from " - + arguments[0].type->getName() - + ", because value is too short"); - } - else if (e.code() == ErrorCodes::CANNOT_PARSE_NUMBER - || e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT - || e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED - || e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING - || e.code() == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE - || e.code() == ErrorCodes::CANNOT_PARSE_DATE - || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME - || e.code() == ErrorCodes::CANNOT_PARSE_UUID - || e.code() == ErrorCodes::CANNOT_PARSE_IPV4 - || e.code() == ErrorCodes::CANNOT_PARSE_IPV6) - { - e.addMessage("Cannot parse " - + result_type->getName() + " from " - + arguments[0].type->getName()); - } - - throw; - } - } - - bool hasInformationAboutMonotonicity() const override - { - return Monotonic::has(); - } - - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override - { - return Monotonic::get(type, left, right); - } - -private: - ContextPtr context; - mutable bool checked_return_type = false; - mutable bool to_nullable = false; - - ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const - { - if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least 1 argument", getName()); - - if (result_type->onlyNull()) - return result_type->createColumnConstWithDefaultValue(input_rows_count); - - const DataTypePtr from_type = removeNullable(arguments[0].type); - ColumnPtr result_column; - - FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; - - if (context) - date_time_overflow_behavior = context->getSettingsRef()[Setting::date_time_overflow_behavior].value; - - auto call = [&](const auto & types, BehaviourOnErrorFromString from_string_tag) -> bool - { - using Types = std::decay_t; - using LeftDataType = typename Types::LeftType; - using RightDataType = typename Types::RightType; - - if constexpr (IsDataTypeDecimal) - { - if constexpr (std::is_same_v) - { - /// Account for optional timezone argument. - if (arguments.size() != 2 && arguments.size() != 3) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects 2 or 3 arguments for DataTypeDateTime64.", getName()); - } - else if (arguments.size() != 2) - { - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects 2 arguments for Decimal.", getName()); - } - - const ColumnWithTypeAndName & scale_column = arguments[1]; - UInt32 scale = extractToDecimalScale(scale_column); - - switch (date_time_overflow_behavior) - { - case FormatSettings::DateTimeOverflowBehavior::Throw: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); - break; - case FormatSettings::DateTimeOverflowBehavior::Ignore: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); - break; - case FormatSettings::DateTimeOverflowBehavior::Saturate: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); - break; - } - } - else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) - { - const auto * dt64 = assert_cast(arguments[0].type.get()); - switch (date_time_overflow_behavior) - { - case FormatSettings::DateTimeOverflowBehavior::Throw: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); - break; - case FormatSettings::DateTimeOverflowBehavior::Ignore: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); - break; - case FormatSettings::DateTimeOverflowBehavior::Saturate: - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); - break; - } - } - else if constexpr ((IsDataTypeNumber - || IsDataTypeDateOrDateTime)&&IsDataTypeDateOrDateTime) - { -#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \ - case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ - result_column = ConvertImpl::execute( \ - arguments, result_type, input_rows_count, from_string_tag); \ - break; - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw) - GENERATE_OVERFLOW_MODE_CASE(Ignore) - GENERATE_OVERFLOW_MODE_CASE(Saturate) - } - -#undef GENERATE_OVERFLOW_MODE_CASE - } - else if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber) - { - using LeftT = typename LeftDataType::FieldType; - using RightT = typename RightDataType::FieldType; - - static constexpr bool bad_left = - is_decimal || is_floating_point || is_big_int_v || is_signed_v; - static constexpr bool bad_right = - is_decimal || is_floating_point || is_big_int_v || is_signed_v; - - /// Disallow int vs UUID conversion (but support int vs UInt128 conversion) - if constexpr ((bad_left && std::is_same_v) || - (bad_right && std::is_same_v)) - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Wrong UUID conversion"); - } - else - { - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, from_string_tag); - } - } - else - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); - - return true; - }; - - if constexpr (mightBeDateTime()) - { - if (isDateTime64(arguments)) - { - /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 - const ColumnWithTypeAndName & scale_column = arguments[1]; - UInt32 scale = extractToDecimalScale(scale_column); - - if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 - { - if (!callOnIndexAndDataType( - from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}", - arguments[0].type->getName(), - getName()); - - return result_column; - } - } - } - - if constexpr (std::is_same_v) - { - if (from_type->getCustomSerialization()) - return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); - } - - bool done = false; - if constexpr (is_any_of) - { - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); - } - else - { - bool cast_ipv4_ipv6_default_on_conversion_error = false; - if constexpr (is_any_of) - { - if (context && (cast_ipv4_ipv6_default_on_conversion_error = context->getSettingsRef()[Setting::cast_ipv4_ipv6_default_on_conversion_error])) - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag); - } - - if (!cast_ipv4_ipv6_default_on_conversion_error) - { - /// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString) - /// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values. - if (to_nullable && WhichDataType(from_type).isStringOrFixedString()) - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag); - else - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); - } - - if constexpr (std::is_same_v) - if (WhichDataType(from_type).isInterval()) - done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); - } - - if (!done) - { - /// Generic conversion of any type to String. - if (std::is_same_v) - { - return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); - } - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0].type->getName(), getName()); - } - - return result_column; - } -}; - - -/** Function toTOrZero (where T is number of date or datetime type): - * try to convert from String to type T through parsing, - * if cannot parse, return default value instead of throwing exception. - * Function toTOrNull will return Nullable type with NULL when cannot parse. - * NOTE Also need to implement tryToUnixTimestamp with timezone. - */ -template -class FunctionConvertFromString : public IFunction -{ -public: - static constexpr auto name = Name::name; - static constexpr bool to_datetime64 = std::is_same_v; - static constexpr bool to_decimal = IsDataTypeDecimal && !to_datetime64; - - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - bool useDefaultImplementationForConstants() const override { return true; } - bool canBeExecutedOnDefaultArguments() const override { return false; } - - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - DataTypePtr res; - - if (isDateTime64(arguments)) - { - validateFunctionArguments(*this, arguments, - FunctionArgumentDescriptors{{"string", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"}}, - // optional - FunctionArgumentDescriptors{ - {"precision", static_cast(&isUInt8), isColumnConst, "const UInt8"}, - {"timezone", static_cast(&isStringOrFixedString), isColumnConst, "const String or FixedString"}, - }); - - UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; - if (arguments.size() > 1) - scale = extractToDecimalScale(arguments[1]); - const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false); - - res = scale == 0 ? res = std::make_shared(timezone) : std::make_shared(scale, timezone); - } - else - { - if ((arguments.size() != 1 && arguments.size() != 2) || (to_decimal && arguments.size() != 2)) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2. " - "Second argument only make sense for DateTime (time zone, optional) and Decimal (scale).", - getName(), arguments.size()); - - if (!isStringOrFixedString(arguments[0].type)) - { - if (this->getName().find("OrZero") != std::string::npos || - this->getName().find("OrNull") != std::string::npos) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " - "Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument", - arguments[0].type->getName(), getName()); - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", - arguments[0].type->getName(), getName()); - } - - if (arguments.size() == 2) - { - if constexpr (std::is_same_v) - { - if (!isString(arguments[1].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}", - arguments[1].type->getName(), getName()); - } - else if constexpr (to_decimal) - { - if (!isInteger(arguments[1].type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}", - arguments[1].type->getName(), getName()); - if (!arguments[1].column) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant", getName()); - } - else - { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1. " - "Second argument makes sense only for DateTime and Decimal.", - getName(), arguments.size()); - } - } - - if constexpr (std::is_same_v) - res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); - else if constexpr (std::is_same_v) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); - else if constexpr (to_decimal) - { - UInt64 scale = extractToDecimalScale(arguments[1]); - res = createDecimalMaxPrecision(scale); - if (!res) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Something wrong with toDecimalNNOrZero() or toDecimalNNOrNull()"); - } - else - res = std::make_shared(); - } - - if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) - res = std::make_shared(res); - - return res; - } - - template - ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, UInt32 scale) const - { - const IDataType * from_type = arguments[0].type.get(); - - if (checkAndGetDataType(from_type)) - { - return ConvertThroughParsing::execute( - arguments, result_type, input_rows_count, scale); - } - else if (checkAndGetDataType(from_type)) - { - return ConvertThroughParsing::execute( - arguments, result_type, input_rows_count, scale); - } - - return nullptr; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - ColumnPtr result_column; - - if constexpr (to_decimal) - { - result_column = executeInternal(arguments, result_type, input_rows_count, - assert_cast(*removeNullable(result_type)).getScale()); - } - else if constexpr (mightBeDateTime()) - { - if (isDateTime64(arguments)) - { - UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; - if (arguments.size() > 1) - scale = extractToDecimalScale(arguments[1]); - - if (scale == 0) - { - result_column = executeInternal(arguments, result_type, input_rows_count, 0); - } - else - { - result_column - = executeInternal(arguments, result_type, input_rows_count, static_cast(scale)); - } - } - else - { - result_column = executeInternal(arguments, result_type, input_rows_count, 0); - } - } - else - { - result_column = executeInternal(arguments, result_type, input_rows_count, 0); - } - - if (!result_column) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " - "Only String or FixedString argument is accepted for try-conversion function. For other arguments, " - "use function without 'orZero' or 'orNull'.", arguments[0].type->getName(), getName()); - - return result_column; - } -}; - - -/// Monotonicity. - -struct PositiveMonotonicity -{ - static bool has() { return true; } - static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &) - { - return { .is_monotonic = true }; - } -}; - -struct UnknownMonotonicity -{ - static bool has() { return false; } - static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &) - { - return { }; - } -}; - -template -struct ToNumberMonotonicity -{ - static bool has() { return true; } - - static UInt64 divideByRangeOfType(UInt64 x) - { - if constexpr (sizeof(T) < sizeof(UInt64)) - return x >> (sizeof(T) * 8); - else - return 0; - } - - static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) - { - if (!type.isValueRepresentedByNumber()) - return {}; - - /// If type is same, the conversion is always monotonic. - /// (Enum has separate case, because it is different data type) - if (checkAndGetDataType>(&type) || - checkAndGetDataType>(&type)) - return { .is_monotonic = true, .is_always_monotonic = true }; - - /// Float cases. - - /// When converting to Float, the conversion is always monotonic. - if constexpr (is_floating_point) - return { .is_monotonic = true, .is_always_monotonic = true }; - - const auto * low_cardinality = typeid_cast(&type); - const IDataType * low_cardinality_dictionary_type = nullptr; - if (low_cardinality) - low_cardinality_dictionary_type = low_cardinality->getDictionaryType().get(); - - WhichDataType which_type(type); - WhichDataType which_inner_type = low_cardinality - ? WhichDataType(low_cardinality_dictionary_type) - : WhichDataType(type); - - /// If converting from Float, for monotonicity, arguments must fit in range of result type. - if (which_inner_type.isFloat()) - { - if (left.isNull() || right.isNull()) - return {}; - - Float64 left_float = left.safeGet(); - Float64 right_float = right.safeGet(); - - if (left_float >= static_cast(std::numeric_limits::min()) - && left_float <= static_cast(std::numeric_limits::max()) - && right_float >= static_cast(std::numeric_limits::min()) - && right_float <= static_cast(std::numeric_limits::max())) - return { .is_monotonic = true }; - - return {}; - } - - /// Integer cases. - - /// Only support types represented by native integers. - /// It can be extended to big integers, decimals and DateTime64 later. - /// By the way, NULLs are representing unbounded ranges. - if (!((left.isNull() || left.getType() == Field::Types::UInt64 || left.getType() == Field::Types::Int64) - && (right.isNull() || right.getType() == Field::Types::UInt64 || right.getType() == Field::Types::Int64))) - return {}; - - const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); - const bool to_is_unsigned = is_unsigned_v; - - const size_t size_of_from = type.getSizeOfValueInMemory(); - const size_t size_of_to = sizeof(T); - - const bool left_in_first_half = left.isNull() - ? from_is_unsigned - : (left.safeGet() >= 0); - - const bool right_in_first_half = right.isNull() - ? !from_is_unsigned - : (right.safeGet() >= 0); - - /// Size of type is the same. - if (size_of_from == size_of_to) - { - if (from_is_unsigned == to_is_unsigned) - return { .is_monotonic = true, .is_always_monotonic = true }; - - if (left_in_first_half == right_in_first_half) - return { .is_monotonic = true }; - - return {}; - } - - /// Size of type is expanded. - if (size_of_from < size_of_to) - { - if (from_is_unsigned == to_is_unsigned) - return { .is_monotonic = true, .is_always_monotonic = true }; - - if (!to_is_unsigned) - return { .is_monotonic = true, .is_always_monotonic = true }; - - /// signed -> unsigned. If arguments from the same half, then function is monotonic. - if (left_in_first_half == right_in_first_half) - return { .is_monotonic = true }; - - return {}; - } - - /// Size of type is shrunk. - if (size_of_from > size_of_to) - { - /// Function cannot be monotonic on unbounded ranges. - if (left.isNull() || right.isNull()) - return {}; - - /// Function cannot be monotonic when left and right are not on the same ranges. - if (divideByRangeOfType(left.safeGet()) != divideByRangeOfType(right.safeGet())) - return {}; - - if (to_is_unsigned) - return { .is_monotonic = true }; - else - { - // If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly. - const bool is_monotonic = (T(left.safeGet()) >= 0) == (T(right.safeGet()) >= 0); - - return { .is_monotonic = is_monotonic }; - } - } - - UNREACHABLE(); - } -}; - -struct ToDateMonotonicity -{ - static bool has() { return true; } - - static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) - { - auto which = WhichDataType(type); - if (which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() - || which.isUInt16()) - { - return {.is_monotonic = true, .is_always_monotonic = true}; - } - else if ( - ((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull()) - && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) - || ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull()) - && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) - || (( - (left.getType() == Field::Types::Float64 || left.isNull()) - && (right.getType() == Field::Types::Float64 || right.isNull()) - && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF)))) - || !isNativeNumber(type)) - { - return {}; - } - else - { - return {.is_monotonic = true, .is_always_monotonic = true}; - } - } -}; - -struct ToDateTimeMonotonicity -{ - static bool has() { return true; } - - static IFunction::Monotonicity get(const IDataType & type, const Field &, const Field &) - { - if (type.isValueRepresentedByNumber()) - return {.is_monotonic = true, .is_always_monotonic = true}; - else - return {}; - } -}; - -/** The monotonicity for the `toString` function is mainly determined for test purposes. - * It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`. - */ -struct ToStringMonotonicity -{ - static bool has() { return true; } - - static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) - { - IFunction::Monotonicity positive{ .is_monotonic = true }; - IFunction::Monotonicity not_monotonic; - - const auto * type_ptr = &type; - if (const auto * low_cardinality_type = checkAndGetDataType(type_ptr)) - type_ptr = low_cardinality_type->getDictionaryType().get(); - - /// Order on enum values (which is the order on integers) is completely arbitrary in respect to the order on strings. - if (WhichDataType(type).isEnum()) - return not_monotonic; - - /// `toString` function is monotonous if the argument is Date or Date32 or DateTime or String, or non-negative numbers with the same number of symbols. - if (checkDataTypes(type_ptr)) - return positive; - - if (left.isNull() || right.isNull()) - return {}; - - if (left.getType() == Field::Types::UInt64 - && right.getType() == Field::Types::UInt64) - { - return (left.safeGet() == 0 && right.safeGet() == 0) - || (floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) - ? positive : not_monotonic; - } - - if (left.getType() == Field::Types::Int64 - && right.getType() == Field::Types::Int64) - { - return (left.safeGet() == 0 && right.safeGet() == 0) - || (left.safeGet() > 0 && right.safeGet() > 0 && floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) - ? positive : not_monotonic; - } - - return not_monotonic; - } -}; - - -struct NameToUInt8 { static constexpr auto name = "toUInt8"; }; -struct NameToUInt16 { static constexpr auto name = "toUInt16"; }; -struct NameToUInt32 { static constexpr auto name = "toUInt32"; }; -struct NameToUInt64 { static constexpr auto name = "toUInt64"; }; -struct NameToUInt128 { static constexpr auto name = "toUInt128"; }; -struct NameToUInt256 { static constexpr auto name = "toUInt256"; }; -struct NameToInt8 { static constexpr auto name = "toInt8"; }; -struct NameToInt16 { static constexpr auto name = "toInt16"; }; -struct NameToInt32 { static constexpr auto name = "toInt32"; }; -struct NameToInt64 { static constexpr auto name = "toInt64"; }; -struct NameToInt128 { static constexpr auto name = "toInt128"; }; -struct NameToInt256 { static constexpr auto name = "toInt256"; }; -struct NameToBFloat16 { static constexpr auto name = "toBFloat16"; }; -struct NameToFloat32 { static constexpr auto name = "toFloat32"; }; -struct NameToFloat64 { static constexpr auto name = "toFloat64"; }; -struct NameToUUID { static constexpr auto name = "toUUID"; }; -struct NameToIPv4 { static constexpr auto name = "toIPv4"; }; -struct NameToIPv6 { static constexpr auto name = "toIPv6"; }; - -using FunctionToUInt8 = FunctionConvert>; -using FunctionToUInt16 = FunctionConvert>; -using FunctionToUInt32 = FunctionConvert>; -using FunctionToUInt64 = FunctionConvert>; -using FunctionToUInt128 = FunctionConvert>; -using FunctionToUInt256 = FunctionConvert>; -using FunctionToInt8 = FunctionConvert>; -using FunctionToInt16 = FunctionConvert>; -using FunctionToInt32 = FunctionConvert>; -using FunctionToInt64 = FunctionConvert>; -using FunctionToInt128 = FunctionConvert>; -using FunctionToInt256 = FunctionConvert>; -using FunctionToBFloat16 = FunctionConvert>; -using FunctionToFloat32 = FunctionConvert>; -using FunctionToFloat64 = FunctionConvert>; - -using FunctionToDate = FunctionConvert; - -using FunctionToDate32 = FunctionConvert; - -using FunctionToDateTime = FunctionConvert; - -using FunctionToDateTime32 = FunctionConvert; - -using FunctionToDateTime64 = FunctionConvert; - -using FunctionToUUID = FunctionConvert>; -using FunctionToIPv4 = FunctionConvert>; -using FunctionToIPv6 = FunctionConvert>; -using FunctionToString = FunctionConvert; -using FunctionToUnixTimestamp = FunctionConvert>; -using FunctionToDecimal32 = FunctionConvert, NameToDecimal32, UnknownMonotonicity>; -using FunctionToDecimal64 = FunctionConvert, NameToDecimal64, UnknownMonotonicity>; -using FunctionToDecimal128 = FunctionConvert, NameToDecimal128, UnknownMonotonicity>; -using FunctionToDecimal256 = FunctionConvert, NameToDecimal256, UnknownMonotonicity>; - -template struct FunctionTo; - -template <> struct FunctionTo { using Type = FunctionToUInt8; }; -template <> struct FunctionTo { using Type = FunctionToUInt16; }; -template <> struct FunctionTo { using Type = FunctionToUInt32; }; -template <> struct FunctionTo { using Type = FunctionToUInt64; }; -template <> struct FunctionTo { using Type = FunctionToUInt128; }; -template <> struct FunctionTo { using Type = FunctionToUInt256; }; -template <> struct FunctionTo { using Type = FunctionToInt8; }; -template <> struct FunctionTo { using Type = FunctionToInt16; }; -template <> struct FunctionTo { using Type = FunctionToInt32; }; -template <> struct FunctionTo { using Type = FunctionToInt64; }; -template <> struct FunctionTo { using Type = FunctionToInt128; }; -template <> struct FunctionTo { using Type = FunctionToInt256; }; -template <> struct FunctionTo { using Type = FunctionToBFloat16; }; -template <> struct FunctionTo { using Type = FunctionToFloat32; }; -template <> struct FunctionTo { using Type = FunctionToFloat64; }; - -template -struct FunctionTo { using Type = FunctionToDate; }; - -template -struct FunctionTo { using Type = FunctionToDate32; }; - -template -struct FunctionTo { using Type = FunctionToDateTime; }; - -template -struct FunctionTo { using Type = FunctionToDateTime64; }; - -template <> struct FunctionTo { using Type = FunctionToUUID; }; -template <> struct FunctionTo { using Type = FunctionToIPv4; }; -template <> struct FunctionTo { using Type = FunctionToIPv6; }; -template <> struct FunctionTo { using Type = FunctionToString; }; -template <> struct FunctionTo { using Type = FunctionToFixedString; }; -template <> struct FunctionTo> { using Type = FunctionToDecimal32; }; -template <> struct FunctionTo> { using Type = FunctionToDecimal64; }; -template <> struct FunctionTo> { using Type = FunctionToDecimal128; }; -template <> struct FunctionTo> { using Type = FunctionToDecimal256; }; - -template struct FunctionTo> - : FunctionTo> -{ -}; - -struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; }; -struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; }; -struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; }; -struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; }; -struct NameToUInt128OrZero { static constexpr auto name = "toUInt128OrZero"; }; -struct NameToUInt256OrZero { static constexpr auto name = "toUInt256OrZero"; }; -struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; }; -struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; }; -struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; }; -struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; }; -struct NameToInt128OrZero { static constexpr auto name = "toInt128OrZero"; }; -struct NameToInt256OrZero { static constexpr auto name = "toInt256OrZero"; }; -struct NameToBFloat16OrZero { static constexpr auto name = "toBFloat16OrZero"; }; -struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; }; -struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; }; -struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; }; -struct NameToDate32OrZero { static constexpr auto name = "toDate32OrZero"; }; -struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; }; -struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; }; -struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; }; -struct NameToDecimal64OrZero { static constexpr auto name = "toDecimal64OrZero"; }; -struct NameToDecimal128OrZero { static constexpr auto name = "toDecimal128OrZero"; }; -struct NameToDecimal256OrZero { static constexpr auto name = "toDecimal256OrZero"; }; -struct NameToUUIDOrZero { static constexpr auto name = "toUUIDOrZero"; }; -struct NameToIPv4OrZero { static constexpr auto name = "toIPv4OrZero"; }; -struct NameToIPv6OrZero { static constexpr auto name = "toIPv6OrZero"; }; - -using FunctionToUInt8OrZero = FunctionConvertFromString; -using FunctionToUInt16OrZero = FunctionConvertFromString; -using FunctionToUInt32OrZero = FunctionConvertFromString; -using FunctionToUInt64OrZero = FunctionConvertFromString; -using FunctionToUInt128OrZero = FunctionConvertFromString; -using FunctionToUInt256OrZero = FunctionConvertFromString; -using FunctionToInt8OrZero = FunctionConvertFromString; -using FunctionToInt16OrZero = FunctionConvertFromString; -using FunctionToInt32OrZero = FunctionConvertFromString; -using FunctionToInt64OrZero = FunctionConvertFromString; -using FunctionToInt128OrZero = FunctionConvertFromString; -using FunctionToInt256OrZero = FunctionConvertFromString; -using FunctionToBFloat16OrZero = FunctionConvertFromString; -using FunctionToFloat32OrZero = FunctionConvertFromString; -using FunctionToFloat64OrZero = FunctionConvertFromString; -using FunctionToDateOrZero = FunctionConvertFromString; -using FunctionToDate32OrZero = FunctionConvertFromString; -using FunctionToDateTimeOrZero = FunctionConvertFromString; -using FunctionToDateTime64OrZero = FunctionConvertFromString; -using FunctionToDecimal32OrZero = FunctionConvertFromString, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>; -using FunctionToDecimal64OrZero = FunctionConvertFromString, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>; -using FunctionToDecimal128OrZero = FunctionConvertFromString, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>; -using FunctionToDecimal256OrZero = FunctionConvertFromString, NameToDecimal256OrZero, ConvertFromStringExceptionMode::Zero>; -using FunctionToUUIDOrZero = FunctionConvertFromString; -using FunctionToIPv4OrZero = FunctionConvertFromString; -using FunctionToIPv6OrZero = FunctionConvertFromString; - -struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; }; -struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; }; -struct NameToUInt32OrNull { static constexpr auto name = "toUInt32OrNull"; }; -struct NameToUInt64OrNull { static constexpr auto name = "toUInt64OrNull"; }; -struct NameToUInt128OrNull { static constexpr auto name = "toUInt128OrNull"; }; -struct NameToUInt256OrNull { static constexpr auto name = "toUInt256OrNull"; }; -struct NameToInt8OrNull { static constexpr auto name = "toInt8OrNull"; }; -struct NameToInt16OrNull { static constexpr auto name = "toInt16OrNull"; }; -struct NameToInt32OrNull { static constexpr auto name = "toInt32OrNull"; }; -struct NameToInt64OrNull { static constexpr auto name = "toInt64OrNull"; }; -struct NameToInt128OrNull { static constexpr auto name = "toInt128OrNull"; }; -struct NameToInt256OrNull { static constexpr auto name = "toInt256OrNull"; }; -struct NameToBFloat16OrNull { static constexpr auto name = "toBFloat16OrNull"; }; -struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; }; -struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; }; -struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; }; -struct NameToDate32OrNull { static constexpr auto name = "toDate32OrNull"; }; -struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; }; -struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; }; -struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; }; -struct NameToDecimal64OrNull { static constexpr auto name = "toDecimal64OrNull"; }; -struct NameToDecimal128OrNull { static constexpr auto name = "toDecimal128OrNull"; }; -struct NameToDecimal256OrNull { static constexpr auto name = "toDecimal256OrNull"; }; -struct NameToUUIDOrNull { static constexpr auto name = "toUUIDOrNull"; }; -struct NameToIPv4OrNull { static constexpr auto name = "toIPv4OrNull"; }; -struct NameToIPv6OrNull { static constexpr auto name = "toIPv6OrNull"; }; - -using FunctionToUInt8OrNull = FunctionConvertFromString; -using FunctionToUInt16OrNull = FunctionConvertFromString; -using FunctionToUInt32OrNull = FunctionConvertFromString; -using FunctionToUInt64OrNull = FunctionConvertFromString; -using FunctionToUInt128OrNull = FunctionConvertFromString; -using FunctionToUInt256OrNull = FunctionConvertFromString; -using FunctionToInt8OrNull = FunctionConvertFromString; -using FunctionToInt16OrNull = FunctionConvertFromString; -using FunctionToInt32OrNull = FunctionConvertFromString; -using FunctionToInt64OrNull = FunctionConvertFromString; -using FunctionToInt128OrNull = FunctionConvertFromString; -using FunctionToInt256OrNull = FunctionConvertFromString; -using FunctionToBFloat16OrNull = FunctionConvertFromString; -using FunctionToFloat32OrNull = FunctionConvertFromString; -using FunctionToFloat64OrNull = FunctionConvertFromString; -using FunctionToDateOrNull = FunctionConvertFromString; -using FunctionToDate32OrNull = FunctionConvertFromString; -using FunctionToDateTimeOrNull = FunctionConvertFromString; -using FunctionToDateTime64OrNull = FunctionConvertFromString; -using FunctionToDecimal32OrNull = FunctionConvertFromString, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>; -using FunctionToDecimal64OrNull = FunctionConvertFromString, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>; -using FunctionToDecimal128OrNull = FunctionConvertFromString, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>; -using FunctionToDecimal256OrNull = FunctionConvertFromString, NameToDecimal256OrNull, ConvertFromStringExceptionMode::Null>; -using FunctionToUUIDOrNull = FunctionConvertFromString; -using FunctionToIPv4OrNull = FunctionConvertFromString; -using FunctionToIPv6OrNull = FunctionConvertFromString; - -struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; }; -struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; }; -struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; }; -struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; }; -struct NameParseDateTimeBestEffortUSOrZero { static constexpr auto name = "parseDateTimeBestEffortUSOrZero"; }; -struct NameParseDateTimeBestEffortUSOrNull { static constexpr auto name = "parseDateTimeBestEffortUSOrNull"; }; -struct NameParseDateTime32BestEffort { static constexpr auto name = "parseDateTime32BestEffort"; }; -struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = "parseDateTime32BestEffortOrZero"; }; -struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = "parseDateTime32BestEffortOrNull"; }; -struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; }; -struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; }; -struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; }; -struct NameParseDateTime64BestEffortUS { static constexpr auto name = "parseDateTime64BestEffortUS"; }; -struct NameParseDateTime64BestEffortUSOrZero { static constexpr auto name = "parseDateTime64BestEffortUSOrZero"; }; -struct NameParseDateTime64BestEffortUSOrNull { static constexpr auto name = "parseDateTime64BestEffortUSOrNull"; }; - - -using FunctionParseDateTimeBestEffort = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; - -using FunctionParseDateTimeBestEffortUS = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; -using FunctionParseDateTimeBestEffortUSOrZero = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; -using FunctionParseDateTimeBestEffortUSOrNull = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTimeBestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; - -using FunctionParseDateTime32BestEffort = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTime32BestEffortOrNull = FunctionConvertFromString< - DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; - -using FunctionParseDateTime64BestEffort = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; -using FunctionParseDateTime64BestEffortOrNull = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; - -using FunctionParseDateTime64BestEffortUS = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; -using FunctionParseDateTime64BestEffortUSOrZero = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; -using FunctionParseDateTime64BestEffortUSOrNull = FunctionConvertFromString< - DataTypeDateTime64, NameParseDateTime64BestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; - - -class ExecutableFunctionCast : public IExecutableFunction -{ -public: - using WrapperType = std::function; - - explicit ExecutableFunctionCast( - WrapperType && wrapper_function_, const char * name_, std::optional diagnostic_) - : wrapper_function(std::move(wrapper_function_)), name(name_), diagnostic(std::move(diagnostic_)) {} - - String getName() const override { return name; } - -protected: - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - /// drop second argument, pass others - ColumnsWithTypeAndName new_arguments{arguments.front()}; - if (arguments.size() > 2) - new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments)); - - try - { - return wrapper_function(new_arguments, result_type, nullptr, input_rows_count); - } - catch (Exception & e) - { - if (diagnostic) - e.addMessage("while converting source column " + backQuoteIfNeed(diagnostic->column_from) + - " to destination column " + backQuoteIfNeed(diagnostic->column_to)); - throw; - } - } - - bool useDefaultImplementationForNulls() const override { return false; } - /// CAST(Nothing, T) -> T - bool useDefaultImplementationForNothing() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - -private: - WrapperType wrapper_function; - const char * name; - std::optional diagnostic; -}; - - -struct FunctionCastName -{ - static constexpr auto name = "CAST"; -}; - -class FunctionCast final : public IFunctionBase -{ -public: - using MonotonicityForRange = std::function; - using WrapperType = std::function; - - FunctionCast(ContextPtr context_ - , const char * cast_name_ - , MonotonicityForRange && monotonicity_for_range_ - , const DataTypes & argument_types_ - , const DataTypePtr & return_type_ - , std::optional diagnostic_ - , CastType cast_type_) - : cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_)) - , argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_)) - , cast_type(cast_type_) - , context(context_) - { - } - - const DataTypes & getArgumentTypes() const override { return argument_types; } - const DataTypePtr & getResultType() const override { return return_type; } - - ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & /*sample_columns*/) const override - { - try - { - return std::make_unique( - prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), cast_name, diagnostic); - } - catch (Exception & e) - { - if (diagnostic) - e.addMessage("while converting source column " + backQuoteIfNeed(diagnostic->column_from) + - " to destination column " + backQuoteIfNeed(diagnostic->column_to)); - throw; - } - } - - String getName() const override { return cast_name; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - bool hasInformationAboutMonotonicity() const override - { - return static_cast(monotonicity_for_range); - } - - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override - { - return monotonicity_for_range(type, left, right); - } - -private: - const char * cast_name; - MonotonicityForRange monotonicity_for_range; - - DataTypes argument_types; - DataTypePtr return_type; - - std::optional diagnostic; - CastType cast_type; - ContextPtr context; - - static WrapperType createFunctionAdaptor(FunctionPtr function, const DataTypePtr & from_type) - { - auto function_adaptor = std::make_unique(function)->build({ColumnWithTypeAndName{nullptr, from_type, ""}}); - - return [function_adaptor]( - ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) - { return function_adaptor->execute(arguments, result_type, input_rows_count, /* dry_run = */ false); }; - } - - static WrapperType createToNullableColumnWrapper() - { - return [] (ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) - { - ColumnPtr res = result_type->createColumn(); - ColumnUInt8::Ptr col_null_map_to = ColumnUInt8::create(input_rows_count, true); - return ColumnNullable::create(res->cloneResized(input_rows_count), std::move(col_null_map_to)); - }; - } - - template - WrapperType createWrapper(const DataTypePtr & from_type, const ToDataType * const to_type, bool requested_result_is_nullable) const - { - TypeIndex from_type_index = from_type->getTypeId(); - WhichDataType which(from_type_index); - TypeIndex to_type_index = to_type->getTypeId(); - WhichDataType to(to_type_index); - bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull) - && (which.isInt() || which.isUInt() || which.isFloat()); - can_apply_accurate_cast |= cast_type == CastType::accurate && which.isStringOrFixedString() && to.isNativeInteger(); - - FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; - if (context) - date_time_overflow_behavior = context->getSettingsRef()[Setting::date_time_overflow_behavior]; - - if (requested_result_is_nullable && checkAndGetDataType(from_type.get())) - { - /// In case when converting to Nullable type, we apply different parsing rule, - /// that will not throw an exception but return NULL in case of malformed input. - FunctionPtr function = FunctionConvertFromString::create(context); - return createFunctionAdaptor(function, from_type); - } - else if (!can_apply_accurate_cast) - { - FunctionPtr function = FunctionTo::Type::create(context); - return createFunctionAdaptor(function, from_type); - } - - return [wrapper_cast_type = cast_type, from_type_index, to_type, date_time_overflow_behavior] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) - { - ColumnPtr result_column; - auto res = callOnIndexAndDataType(from_type_index, [&](const auto & types) -> bool - { - using Types = std::decay_t; - using LeftDataType = typename Types::LeftType; - using RightDataType = typename Types::RightType; - - if constexpr (IsDataTypeNumber) - { - if constexpr (IsDataTypeDateOrDateTime) - { -#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ - case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ - result_column \ - = ConvertImpl:: \ - execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, ADDITIONS()); \ - break; - if (wrapper_cast_type == CastType::accurate) - { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateConvertStrategyAdditions) - } - } - else - { - switch (date_time_overflow_behavior) - { - GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) - GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateOrNullConvertStrategyAdditions) - } - } -#undef GENERATE_OVERFLOW_MODE_CASE - - return true; - } - else if constexpr (IsDataTypeNumber) - { - if (wrapper_cast_type == CastType::accurate) - { - result_column = ConvertImpl::execute( - arguments, - result_type, - input_rows_count, - BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, - AccurateConvertStrategyAdditions()); - } - else - { - result_column = ConvertImpl::execute( - arguments, - result_type, - input_rows_count, - BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, - AccurateOrNullConvertStrategyAdditions()); - } - - return true; - } - } - else if constexpr (IsDataTypeStringOrFixedString) - { - if constexpr (IsDataTypeNumber) - { - chassert(wrapper_cast_type == CastType::accurate); - result_column = ConvertImpl::execute( - arguments, - result_type, - input_rows_count, - BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, - AccurateConvertStrategyAdditions()); - } - return true; - } - - return false; - }); - - /// Additionally check if callOnIndexAndDataType wasn't called at all. - if (!res) - { - if (wrapper_cast_type == CastType::accurateOrNull) - { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); - return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); - } - else - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, - "Conversion from {} to {} is not supported", - from_type_index, to_type->getName()); - } - } - - return result_column; - }; - } - - template - WrapperType createBoolWrapper(const DataTypePtr & from_type, const ToDataType * const to_type, bool requested_result_is_nullable) const - { - if (checkAndGetDataType(from_type.get())) - { - if (cast_type == CastType::accurateOrNull) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); - }; - } - - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); - }; - } - - return createWrapper(from_type, to_type, requested_result_is_nullable); - } - - WrapperType createUInt8ToBoolWrapper(const DataTypePtr from_type, const DataTypePtr to_type) const - { - return [from_type, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr - { - /// Special case when we convert UInt8 column to Bool column. - /// both columns have type UInt8, but we shouldn't use identity wrapper, - /// because Bool column can contain only 0 and 1. - auto res_column = to_type->createColumn(); - const auto & data_from = checkAndGetColumn(*arguments[0].column).getData(); - auto & data_to = assert_cast(res_column.get())->getData(); - data_to.resize(data_from.size()); - for (size_t i = 0; i != data_from.size(); ++i) - data_to[i] = static_cast(data_from[i]); - return res_column; - }; - } - - WrapperType createStringWrapper(const DataTypePtr & from_type) const - { - FunctionPtr function = FunctionToString::create(context); - return createFunctionAdaptor(function, from_type); - } - - WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N) const - { - if (!isStringOrFixedString(from_type)) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CAST AS FixedString is only implemented for types String and FixedString"); - - bool exception_mode_null = cast_type == CastType::accurateOrNull; - return [exception_mode_null, N] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) - { - if (exception_mode_null) - return FunctionToFixedString::executeForN(arguments, N); - else - return FunctionToFixedString::executeForN(arguments, N); - }; - } - -#define GENERATE_INTERVAL_CASE(INTERVAL_KIND) \ - case IntervalKind::Kind::INTERVAL_KIND: \ - return createFunctionAdaptor(FunctionConvert::create(context), from_type); - - WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) const - { - switch (kind.kind) - { - GENERATE_INTERVAL_CASE(Nanosecond) - GENERATE_INTERVAL_CASE(Microsecond) - GENERATE_INTERVAL_CASE(Millisecond) - GENERATE_INTERVAL_CASE(Second) - GENERATE_INTERVAL_CASE(Minute) - GENERATE_INTERVAL_CASE(Hour) - GENERATE_INTERVAL_CASE(Day) - GENERATE_INTERVAL_CASE(Week) - GENERATE_INTERVAL_CASE(Month) - GENERATE_INTERVAL_CASE(Quarter) - GENERATE_INTERVAL_CASE(Year) - } - throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion to unexpected IntervalKind: {}", kind.toString()}; - } - -#undef GENERATE_INTERVAL_CASE - - template - requires IsDataTypeDecimal - WrapperType createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const - { - TypeIndex type_index = from_type->getTypeId(); - UInt32 scale = to_type->getScale(); - - WhichDataType which(type_index); - bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() - || which.isStringOrFixedString(); - if (!ok) - { - if (cast_type == CastType::accurateOrNull) - return createToNullableColumnWrapper(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", - from_type->getName(), to_type->getName()); - } - - auto wrapper_cast_type = cast_type; - - return [wrapper_cast_type, type_index, scale, to_type, requested_result_is_nullable] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *column_nullable, size_t input_rows_count) - { - ColumnPtr result_column; - auto res = callOnIndexAndDataType(type_index, [&](const auto & types) -> bool - { - using Types = std::decay_t; - using LeftDataType = typename Types::LeftType; - using RightDataType = typename Types::RightType; - - if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber && !std::is_same_v) - { - if (wrapper_cast_type == CastType::accurate) - { - AccurateConvertStrategyAdditions additions; - additions.scale = scale; - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions); - - return true; - } - else if (wrapper_cast_type == CastType::accurateOrNull) - { - AccurateOrNullConvertStrategyAdditions additions; - additions.scale = scale; - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions); - - return true; - } - } - else if constexpr (std::is_same_v) - { - if (requested_result_is_nullable) - { - /// Consistent with CAST(Nullable(String) AS Nullable(Numbers)) - /// In case when converting to Nullable type, we apply different parsing rule, - /// that will not throw an exception but return NULL in case of malformed input. - result_column = ConvertImpl::execute( - arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag, scale); - - return true; - } - } - - result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, scale); - - return true; - }); - - /// Additionally check if callOnIndexAndDataType wasn't called at all. - if (!res) - { - if (wrapper_cast_type == CastType::accurateOrNull) - { - auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); - return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); - } - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, - "Conversion from {} to {} is not supported", - type_index, to_type->getName()); - } - - return result_column; - }; - } - - WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type) const - { - /// Conversion from String through parsing. - if (checkAndGetDataType(from_type_untyped.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); - }; - } - else if (const auto * agg_type = checkAndGetDataType(from_type_untyped.get())) - { - if (agg_type->getFunction()->haveSameStateRepresentation(*to_type->getFunction())) - { - return [function = to_type->getFunction()]( - ColumnsWithTypeAndName & arguments, - const DataTypePtr & /* result_type */, - const ColumnNullable * /* nullable_source */, - size_t /*input_rows_count*/) -> ColumnPtr - { - const auto & argument_column = arguments.front(); - const auto * col_agg = checkAndGetColumn(argument_column.column.get()); - if (col_agg) - { - auto new_col_agg = ColumnAggregateFunction::create(*col_agg); - new_col_agg->set(function); - return new_col_agg; - } - else - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Illegal column {} for function CAST AS AggregateFunction", - argument_column.column->getName()); - } - }; - } - } - - if (cast_type == CastType::accurateOrNull) - return createToNullableColumnWrapper(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", - from_type_untyped->getName(), to_type->getName()); - } - - WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray & to_type) const - { - /// Conversion from String through parsing. - if (checkAndGetDataType(from_type_untyped.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); - }; - } - - DataTypePtr from_type_holder; - const auto * from_type = checkAndGetDataType(from_type_untyped.get()); - const auto * from_type_map = checkAndGetDataType(from_type_untyped.get()); - - /// Convert from Map - if (from_type_map) - { - /// Recreate array of unnamed tuples because otherwise it may work - /// unexpectedly while converting to array of named tuples. - from_type_holder = from_type_map->getNestedTypeWithUnnamedTuple(); - from_type = assert_cast(from_type_holder.get()); - } - - if (!from_type) - { - throw Exception(ErrorCodes::TYPE_MISMATCH, - "CAST AS Array can only be performed between same-dimensional Array, Map or String types"); - } - - DataTypePtr from_nested_type = from_type->getNestedType(); - - /// In query SELECT CAST([] AS Array(Array(String))) from type is Array(Nothing) - bool from_empty_array = isNothing(from_nested_type); - - if (from_type->getNumberOfDimensions() != to_type.getNumberOfDimensions() && !from_empty_array) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "CAST AS Array can only be performed between same-dimensional array types"); - - const DataTypePtr & to_nested_type = to_type.getNestedType(); - - /// Prepare nested type conversion - const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type); - - return [nested_function, from_nested_type, to_nested_type]( - ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr - { - const auto & argument_column = arguments.front(); - - const ColumnArray * col_array = nullptr; - - if (const ColumnMap * col_map = checkAndGetColumn(argument_column.column.get())) - col_array = &col_map->getNestedColumn(); - else - col_array = checkAndGetColumn(argument_column.column.get()); - - if (col_array) - { - /// create columns for converting nested column containing original and result columns - ColumnsWithTypeAndName nested_columns{{ col_array->getDataPtr(), from_nested_type, "" }}; - - /// convert nested column - auto result_column = nested_function(nested_columns, to_nested_type, nullable_source, nested_columns.front().column->size()); - - /// set converted nested column to result - return ColumnArray::create(result_column, col_array->getOffsetsPtr()); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Illegal column {} for function CAST AS Array", - argument_column.column->getName()); - } - }; - } - - using ElementWrappers = std::vector; - - ElementWrappers getElementWrappers(const DataTypes & from_element_types, const DataTypes & to_element_types) const - { - ElementWrappers element_wrappers; - element_wrappers.reserve(from_element_types.size()); - - /// Create conversion wrapper for each element in tuple - for (size_t i = 0; i < from_element_types.size(); ++i) - { - const DataTypePtr & from_element_type = from_element_types[i]; - const DataTypePtr & to_element_type = to_element_types[i]; - element_wrappers.push_back(prepareUnpackDictionaries(from_element_type, to_element_type)); - } - - return element_wrappers; - } - - WrapperType createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type) const - { - /// Conversion from String through parsing. - if (checkAndGetDataType(from_type_untyped.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); - }; - } - - const auto * from_type = checkAndGetDataType(from_type_untyped.get()); - if (!from_type) - throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types or from String.\n" - "Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName()); - - const auto & from_element_types = from_type->getElements(); - const auto & to_element_types = to_type->getElements(); - - std::vector element_wrappers; - std::vector> to_reverse_index; - - /// For named tuples allow conversions for tuples with - /// different sets of elements. If element exists in @to_type - /// and doesn't exist in @to_type it will be filled by default values. - if (from_type->haveExplicitNames() && to_type->haveExplicitNames()) - { - const auto & from_names = from_type->getElementNames(); - std::unordered_map from_positions; - from_positions.reserve(from_names.size()); - for (size_t i = 0; i < from_names.size(); ++i) - from_positions[from_names[i]] = i; - - const auto & to_names = to_type->getElementNames(); - element_wrappers.reserve(to_names.size()); - to_reverse_index.reserve(from_names.size()); - - for (size_t i = 0; i < to_names.size(); ++i) - { - auto it = from_positions.find(to_names[i]); - if (it != from_positions.end()) - { - element_wrappers.emplace_back(prepareUnpackDictionaries(from_element_types[it->second], to_element_types[i])); - to_reverse_index.emplace_back(it->second); - } - else - { - element_wrappers.emplace_back(); - to_reverse_index.emplace_back(); - } - } - } - else - { - if (from_element_types.size() != to_element_types.size()) - throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types " - "with the same number of elements or from String.\nLeft type: {}, right type: {}", - from_type->getName(), to_type->getName()); - - element_wrappers = getElementWrappers(from_element_types, to_element_types); - to_reverse_index.reserve(to_element_types.size()); - for (size_t i = 0; i < to_element_types.size(); ++i) - to_reverse_index.emplace_back(i); - } - - return [element_wrappers, from_element_types, to_element_types, to_reverse_index] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count) -> ColumnPtr - { - const auto * col = arguments.front().column.get(); - - size_t tuple_size = to_element_types.size(); - const ColumnTuple & column_tuple = typeid_cast(*col); - - Columns converted_columns(tuple_size); - - /// invoke conversion for each element - for (size_t i = 0; i < tuple_size; ++i) - { - if (to_reverse_index[i]) - { - size_t from_idx = *to_reverse_index[i]; - ColumnsWithTypeAndName element = {{column_tuple.getColumns()[from_idx], from_element_types[from_idx], "" }}; - converted_columns[i] = element_wrappers[i](element, to_element_types[i], nullable_source, input_rows_count); - } - else - { - converted_columns[i] = to_element_types[i]->createColumn()->cloneResized(input_rows_count); - } - } - - return ColumnTuple::create(converted_columns); - }; - } - - /// The case of: tuple([key1, key2, ..., key_n], [value1, value2, ..., value_n]) - WrapperType createTupleToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const - { - return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr - { - const auto * col = arguments.front().column.get(); - const auto & column_tuple = assert_cast(*col); - - Columns offsets(2); - Columns converted_columns(2); - for (size_t i = 0; i < 2; ++i) - { - const auto & column_array = assert_cast(column_tuple.getColumn(i)); - ColumnsWithTypeAndName element = {{column_array.getDataPtr(), from_kv_types[i], ""}}; - converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); - offsets[i] = column_array.getOffsetsPtr(); - } - - const auto & keys_offsets = assert_cast(*offsets[0]).getData(); - const auto & values_offsets = assert_cast(*offsets[1]).getData(); - if (keys_offsets != values_offsets) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "CAST AS Map can only be performed from tuple of arrays with equal sizes."); - - return ColumnMap::create(converted_columns[0], converted_columns[1], offsets[0]); - }; - } - - WrapperType createMapToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const - { - return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr - { - const auto * col = arguments.front().column.get(); - const auto & column_map = typeid_cast(*col); - const auto & nested_data = column_map.getNestedData(); - - Columns converted_columns(2); - for (size_t i = 0; i < 2; ++i) - { - ColumnsWithTypeAndName element = {{nested_data.getColumnPtr(i), from_kv_types[i], ""}}; - converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); - } - - return ColumnMap::create(converted_columns[0], converted_columns[1], column_map.getNestedColumn().getOffsetsPtr()); - }; - } - - /// The case of: [(key1, value1), (key2, value2), ...] - WrapperType createArrayToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const - { - return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr - { - const auto * col = arguments.front().column.get(); - const auto & column_array = typeid_cast(*col); - const auto & nested_data = typeid_cast(column_array.getData()); - - Columns converted_columns(2); - for (size_t i = 0; i < 2; ++i) - { - ColumnsWithTypeAndName element = {{nested_data.getColumnPtr(i), from_kv_types[i], ""}}; - converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); - } - - return ColumnMap::create(converted_columns[0], converted_columns[1], column_array.getOffsetsPtr()); - }; - } - - - WrapperType createMapWrapper(const DataTypePtr & from_type_untyped, const DataTypeMap * to_type) const - { - if (const auto * from_tuple = checkAndGetDataType(from_type_untyped.get())) - { - if (from_tuple->getElements().size() != 2) - throw Exception( - ErrorCodes::TYPE_MISMATCH, - "CAST AS Map from tuple requires 2 elements. " - "Left type: {}, right type: {}", - from_tuple->getName(), - to_type->getName()); - - DataTypes from_kv_types; - const auto & to_kv_types = to_type->getKeyValueTypes(); - - for (const auto & elem : from_tuple->getElements()) - { - const auto * type_array = checkAndGetDataType(elem.get()); - if (!type_array) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "CAST AS Map can only be performed from tuples of array. Got: {}", from_tuple->getName()); - - from_kv_types.push_back(type_array->getNestedType()); - } - - return createTupleToMapWrapper(from_kv_types, to_kv_types); - } - else if (const auto * from_array = typeid_cast(from_type_untyped.get())) - { - if (typeid_cast(from_array->getNestedType().get())) - return [nested = to_type->getNestedType()](ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t size) - { - return ColumnMap::create(nested->createColumnConstWithDefaultValue(size)->convertToFullColumnIfConst()); - }; - - const auto * nested_tuple = typeid_cast(from_array->getNestedType().get()); - if (!nested_tuple || nested_tuple->getElements().size() != 2) - throw Exception( - ErrorCodes::TYPE_MISMATCH, - "CAST AS Map from array requires nested tuple of 2 elements. " - "Left type: {}, right type: {}", - from_array->getName(), - to_type->getName()); - - return createArrayToMapWrapper(nested_tuple->getElements(), to_type->getKeyValueTypes()); - } - else if (const auto * from_type = checkAndGetDataType(from_type_untyped.get())) - { - return createMapToMapWrapper(from_type->getKeyValueTypes(), to_type->getKeyValueTypes()); - } - else - { - throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported types to CAST AS Map. " - "Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName()); - } - } - - WrapperType createTupleToObjectDeprecatedWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const - { - if (!from_tuple.haveExplicitNames()) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_tuple.getName()); - - PathsInData paths; - DataTypes from_types; - - std::tie(paths, from_types) = flattenTuple(from_tuple.getPtr()); - auto to_types = from_types; - - for (auto & type : to_types) - { - if (isTuple(type) || isNested(type)) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Cast to Object can be performed only from flatten Named Tuple. Got: {}", - from_tuple.getName()); - - type = recursiveRemoveLowCardinality(type); - } - - return [element_wrappers = getElementWrappers(from_types, to_types), - has_nullable_subcolumns, from_types, to_types, paths] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count) - { - size_t tuple_size = to_types.size(); - auto flattened_column = flattenTuple(arguments.front().column); - const auto & column_tuple = assert_cast(*flattened_column); - - if (tuple_size != column_tuple.getColumns().size()) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Expected tuple with {} subcolumn, but got {} subcolumns", - tuple_size, column_tuple.getColumns().size()); - - auto res = ColumnObjectDeprecated::create(has_nullable_subcolumns); - for (size_t i = 0; i < tuple_size; ++i) - { - ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }}; - auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count); - res->addSubcolumn(paths[i], converted_column->assumeMutable()); - } - - return res; - }; - } - - WrapperType createMapToObjectDeprecatedWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const - { - auto key_value_types = from_map.getKeyValueTypes(); - - if (!isStringOrFixedString(key_value_types[0])) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Cast to Object from Map can be performed only from Map " - "with String or FixedString key. Got: {}", from_map.getName()); - - const auto & value_type = key_value_types[1]; - auto to_value_type = value_type; - - if (!has_nullable_subcolumns && value_type->isNullable()) - to_value_type = removeNullable(value_type); - - if (has_nullable_subcolumns && !value_type->isNullable()) - to_value_type = makeNullable(value_type); - - DataTypes to_key_value_types{std::make_shared(), std::move(to_value_type)}; - auto element_wrappers = getElementWrappers(key_value_types, to_key_value_types); - - return [has_nullable_subcolumns, element_wrappers, key_value_types, to_key_value_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t) -> ColumnPtr - { - const auto & column_map = assert_cast(*arguments.front().column); - const auto & offsets = column_map.getNestedColumn().getOffsets(); - auto key_value_columns = column_map.getNestedData().getColumnsCopy(); - - for (size_t i = 0; i < 2; ++i) - { - ColumnsWithTypeAndName element{{key_value_columns[i], key_value_types[i], ""}}; - key_value_columns[i] = element_wrappers[i](element, to_key_value_types[i], nullable_source, key_value_columns[i]->size()); - } - - const auto & key_column_str = assert_cast(*key_value_columns[0]); - const auto & value_column = *key_value_columns[1]; - - using SubcolumnsMap = HashMap; - SubcolumnsMap subcolumns; - - for (size_t row = 0; row < offsets.size(); ++row) - { - for (size_t i = offsets[static_cast(row) - 1]; i < offsets[row]; ++i) - { - auto ref = key_column_str.getDataAt(i); - - bool inserted; - SubcolumnsMap::LookupResult it; - subcolumns.emplace(ref, it, inserted); - auto & subcolumn = it->getMapped(); - - if (inserted) - subcolumn = value_column.cloneEmpty()->cloneResized(row); - - /// Map can have duplicated keys. We insert only first one. - if (subcolumn->size() == row) - subcolumn->insertFrom(value_column, i); - } - - /// Insert default values for keys missed in current row. - for (const auto & [_, subcolumn] : subcolumns) - if (subcolumn->size() == row) - subcolumn->insertDefault(); - } - - auto column_object = ColumnObjectDeprecated::create(has_nullable_subcolumns); - for (auto && [key, subcolumn] : subcolumns) - { - PathInData path(key.toView()); - column_object->addSubcolumn(path, std::move(subcolumn)); - } - - return column_object; - }; - } - - WrapperType createObjectDeprecatedWrapper(const DataTypePtr & from_type, const DataTypeObjectDeprecated * to_type) const - { - if (const auto * from_tuple = checkAndGetDataType(from_type.get())) - { - return createTupleToObjectDeprecatedWrapper(*from_tuple, to_type->hasNullableSubcolumns()); - } - else if (const auto * from_map = checkAndGetDataType(from_type.get())) - { - return createMapToObjectDeprecatedWrapper(*from_map, to_type->hasNullableSubcolumns()); - } - else if (checkAndGetDataType(from_type.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) - { - auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context)->assumeMutable(); - res->finalize(); - return res; - }; - } - else if (checkAndGetDataType(from_type.get())) - { - return [is_nullable = to_type->hasNullableSubcolumns()] (ColumnsWithTypeAndName & arguments, const DataTypePtr & , const ColumnNullable * , size_t) -> ColumnPtr - { - const auto & column_object = assert_cast(*arguments.front().column); - auto res = ColumnObjectDeprecated::create(is_nullable); - for (size_t i = 0; i < column_object.size(); i++) - res->insert(column_object[i]); - - res->finalize(); - return res; - }; - } - - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); - } - - - WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_object) const - { - if (checkAndGetDataType(from_type.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) - { - return ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context); - }; - } - - /// Cast Tuple/Object/Map to JSON type through serializing into JSON string and parsing back into JSON column. - /// Potentially we can do smarter conversion Tuple -> JSON with type preservation, but it's questionable how exactly Tuple should be - /// converted to JSON (for example, should we recursively convert nested Array(Tuple) to Array(JSON) or not, should we infer types from String fields, etc). - if (checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get())) - { - return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) - { - auto json_string = ColumnString::create(); - ColumnStringHelpers::WriteHelper write_helper(assert_cast(*json_string), input_rows_count); - auto & write_buffer = write_helper.getWriteBuffer(); - FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; - auto serialization = arguments[0].type->getDefaultSerialization(); - for (size_t i = 0; i < input_rows_count; ++i) - { - serialization->serializeTextJSON(*arguments[0].column, i, write_buffer, format_settings); - write_helper.rowWritten(); - } - write_helper.finalize(); - - ColumnsWithTypeAndName args_with_json_string = {ColumnWithTypeAndName(json_string->getPtr(), std::make_shared(), "")}; - return ConvertImplGenericFromString::execute(args_with_json_string, result_type, nullable_source, input_rows_count, context); - }; - } - - /// TODO: support CAST between JSON types with different parameters - throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String/Map/Object/Tuple. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); - } - - WrapperType createVariantToVariantWrapper(const DataTypeVariant & from_variant, const DataTypeVariant & to_variant) const - { - /// We support only extension of variant type, so, only new types can be added. - /// For example: Variant(T1, T2) -> Variant(T1, T2, T3) is supported, but Variant(T1, T2) -> Variant(T1, T3) is not supported. - /// We want to extend Variant type for free without rewriting the data, but we sort data types inside Variant during type creation - /// (we do it because we want Variant(T1, T2) to be the same as Variant(T2, T1)), but after extension the order of variant types - /// (and so their discriminators) can be different. For example: Variant(T1, T3) -> Variant(T1, T2, T3). - /// To avoid full rewrite of discriminators column, ColumnVariant supports it's local order of variant columns (and so local - /// discriminators) and stores mapping global order -> local order. - /// So, to extend Variant with new types for free, we should keep old local order for old variants, append new variants and change - /// mapping global order -> local order according to the new global order. - - /// Create map (new variant type) -> (it's global discriminator in new order). - const auto & new_variants = to_variant.getVariants(); - std::unordered_map new_variant_types_to_new_global_discriminator; - new_variant_types_to_new_global_discriminator.reserve(new_variants.size()); - for (size_t i = 0; i != new_variants.size(); ++i) - new_variant_types_to_new_global_discriminator[new_variants[i]->getName()] = i; - - /// Create set of old variant types. - const auto & old_variants = from_variant.getVariants(); - std::unordered_map old_variant_types_to_old_global_discriminator; - old_variant_types_to_old_global_discriminator.reserve(old_variants.size()); - for (size_t i = 0; i != old_variants.size(); ++i) - old_variant_types_to_old_global_discriminator[old_variants[i]->getName()] = i; - - /// Check that the set of old variants types is a subset of new variant types and collect new global discriminator for each old global discriminator. - std::unordered_map old_global_discriminator_to_new; - old_global_discriminator_to_new.reserve(old_variants.size()); - for (const auto & [old_variant_type, old_discriminator] : old_variant_types_to_old_global_discriminator) - { - auto it = new_variant_types_to_new_global_discriminator.find(old_variant_type); - if (it == new_variant_types_to_new_global_discriminator.end()) - throw Exception( - ErrorCodes::CANNOT_CONVERT_TYPE, - "Cannot convert type {} to {}. Conversion between Variant types is allowed only when new Variant type is an extension " - "of an initial one", from_variant.getName(), to_variant.getName()); - old_global_discriminator_to_new[old_discriminator] = it->second; - } - - /// Collect variant types and their global discriminators that should be added to the old Variant to get the new Variant. - std::vector> variant_types_and_discriminators_to_add; - variant_types_and_discriminators_to_add.reserve(new_variants.size() - old_variants.size()); - for (size_t i = 0; i != new_variants.size(); ++i) - { - if (!old_variant_types_to_old_global_discriminator.contains(new_variants[i]->getName())) - variant_types_and_discriminators_to_add.emplace_back(new_variants[i], i); - } - - return [old_global_discriminator_to_new, variant_types_and_discriminators_to_add] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & column_variant = assert_cast(*arguments.front().column.get()); - size_t num_old_variants = column_variant.getNumVariants(); - Columns new_variant_columns; - new_variant_columns.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); - std::vector new_local_to_global_discriminators; - new_local_to_global_discriminators.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); - for (size_t i = 0; i != num_old_variants; ++i) - { - new_variant_columns.push_back(column_variant.getVariantPtrByLocalDiscriminator(i)); - new_local_to_global_discriminators.push_back(old_global_discriminator_to_new.at(column_variant.globalDiscriminatorByLocal(i))); - } - - for (const auto & [new_variant_type, new_global_discriminator] : variant_types_and_discriminators_to_add) - { - new_variant_columns.push_back(new_variant_type->createColumn()); - new_local_to_global_discriminators.push_back(new_global_discriminator); - } - - return ColumnVariant::create(column_variant.getLocalDiscriminatorsPtr(), column_variant.getOffsetsPtr(), new_variant_columns, new_local_to_global_discriminators); - }; - } - - /// Create wrapper only if we support this conversion. - WrapperType createWrapperIfCanConvert(const DataTypePtr & from, const DataTypePtr & to) const - { - try - { - /// We can avoid try/catch here if we will implement check that 2 types can be cast, but it - /// requires quite a lot of work. By now let's simply use try/catch. - /// First, check that we can create a wrapper. - WrapperType wrapper = prepareUnpackDictionaries(from, to); - /// Second, check if we can perform a conversion on column with default value. - /// (we cannot just check empty column as we do some checks only during iteration over rows). - auto test_col = from->createColumn(); - test_col->insertDefault(); - ColumnsWithTypeAndName column_from = {{test_col->getPtr(), from, "" }}; - wrapper(column_from, to, nullptr, 1); - return wrapper; - } - catch (const Exception &) - { - return {}; - } - } - - WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const - { - const auto & variant_types = from_variant.getVariants(); - std::vector variant_wrappers; - variant_wrappers.reserve(variant_types.size()); - - /// Create conversion wrapper for each variant. - for (const auto & variant_type : variant_types) - { - WrapperType wrapper; - if (cast_type == CastType::accurateOrNull) - { - /// Create wrapper only if we support conversion from variant to the resulting type. - wrapper = createWrapperIfCanConvert(variant_type, to_type); - } - else - { - wrapper = prepareUnpackDictionaries(variant_type, to_type); - } - variant_wrappers.push_back(wrapper); - } - - return [variant_wrappers, variant_types, to_type] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - const auto & column_variant = assert_cast(*arguments.front().column.get()); - - /// First, cast each variant to the result type. - std::vector cast_variant_columns; - cast_variant_columns.reserve(variant_types.size()); - for (size_t i = 0; i != variant_types.size(); ++i) - { - auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i); - ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }}; - const auto & variant_wrapper = variant_wrappers[i]; - ColumnPtr cast_variant; - /// Check if we have wrapper for this variant. - if (variant_wrapper) - cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - cast_variant_columns.push_back(std::move(cast_variant)); - } - - /// Second, construct resulting column from cast variant columns according to discriminators. - const auto & local_discriminators = column_variant.getLocalDiscriminators(); - auto res = result_type->createColumn(); - res->reserve(input_rows_count); - for (size_t i = 0; i != input_rows_count; ++i) - { - auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !cast_variant_columns[global_discr]) - res->insertDefault(); - else - res->insertFrom(*cast_variant_columns[global_discr], column_variant.offsetAt(i)); - } - - return res; - }; - } - - static ColumnPtr createVariantFromDescriptorsAndOneNonEmptyVariant(const DataTypes & variant_types, const ColumnPtr & discriminators, const ColumnPtr & variant, ColumnVariant::Discriminator variant_discr) - { - Columns variants; - variants.reserve(variant_types.size()); - for (size_t i = 0; i != variant_types.size(); ++i) - { - if (i == variant_discr) - variants.emplace_back(variant); - else - variants.push_back(variant_types[i]->createColumn()); - } - - return ColumnVariant::create(discriminators, variants); - } - - WrapperType createStringToVariantWrapper() const - { - return [&](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - auto column = arguments[0].column->convertToFullColumnIfLowCardinality(); - auto args = arguments; - args[0].column = column; - - const ColumnNullable * column_nullable = nullptr; - if (isColumnNullable(*args[0].column)) - { - column_nullable = assert_cast(args[0].column.get()); - args[0].column = column_nullable->getNestedColumnPtr(); - } - - args[0].type = removeNullable(removeLowCardinality(args[0].type)); - - if (cast_type == CastType::accurateOrNull) - return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); - return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); - }; - } - - WrapperType createColumnToVariantWrapper(const DataTypePtr & from_type, const DataTypeVariant & to_variant) const - { - /// We allow converting NULL to Variant(...) as Variant can store NULLs. - if (from_type->onlyNull()) - { - return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - auto result_column = result_type->createColumn(); - result_column->insertManyDefaults(input_rows_count); - return result_column; - }; - } - - auto variant_discr_opt = to_variant.tryGetVariantDiscriminator(removeNullableOrLowCardinalityNullable(from_type)->getName()); - /// Cast String to Variant through parsing if it's not Variant(String). - if (isStringOrFixedString(removeNullable(removeLowCardinality(from_type))) && (!variant_discr_opt || to_variant.getVariants().size() > 1)) - return createStringToVariantWrapper(); - - if (!variant_discr_opt) - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert type {} to {}. Conversion to Variant allowed only for types from this Variant", from_type->getName(), to_variant.getName()); - - return [variant_discr = *variant_discr_opt] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & result_variant_type = assert_cast(*result_type); - const auto & variant_types = result_variant_type.getVariants(); - if (const ColumnNullable * col_nullable = typeid_cast(arguments.front().column.get())) - { - const auto & column = col_nullable->getNestedColumnPtr(); - const auto & null_map = col_nullable->getNullMapData(); - IColumn::Filter filter; - filter.reserve(column->size()); - auto discriminators = ColumnVariant::ColumnDiscriminators::create(); - auto & discriminators_data = discriminators->getData(); - discriminators_data.reserve(column->size()); - size_t variant_size_hint = 0; - for (size_t i = 0; i != column->size(); ++i) - { - if (null_map[i]) - { - discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); - filter.push_back(0); - } - else - { - discriminators_data.push_back(variant_discr); - filter.push_back(1); - ++variant_size_hint; - } - } - - ColumnPtr variant_column; - /// If there were no NULLs, just use the column. - if (variant_size_hint == column->size()) - variant_column = column; - /// Otherwise we should use filtered column. - else - variant_column = column->filter(filter, variant_size_hint); - return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), variant_column, variant_discr); - } - else if (isColumnLowCardinalityNullable(*arguments.front().column)) - { - const auto & column = arguments.front().column; - - /// Variant column cannot have LowCardinality(Nullable(...)) variant, as Variant column stores NULLs itself. - /// We should create a null-map, insert NULL_DISCRIMINATOR on NULL values and filter initial column. - const auto & col_lc = assert_cast(*column); - const auto & indexes = col_lc.getIndexes(); - auto null_index = col_lc.getDictionary().getNullValueIndex(); - IColumn::Filter filter; - filter.reserve(col_lc.size()); - auto discriminators = ColumnVariant::ColumnDiscriminators::create(); - auto & discriminators_data = discriminators->getData(); - discriminators_data.reserve(col_lc.size()); - size_t variant_size_hint = 0; - for (size_t i = 0; i != col_lc.size(); ++i) - { - if (indexes.getUInt(i) == null_index) - { - discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); - filter.push_back(0); - } - else - { - discriminators_data.push_back(variant_discr); - filter.push_back(1); - ++variant_size_hint; - } - } - - MutableColumnPtr variant_column; - /// If there were no NULLs, we can just clone the column. - if (variant_size_hint == col_lc.size()) - variant_column = IColumn::mutate(column); - /// Otherwise we should filter column. - else - variant_column = IColumn::mutate(column->filter(filter, variant_size_hint)); - - assert_cast(*variant_column).nestedRemoveNullable(); - return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), std::move(variant_column), variant_discr); - } - else - { - const auto & column = arguments.front().column; - auto discriminators = ColumnVariant::ColumnDiscriminators::create(); - discriminators->getData().resize_fill(column->size(), variant_discr); - return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), column, variant_discr); - } - }; - } - - /// Wrapper for conversion to/from Variant type - WrapperType createVariantWrapper(const DataTypePtr & from_type, const DataTypePtr & to_type) const - { - if (const auto * from_variant = checkAndGetDataType(from_type.get())) - { - if (const auto * to_variant = checkAndGetDataType(to_type.get())) - return createVariantToVariantWrapper(*from_variant, *to_variant); - - return createVariantToColumnWrapper(*from_variant, to_type); - } - - return createColumnToVariantWrapper(from_type, assert_cast(*to_type)); - } - - WrapperType createDynamicToColumnWrapper(const DataTypePtr &) const - { - return [this] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - /// When casting Dynamic to regular column we should cast all variants from current Dynamic column - /// and construct the result based on discriminators. - const auto & column_dynamic = assert_cast(*arguments.front().column.get()); - const auto & variant_column = column_dynamic.getVariantColumn(); - const auto & variant_info = column_dynamic.getVariantInfo(); - - /// First, cast usual variants to result type. - const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); - std::vector cast_variant_columns; - cast_variant_columns.reserve(variant_types.size()); - for (size_t i = 0; i != variant_types.size(); ++i) - { - /// Skip shared variant, it will be processed later. - if (i == column_dynamic.getSharedVariantDiscriminator()) - { - cast_variant_columns.push_back(nullptr); - continue; - } - - const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); - ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; - WrapperType variant_wrapper; - if (cast_type == CastType::accurateOrNull) - /// Create wrapper only if we support conversion from variant to the resulting type. - variant_wrapper = createWrapperIfCanConvert(variant_types[i], result_type); - else - variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); - - ColumnPtr cast_variant; - /// Check if we have wrapper for this variant. - if (variant_wrapper) - cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); - cast_variant_columns.push_back(cast_variant); - } - - /// Second, collect all variants stored in shared variant and cast them to result type. - std::vector variant_columns_from_shared_variant; - DataTypes variant_types_from_shared_variant; - /// We will need to know what variant to use when we see discriminator of a shared variant. - /// To do it, we remember what variant was extracted from each row and what was it's offset. - PaddedPODArray shared_variant_indexes; - PaddedPODArray shared_variant_offsets; - std::unordered_map shared_variant_to_index; - const auto & shared_variant = column_dynamic.getSharedVariant(); - const auto shared_variant_discr = column_dynamic.getSharedVariantDiscriminator(); - const auto & local_discriminators = variant_column.getLocalDiscriminators(); - const auto & offsets = variant_column.getOffsets(); - if (!shared_variant.empty()) - { - shared_variant_indexes.reserve(input_rows_count); - shared_variant_offsets.reserve(input_rows_count); - FormatSettings format_settings; - const auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(shared_variant_discr); - for (size_t i = 0; i != input_rows_count; ++i) - { - if (local_discriminators[i] == shared_variant_local_discr) - { - auto value = shared_variant.getDataAt(offsets[i]); - ReadBufferFromMemory buf(value.data, value.size); - auto type = decodeDataType(buf); - auto type_name = type->getName(); - auto it = shared_variant_to_index.find(type_name); - /// Check if we didn't create column for this variant yet. - if (it == shared_variant_to_index.end()) - { - it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; - variant_columns_from_shared_variant.push_back(type->createColumn()); - variant_types_from_shared_variant.push_back(type); - } - - shared_variant_indexes.push_back(it->second); - shared_variant_offsets.push_back(variant_columns_from_shared_variant[it->second]->size()); - type->getDefaultSerialization()->deserializeBinary(*variant_columns_from_shared_variant[it->second], buf, format_settings); - } - else - { - shared_variant_indexes.emplace_back(); - shared_variant_offsets.emplace_back(); - } - } - } - - /// Cast all extracted variants into result type. - std::vector cast_shared_variant_columns; - cast_shared_variant_columns.reserve(variant_types_from_shared_variant.size()); - for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) - { - ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; - WrapperType variant_wrapper; - if (cast_type == CastType::accurateOrNull) - /// Create wrapper only if we support conversion from variant to the resulting type. - variant_wrapper = createWrapperIfCanConvert(variant_types_from_shared_variant[i], result_type); - else - variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); - - ColumnPtr cast_variant; - /// Check if we have wrapper for this variant. - if (variant_wrapper) - cast_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size()); - cast_shared_variant_columns.push_back(cast_variant); - } - - /// Construct result column from all cast variants. - auto res = result_type->createColumn(); - res->reserve(input_rows_count); - for (size_t i = 0; i != input_rows_count; ++i) - { - auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) - { - res->insertDefault(); - } - else if (global_discr == shared_variant_discr) - { - if (cast_shared_variant_columns[shared_variant_indexes[i]]) - res->insertFrom(*cast_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); - else - res->insertDefault(); - } - else - { - if (cast_variant_columns[global_discr]) - res->insertFrom(*cast_variant_columns[global_discr], offsets[i]); - else - res->insertDefault(); - } - } - - return res; - }; - } - - WrapperType createStringToDynamicThroughParsingWrapper() const - { - return [&](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - auto column = arguments[0].column->convertToFullColumnIfLowCardinality(); - auto args = arguments; - args[0].column = column; - - const ColumnNullable * column_nullable = nullptr; - if (isColumnNullable(*args[0].column)) - { - column_nullable = assert_cast(args[0].column.get()); - args[0].column = column_nullable->getNestedColumnPtr(); - } - - args[0].type = removeNullable(removeLowCardinality(args[0].type)); - - if (cast_type == CastType::accurateOrNull) - return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); - return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); - }; - } - - WrapperType createVariantToDynamicWrapper(const DataTypeVariant & from_variant_type, const DataTypeDynamic & dynamic_type) const - { - /// First create extended Variant with shared variant type and cast this Variant to it. - auto variants_for_dynamic = from_variant_type.getVariants(); - size_t number_of_variants = variants_for_dynamic.size(); - variants_for_dynamic.push_back(ColumnDynamic::getSharedVariantDataType()); - const auto & variant_type_for_dynamic = std::make_shared(variants_for_dynamic); - auto old_to_new_variant_wrapper = createVariantToVariantWrapper(from_variant_type, *variant_type_for_dynamic); - auto max_dynamic_types = dynamic_type.getMaxDynamicTypes(); - return [old_to_new_variant_wrapper, variant_type_for_dynamic, number_of_variants, max_dynamic_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr - { - auto variant_column_for_dynamic = old_to_new_variant_wrapper(arguments, result_type, col_nullable, input_rows_count); - /// If resulting Dynamic column can contain all variants from this Variant column, just create Dynamic column from it. - if (max_dynamic_types >= number_of_variants) - return ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, max_dynamic_types, max_dynamic_types); - - /// Otherwise some variants should go to the shared variant. Create temporary Dynamic column from this Variant and insert - /// all data to the resulting Dynamic column, this insertion will do all the logic with shared variant. - auto tmp_dynamic_column = ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, number_of_variants, number_of_variants); - auto result_dynamic_column = ColumnDynamic::create(max_dynamic_types); - result_dynamic_column->insertRangeFrom(*tmp_dynamic_column, 0, tmp_dynamic_column->size()); - return result_dynamic_column; - }; - } - - WrapperType createColumnToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const - { - if (const auto * variant_type = typeid_cast(from_type.get())) - return createVariantToDynamicWrapper(*variant_type, dynamic_type); - - if (context && context->getSettingsRef()[Setting::cast_string_to_dynamic_use_inference] && isStringOrFixedString(removeNullable(removeLowCardinality(from_type)))) - return createStringToDynamicThroughParsingWrapper(); - - /// First, cast column to Variant with 2 variants - the type of the column we cast and shared variant type. - auto variant_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(from_type)}); - auto column_to_variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); - /// Second, cast this Variant to Dynamic. - auto variant_to_dynamic_wrapper = createVariantToDynamicWrapper(*variant_type, dynamic_type); - return [column_to_variant_wrapper, variant_to_dynamic_wrapper, variant_type] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr - { - auto variant_res = column_to_variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); - ColumnsWithTypeAndName args = {{variant_res, variant_type, ""}}; - return variant_to_dynamic_wrapper(args, result_type, nullptr, input_rows_count); - }; - } - - WrapperType createDynamicToDynamicWrapper(const DataTypeDynamic & from_dynamic, const DataTypeDynamic & to_dynamic) const - { - size_t from_max_types = from_dynamic.getMaxDynamicTypes(); - size_t to_max_types = to_dynamic.getMaxDynamicTypes(); - if (from_max_types == to_max_types) - return createIdentityWrapper(from_dynamic.getPtr()); - - if (to_max_types > from_max_types) - { - return [to_max_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & dynamic_column = assert_cast(*arguments[0].column); - /// We should use the same limit as already used in column and change only global limit. - /// It's needed because shared variant should contain values only when limit is exceeded, - /// so if there are already some data, we cannot increase the limit. - return ColumnDynamic::create(dynamic_column.getVariantColumnPtr(), dynamic_column.getVariantInfo(), dynamic_column.getMaxDynamicTypes(), to_max_types); - }; - } - - return [to_max_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr - { - const auto & dynamic_column = assert_cast(*arguments[0].column); - /// If real limit in the column is not greater than desired, just use the same variant column. - if (dynamic_column.getMaxDynamicTypes() <= to_max_types) - return ColumnDynamic::create(dynamic_column.getVariantColumnPtr(), dynamic_column.getVariantInfo(), dynamic_column.getMaxDynamicTypes(), to_max_types); - - /// Otherwise some variants should go to the shared variant. We try to keep the most frequent variants. - const auto & variant_info = dynamic_column.getVariantInfo(); - const auto & variants = assert_cast(*variant_info.variant_type).getVariants(); - const auto & statistics = dynamic_column.getStatistics(); - const auto & variant_column = dynamic_column.getVariantColumn(); - auto shared_variant_discr = dynamic_column.getSharedVariantDiscriminator(); - std::vector> variants_with_sizes; - variants_with_sizes.reserve(variant_info.variant_names.size()); - for (const auto & [name, discr] : variant_info.variant_name_to_discriminator) - { - /// Don't include shared variant. - if (discr == shared_variant_discr) - continue; - - size_t size = variant_column.getVariantByGlobalDiscriminator(discr).size(); - /// If column has statistics from the data part, use size from it for consistency. - /// It's important to keep the same dynamic structure of the result column during ALTER. - if (statistics) - { - auto statistics_it = statistics->variants_statistics.find(name); - if (statistics_it != statistics->variants_statistics.end()) - size = statistics_it->second; - } - variants_with_sizes.emplace_back(size, name, variants[discr]); - } - - std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); - DataTypes result_variants; - result_variants.reserve(to_max_types + 1); /// +1 for shared variant. - /// Add new variants from sorted list until we reach to_max_types. - for (const auto & [size, name, type] : variants_with_sizes) - { - if (result_variants.size() < to_max_types) - result_variants.push_back(type); - else - break; - } - - /// Add shared variant. - result_variants.push_back(ColumnDynamic::getSharedVariantDataType()); - /// Create resulting Variant type and Dynamic column. - auto result_variant_type = std::make_shared(result_variants); - auto result_dynamic_column = ColumnDynamic::create(result_variant_type->createColumn(), result_variant_type, to_max_types, to_max_types); - const auto & result_variant_info = result_dynamic_column->getVariantInfo(); - auto & result_variant_column = result_dynamic_column->getVariantColumn(); - auto result_shared_variant_discr = result_dynamic_column->getSharedVariantDiscriminator(); - /// Create mapping from old discriminators to the new ones. - std::vector old_to_new_discriminators; - old_to_new_discriminators.resize(variant_info.variant_name_to_discriminator.size(), result_shared_variant_discr); - for (const auto & [name, discr] : result_variant_info.variant_name_to_discriminator) - { - auto old_discr = variant_info.variant_name_to_discriminator.at(name); - old_to_new_discriminators[old_discr] = discr; - /// Reuse old variant column if it's not shared variant. - if (discr != result_shared_variant_discr) - result_variant_column.getVariantPtrByGlobalDiscriminator(discr) = variant_column.getVariantPtrByGlobalDiscriminator(old_discr); - } - - const auto & local_discriminators = variant_column.getLocalDiscriminators(); - const auto & offsets = variant_column.getOffsets(); - const auto & shared_variant = dynamic_column.getSharedVariant(); - auto & result_local_discriminators = result_variant_column.getLocalDiscriminators(); - result_local_discriminators.reserve(local_discriminators.size()); - auto & result_offsets = result_variant_column.getOffsets(); - result_offsets.reserve(offsets.size()); - auto & result_shared_variant = result_dynamic_column->getSharedVariant(); - for (size_t i = 0; i != local_discriminators.size(); ++i) - { - auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); - if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) - { - result_local_discriminators.push_back(ColumnVariant::NULL_DISCRIMINATOR); - result_offsets.emplace_back(); - } - else if (global_discr == shared_variant_discr) - { - result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_shared_variant_discr)); - result_offsets.push_back(result_shared_variant.size()); - result_shared_variant.insertFrom(shared_variant, offsets[i]); - } - else - { - auto result_global_discr = old_to_new_discriminators[global_discr]; - if (result_global_discr == result_shared_variant_discr) - { - result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_shared_variant_discr)); - result_offsets.push_back(result_shared_variant.size()); - ColumnDynamic::serializeValueIntoSharedVariant( - result_shared_variant, - variant_column.getVariantByGlobalDiscriminator(global_discr), - variants[global_discr], - variants[global_discr]->getDefaultSerialization(), - offsets[i]); - } - else - { - result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_global_discr)); - result_offsets.push_back(offsets[i]); - } - } - } - - return result_dynamic_column; - }; - } - - /// Wrapper for conversion to/from Dynamic type - WrapperType createDynamicWrapper(const DataTypePtr & from_type, const DataTypePtr & to_type) const - { - if (const auto * from_dynamic = checkAndGetDataType(from_type.get())) - { - if (const auto * to_dynamic = checkAndGetDataType(to_type.get())) - return createDynamicToDynamicWrapper(*from_dynamic, *to_dynamic); - - return createDynamicToColumnWrapper(to_type); - } - - return createColumnToDynamicWrapper(from_type, *checkAndGetDataType(to_type.get())); - } - - template - WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum * to_type) const - { - using EnumType = DataTypeEnum; - using Function = typename FunctionTo::Type; - - if (const auto * from_enum8 = checkAndGetDataType(from_type.get())) - checkEnumToEnumConversion(from_enum8, to_type); - else if (const auto * from_enum16 = checkAndGetDataType(from_type.get())) - checkEnumToEnumConversion(from_enum16, to_type); - - if (checkAndGetDataType(from_type.get())) - return createStringToEnumWrapper(); - else if (checkAndGetDataType(from_type.get())) - return createStringToEnumWrapper(); - else if (isNativeNumber(from_type) || isEnum(from_type)) - { - auto function = Function::create(context); - return createFunctionAdaptor(function, from_type); - } - else - { - if (cast_type == CastType::accurateOrNull) - return createToNullableColumnWrapper(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", - from_type->getName(), to_type->getName()); - } - } - - template - void checkEnumToEnumConversion(const EnumTypeFrom * from_type, const EnumTypeTo * to_type) const - { - const auto & from_values = from_type->getValues(); - const auto & to_values = to_type->getValues(); - - using ValueType = std::common_type_t; - using NameValuePair = std::pair; - using EnumValues = std::vector; - - EnumValues name_intersection; - std::set_intersection(std::begin(from_values), std::end(from_values), - std::begin(to_values), std::end(to_values), std::back_inserter(name_intersection), - [] (auto && from, auto && to) { return from.first < to.first; }); - - for (const auto & name_value : name_intersection) - { - const auto & old_value = name_value.second; - const auto & new_value = to_type->getValue(name_value.first); - if (old_value != new_value) - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Enum conversion changes value for element '{}' from {} to {}", - name_value.first, toString(old_value), toString(new_value)); - } - } - - template - WrapperType createStringToEnumWrapper() const - { - const char * function_name = cast_name; - return [function_name] ( - ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) - { - const auto & first_col = arguments.front().column.get(); - const auto & result_type = typeid_cast(*res_type); - - const ColumnStringType * col = typeid_cast(first_col); - - if (col && nullable_col && nullable_col->size() != col->size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnNullable is not compatible with original"); - - if (col) - { - const auto size = col->size(); - - auto res = result_type.createColumn(); - auto & out_data = static_cast(*res).getData(); - out_data.resize(size); - - auto default_enum_value = result_type.getValues().front().second; - - if (nullable_col) - { - for (size_t i = 0; i < size; ++i) - { - if (!nullable_col->isNullAt(i)) - out_data[i] = result_type.getValue(col->getDataAt(i)); - else - out_data[i] = default_enum_value; - } - } - else - { - for (size_t i = 0; i < size; ++i) - out_data[i] = result_type.getValue(col->getDataAt(i)); - } - - return res; - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column {} as first argument of function {}", - first_col->getName(), function_name); - }; - } - - template - WrapperType createEnumToStringWrapper() const - { - const char * function_name = cast_name; - return [function_name] ( - ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) - { - using ColumnEnumType = typename EnumType::ColumnType; - - const auto & first_col = arguments.front().column.get(); - const auto & first_type = arguments.front().type.get(); - - const ColumnEnumType * enum_col = typeid_cast(first_col); - const EnumType * enum_type = typeid_cast(first_type); - - if (enum_col && nullable_col && nullable_col->size() != enum_col->size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnNullable is not compatible with original"); - - if (enum_col && enum_type) - { - const auto size = enum_col->size(); - const auto & enum_data = enum_col->getData(); - - auto res = res_type->createColumn(); - - if (nullable_col) - { - for (size_t i = 0; i < size; ++i) - { - if (!nullable_col->isNullAt(i)) - { - const auto & value = enum_type->getNameForValue(enum_data[i]); - res->insertData(value.data, value.size); - } - else - res->insertDefault(); - } - } - else - { - for (size_t i = 0; i < size; ++i) - { - const auto & value = enum_type->getNameForValue(enum_data[i]); - res->insertData(value.data, value.size); - } - } - - return res; - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column {} as first argument of function {}", - first_col->getName(), function_name); - }; - } - - static WrapperType createIdentityWrapper(const DataTypePtr &) - { - return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) - { - return arguments.front().column; - }; - } - - static WrapperType createNothingWrapper(const IDataType * to_type) - { - ColumnPtr res = to_type->createColumnConstWithDefaultValue(1); - return [res] (ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t input_rows_count) - { - /// Column of Nothing type is trivially convertible to any other column - return res->cloneResized(input_rows_count)->convertToFullColumnIfConst(); - }; - } - - WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const - { - /// Conversion from/to Variant/Dynamic data type is processed in a special way. - /// We don't need to remove LowCardinality/Nullable. - if (isDynamic(to_type) || isDynamic(from_type)) - return createDynamicWrapper(from_type, to_type); - - if (isVariant(to_type) || isVariant(from_type)) - return createVariantWrapper(from_type, to_type); - - const auto * from_low_cardinality = typeid_cast(from_type.get()); - const auto * to_low_cardinality = typeid_cast(to_type.get()); - const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type; - const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type; - - if (from_type->onlyNull()) - { - if (!to_nested->isNullable() && !isVariant(to_type)) - { - if (cast_type == CastType::accurateOrNull) - { - return createToNullableColumnWrapper(); - } - else - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert NULL to a non-nullable type"); - } - } - - return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) - { - return result_type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst(); - }; - } - - bool skip_not_null_check = false; - - if (from_low_cardinality && from_nested->isNullable() && !to_nested->isNullable()) - /// Disable check for dictionary. Will check that column doesn't contain NULL in wrapper below. - skip_not_null_check = true; - - auto wrapper = prepareRemoveNullable(from_nested, to_nested, skip_not_null_check); - if (!from_low_cardinality && !to_low_cardinality) - return wrapper; - - return [wrapper, from_low_cardinality, to_low_cardinality, skip_not_null_check] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) -> ColumnPtr - { - ColumnsWithTypeAndName args = {arguments[0]}; - auto & arg = args.front(); - auto res_type = result_type; - - ColumnPtr converted_column; - - ColumnPtr res_indexes; - /// For some types default can't be cast (for example, String to Int). In that case convert column to full. - bool src_converted_to_full_column = false; - - { - auto tmp_rows_count = input_rows_count; - - if (to_low_cardinality) - res_type = to_low_cardinality->getDictionaryType(); - - if (from_low_cardinality) - { - const auto & col_low_cardinality = typeid_cast(*arguments[0].column); - - if (skip_not_null_check && col_low_cardinality.containsNull()) - throw Exception(ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, "Cannot convert NULL value to non-Nullable type"); - - arg.column = col_low_cardinality.getDictionary().getNestedColumn(); - arg.type = from_low_cardinality->getDictionaryType(); - - /// TODO: Make map with defaults conversion. - src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res_type)); - if (src_converted_to_full_column) - arg.column = arg.column->index(col_low_cardinality.getIndexes(), 0); - else - res_indexes = col_low_cardinality.getIndexesPtr(); - - tmp_rows_count = arg.column->size(); - } - - /// Perform the requested conversion. - converted_column = wrapper(args, res_type, nullable_source, tmp_rows_count); - } - - if (to_low_cardinality) - { - auto res_column = to_low_cardinality->createColumn(); - auto & col_low_cardinality = typeid_cast(*res_column); - - if (from_low_cardinality && !src_converted_to_full_column) - col_low_cardinality.insertRangeFromDictionaryEncodedColumn(*converted_column, *res_indexes); - else - col_low_cardinality.insertRangeFromFullColumn(*converted_column, 0, converted_column->size()); - - return res_column; - } - else if (!src_converted_to_full_column) - return converted_column->index(*res_indexes, 0); - else - return converted_column; - }; - } - - WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type, bool skip_not_null_check) const - { - /// Determine whether pre-processing and/or post-processing must take place during conversion. - - bool source_is_nullable = from_type->isNullable(); - bool result_is_nullable = to_type->isNullable(); - - auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable); - - if (result_is_nullable) - { - return [wrapper, source_is_nullable] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - /// Create a temporary columns on which to perform the operation. - const auto & nullable_type = static_cast(*result_type); - const auto & nested_type = nullable_type.getNestedType(); - - ColumnsWithTypeAndName tmp_args; - if (source_is_nullable) - tmp_args = createBlockWithNestedColumns(arguments); - else - tmp_args = arguments; - - const ColumnNullable * nullable_source = nullptr; - - /// Add original ColumnNullable for createStringToEnumWrapper() - if (source_is_nullable) - { - if (arguments.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of arguments"); - nullable_source = typeid_cast(arguments.front().column.get()); - } - - /// Perform the requested conversion. - auto tmp_res = wrapper(tmp_args, nested_type, nullable_source, input_rows_count); - - /// May happen in fuzzy tests. For debug purpose. - if (!tmp_res) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Couldn't convert {} to {} in prepareRemoveNullable wrapper.", - arguments[0].type->getName(), nested_type->getName()); - - return wrapInNullable(tmp_res, arguments, nested_type, input_rows_count); - }; - } - else if (source_is_nullable) - { - /// Conversion from Nullable to non-Nullable. - - return [wrapper, skip_not_null_check] - (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - auto tmp_args = createBlockWithNestedColumns(arguments); - auto nested_type = removeNullable(result_type); - - /// Check that all values are not-NULL. - /// Check can be skipped in case if LowCardinality dictionary is transformed. - /// In that case, correctness will be checked beforehand. - if (!skip_not_null_check) - { - const auto & col = arguments[0].column; - const auto & nullable_col = assert_cast(*col); - const auto & null_map = nullable_col.getNullMapData(); - - if (!memoryIsZero(null_map.data(), 0, null_map.size())) - throw Exception(ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, "Cannot convert NULL value to non-Nullable type"); - } - const ColumnNullable * nullable_source = typeid_cast(arguments.front().column.get()); - return wrapper(tmp_args, nested_type, nullable_source, input_rows_count); - }; - } - else - return wrapper; - } - - /// 'from_type' and 'to_type' are nested types in case of Nullable. - /// 'requested_result_is_nullable' is true if CAST to Nullable type is requested. - WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const - { - if (isUInt8(from_type) && isBool(to_type)) - return createUInt8ToBoolWrapper(from_type, to_type); - - /// We can cast IPv6 into IPv6, IPv4 into IPv4, but we should not allow to cast FixedString(16) into IPv6 as part of identity cast - bool safe_convert_custom_types = true; - - if (const auto * to_type_custom_name = to_type->getCustomName()) - safe_convert_custom_types = from_type->getCustomName() && from_type->getCustomName()->getName() == to_type_custom_name->getName(); - else if (const auto * from_type_custom_name = from_type->getCustomName()) - safe_convert_custom_types = to_type->getCustomName() && from_type_custom_name->getName() == to_type->getCustomName()->getName(); - - if (from_type->equals(*to_type) && safe_convert_custom_types) - { - /// We can only use identity conversion for DataTypeAggregateFunction when they are strictly equivalent. - if (typeid_cast(from_type.get())) - { - if (DataTypeAggregateFunction::strictEquals(from_type, to_type)) - return createIdentityWrapper(from_type); - } - else - return createIdentityWrapper(from_type); - } - else if (WhichDataType(from_type).isNothing()) - return createNothingWrapper(to_type.get()); - - WrapperType ret; - - auto make_default_wrapper = [&](const auto & types) -> bool - { - using Types = std::decay_t; - using ToDataType = typename Types::LeftType; - - if constexpr (is_any_of) - { - ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); - return true; - } - if constexpr (std::is_same_v) - { - if (isBool(to_type)) - ret = createBoolWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); - else - ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); - return true; - } - if constexpr ( - std::is_same_v || - std::is_same_v) - { - ret = createEnumWrapper(from_type, checkAndGetDataType(to_type.get())); - return true; - } - if constexpr (is_any_of, DataTypeDecimal, - DataTypeDecimal, DataTypeDecimal, - DataTypeDateTime64>) - { - ret = createDecimalWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); - return true; - } - - return false; - }; - - bool cast_ipv4_ipv6_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::cast_ipv4_ipv6_default_on_conversion_error]; - bool input_format_ipv4_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::input_format_ipv4_default_on_conversion_error]; - bool input_format_ipv6_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::input_format_ipv6_default_on_conversion_error]; - - auto make_custom_serialization_wrapper = [&, cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value](const auto & types) -> bool - { - using Types = std::decay_t; - using ToDataType = typename Types::RightType; - using FromDataType = typename Types::LeftType; - - if constexpr (WhichDataType(FromDataType::type_id).isStringOrFixedString()) - { - if constexpr (std::is_same_v) - { - ret = [cast_ipv4_ipv6_default_on_conversion_error_value, - input_format_ipv4_default_on_conversion_error_value, - requested_result_is_nullable]( - ColumnsWithTypeAndName & arguments, - const DataTypePtr & result_type, - const ColumnNullable * column_nullable, - size_t) -> ColumnPtr - { - if (!WhichDataType(result_type).isIPv4()) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); - - const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (requested_result_is_nullable) - return convertToIPv4(arguments[0].column, null_map); - else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value) - return convertToIPv4(arguments[0].column, null_map); - else - return convertToIPv4(arguments[0].column, null_map); - }; - - return true; - } - - if constexpr (std::is_same_v) - { - ret = [cast_ipv4_ipv6_default_on_conversion_error_value, - input_format_ipv6_default_on_conversion_error_value, - requested_result_is_nullable]( - ColumnsWithTypeAndName & arguments, - const DataTypePtr & result_type, - const ColumnNullable * column_nullable, - size_t) -> ColumnPtr - { - if (!WhichDataType(result_type).isIPv6()) - throw Exception( - ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName()); - - const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (requested_result_is_nullable) - return convertToIPv6(arguments[0].column, null_map); - else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value) - return convertToIPv6(arguments[0].column, null_map); - else - return convertToIPv6(arguments[0].column, null_map); - }; - - return true; - } - - if (to_type->getCustomSerialization() && to_type->getCustomName()) - { - ret = [requested_result_is_nullable, this]( - ColumnsWithTypeAndName & arguments, - const DataTypePtr & result_type, - const ColumnNullable * column_nullable, - size_t input_rows_count) -> ColumnPtr - { - auto wrapped_result_type = result_type; - if (requested_result_is_nullable) - wrapped_result_type = makeNullable(result_type); - if (this->cast_type == CastType::accurateOrNull) - return ConvertImplGenericFromString::execute( - arguments, wrapped_result_type, column_nullable, input_rows_count, context); - return ConvertImplGenericFromString::execute( - arguments, wrapped_result_type, column_nullable, input_rows_count, context); - }; - return true; - } - } - else if constexpr (WhichDataType(FromDataType::type_id).isIPv6() && WhichDataType(ToDataType::type_id).isIPv4()) - { - ret = [cast_ipv4_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( - ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) - -> ColumnPtr - { - if (!WhichDataType(result_type).isIPv4()) - throw Exception( - ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); - - const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; - if (requested_result_is_nullable) - return convertIPv6ToIPv4(arguments[0].column, null_map); - else if (cast_ipv4_ipv6_default_on_conversion_error_value) - return convertIPv6ToIPv4(arguments[0].column, null_map); - else - return convertIPv6ToIPv4(arguments[0].column, null_map); - }; - - return true; - } - - if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) - { - if constexpr (WhichDataType(FromDataType::type_id).isEnum()) - { - ret = createEnumToStringWrapper(); - return true; - } - else if (from_type->getCustomSerialization()) - { - ret = [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr - { - return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); - }; - return true; - } - } - - return false; - }; - - if (callOnTwoTypeIndexes(from_type->getTypeId(), to_type->getTypeId(), make_custom_serialization_wrapper)) - return ret; - - if (callOnIndexAndDataType(to_type->getTypeId(), make_default_wrapper)) - return ret; - - switch (to_type->getTypeId()) - { - case TypeIndex::String: - return createStringWrapper(from_type); - case TypeIndex::FixedString: - return createFixedStringWrapper(from_type, checkAndGetDataType(to_type.get())->getN()); - case TypeIndex::Array: - return createArrayWrapper(from_type, static_cast(*to_type)); - case TypeIndex::Tuple: - return createTupleWrapper(from_type, checkAndGetDataType(to_type.get())); - case TypeIndex::Map: - return createMapWrapper(from_type, checkAndGetDataType(to_type.get())); - case TypeIndex::ObjectDeprecated: - return createObjectDeprecatedWrapper(from_type, checkAndGetDataType(to_type.get())); - case TypeIndex::Object: - return createObjectWrapper(from_type, checkAndGetDataType(to_type.get())); - case TypeIndex::AggregateFunction: - return createAggregateFunctionWrapper(from_type, checkAndGetDataType(to_type.get())); - case TypeIndex::Interval: - return createIntervalWrapper(from_type, checkAndGetDataType(to_type.get())->getKind()); - default: - break; - } - - if (cast_type == CastType::accurateOrNull) - return createToNullableColumnWrapper(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", - from_type->getName(), to_type->getName()); - } -}; - -} - - -FunctionBasePtr createFunctionBaseCast( - ContextPtr context, - const char * name, - const ColumnsWithTypeAndName & arguments, - const DataTypePtr & return_type, - std::optional diagnostic, - CastType cast_type) -{ - DataTypes data_types(arguments.size()); - - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - FunctionCast::MonotonicityForRange monotonicity; - - if (isEnum(arguments.front().type) - && castTypeToEither(return_type.get(), [&](auto & type) - { - monotonicity = FunctionTo>::Type::Monotonic::get; - return true; - })) - { - } - else if (castTypeToEither< - DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, - DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, - DataTypeFloat32, DataTypeFloat64, - DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, - DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) - { - monotonicity = FunctionTo>::Type::Monotonic::get; - return true; - })) - { - } - - return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); -} - -REGISTER_FUNCTION(Conversion) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Converts Float32 to BFloat16 with losing the precision. - -Example: -[example:typical] -)", - .examples{ - {"typical", "SELECT toBFloat16(12.3::Float32);", "12.3125"}}, - .categories{"Conversion"}}); - - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - - /// MySQL compatibility alias. Cannot be registered as alias, - /// because we don't want it to be normalized to toDate in queries, - /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Converts String to BFloat16. - -If the string does not represent a floating point value, the function returns zero. - -The function allows a silent loss of precision while converting from the string representation. In that case, it will return the truncated result. - -Example of successful conversion: -[example:typical] - -Examples of not successful conversion: -[example:invalid1] -[example:invalid2] - -Example of a loss of precision: -[example:precision] -)", - .examples{ - {"typical", "SELECT toBFloat16OrZero('12.3');", "12.3125"}, - {"invalid1", "SELECT toBFloat16OrZero('abc');", "0"}, - {"invalid2", "SELECT toBFloat16OrZero(' 1');", "0"}, - {"precision", "SELECT toBFloat16OrZero('12.3456789');", "12.375"}}, - .categories{"Conversion"}}); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(FunctionDocumentation{.description=R"( -Converts String to Nullable(BFloat16). - -If the string does not represent a floating point value, the function returns NULL. - -The function allows a silent loss of precision while converting from the string representation. In that case, it will return the truncated result. - -Example of successful conversion: -[example:typical] - -Examples of not successful conversion: -[example:invalid1] -[example:invalid2] - -Example of a loss of precision: -[example:precision] -)", - .examples{ - {"typical", "SELECT toBFloat16OrNull('12.3');", "12.3125"}, - {"invalid1", "SELECT toBFloat16OrNull('abc');", "NULL"}, - {"invalid2", "SELECT toBFloat16OrNull(' 1');", "NULL"}, - {"precision", "SELECT toBFloat16OrNull('12.3456789');", "12.375"}}, - .categories{"Conversion"}}); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); -} - -} diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h new file mode 100644 index 00000000000..171c2f2a8ff --- /dev/null +++ b/src/Functions/FunctionsConversion.h @@ -0,0 +1,5567 @@ +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace Setting +{ + extern const SettingsBool cast_ipv4_ipv6_default_on_conversion_error; + extern const SettingsBool cast_string_to_dynamic_use_inference; + extern const SettingsDateTimeOverflowBehavior date_time_overflow_behavior; + extern const SettingsBool input_format_ipv4_default_on_conversion_error; + extern const SettingsBool input_format_ipv6_default_on_conversion_error; + extern const SettingsBool precise_float_parsing; + extern const SettingsBool date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands; +} + +namespace ErrorCodes +{ + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int CANNOT_PARSE_NUMBER; + extern const int CANNOT_READ_ARRAY_FROM_TEXT; + extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; + extern const int CANNOT_PARSE_QUOTED_STRING; + extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; + extern const int CANNOT_PARSE_DATE; + extern const int CANNOT_PARSE_DATETIME; + extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_UUID; + extern const int CANNOT_PARSE_IPV4; + extern const int CANNOT_PARSE_IPV6; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int LOGICAL_ERROR; + extern const int TYPE_MISMATCH; + extern const int CANNOT_CONVERT_TYPE; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} + +namespace FunctionsConversion +{ + +/** Type conversion functions. + * toType - conversion in "natural way"; + */ + +UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) +{ + const auto * arg_type = named_column.type.get(); + bool ok = checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type); + if (!ok) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of toDecimal() scale {}", named_column.type->getName()); + + Field field; + named_column.column->get(0, field); + return static_cast(field.safeGet()); +} + + +/** Conversion of Date to DateTime: adding 00:00:00 time component. + */ +template +struct ToDateTimeImpl +{ + static constexpr auto name = "toDateTime"; + + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (d > MAX_DATETIME_DAY_NUM) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Day number {} is out of bounds of type DateTime", d); + } + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) + { + d = std::min(d, MAX_DATETIME_DAY_NUM); + } + return static_cast(time_zone.fromDayNum(DayNum(d))); + } + + static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) + { + if (d < 0) + return 0; + else if (d > MAX_DATETIME_DAY_NUM) + d = MAX_DATETIME_DAY_NUM; + } + else if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (d < 0 || d > MAX_DATETIME_DAY_NUM) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", d); + } + return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); + } + + static UInt32 execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) + { + return dt; + } + + static UInt32 execute(Int64 dt64, const DateLUTImpl & /*time_zone*/) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Ignore) + return static_cast(dt64); + else + { + if (dt64 < 0 || dt64 >= MAX_DATETIME_TIMESTAMP) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Saturate) + return dt64 < 0 ? 0 : std::numeric_limits::max(); + else + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type DateTime", dt64); + } + else + return static_cast(dt64); + } + } +}; + + +/// Implementation of toDate function. + +template +struct ToDateTransform32Or64 +{ + static constexpr auto name = "toDate"; + + static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl & time_zone) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); + } + /// if value is smaller (or equal) than maximum day value for Date, than treat it as day num, + /// otherwise treat it as unix timestamp. This is a bit weird, but we leave this behavior. + if (from <= DATE_LUT_MAX_DAY_NUM) + return from; + else + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); + } +}; + + +template +struct ToDateTransform32Or64Signed +{ + static constexpr auto name = "toDate"; + + static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl & time_zone) + { + // TODO: decide narrow or extended range based on FromType + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from < 0 || from > MAX_DATE_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); + } + else + { + if (from < 0) + return 0; + } + return (from <= DATE_LUT_MAX_DAY_NUM) + ? static_cast(from) + : time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATE_TIMESTAMP))); + } +}; + +template +struct ToDateTransform8Or16Signed +{ + static constexpr auto name = "toDate"; + + static NO_SANITIZE_UNDEFINED UInt16 execute(const FromType & from, const DateLUTImpl &) + { + if (from < 0) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Value {} is out of bounds of type Date", from); + else + return 0; + } + return from; + } +}; + +/// Implementation of toDate32 function. + +template +struct ToDate32Transform32Or64 +{ + static constexpr auto name = "toDate32"; + + static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone) + { + if (from < DATE_LUT_MAX_EXTEND_DAY_NUM) + { + return static_cast(from); + } + else + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); + } + return time_zone.toDayNum(std::min(time_t(from), time_t(MAX_DATETIME64_TIMESTAMP))); + } + } +}; + +template +struct ToDate32Transform32Or64Signed +{ + static constexpr auto name = "toDate32"; + + static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone) + { + static const Int32 daynum_min_offset = -static_cast(DateLUTImpl::getDayNumOffsetEpoch()); + + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from < daynum_min_offset || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type Date32", from); + } + + if (from < daynum_min_offset) + return daynum_min_offset; + + return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) + ? static_cast(from) + : time_zone.toDayNum(std::min(time_t(Int64(from)), time_t(MAX_DATETIME64_TIMESTAMP))); + } +}; + +template +struct ToDate32Transform8Or16Signed +{ + static constexpr auto name = "toDate32"; + + static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl &) + { + return from; + } +}; + +template +struct ToDateTimeTransform64 +{ + static constexpr auto name = "toDateTime"; + + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from > MAX_DATETIME_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); + } + return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); + } +}; + +template +struct ToDateTimeTransformSigned +{ + static constexpr auto name = "toDateTime"; + + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if (from < 0) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); + else + return 0; + } + return from; + } +}; + +template +struct ToDateTimeTransform64Signed +{ + static constexpr auto name = "toDateTime"; + + static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl &) + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from < 0 || from > MAX_DATETIME_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime", from); + } + + if (from < 0) + return 0; + return static_cast(std::min(time_t(from), time_t(MAX_DATETIME_TIMESTAMP))); + } +}; + +/** Conversion of numeric to DateTime64 + */ + +template +struct ToDateTime64TransformUnsigned +{ + static constexpr auto name = "toDateTime64"; + + const DateTime64::NativeType scale_multiplier; + + ToDateTime64TransformUnsigned(UInt32 scale) /// NOLINT + : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) + {} + + NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); + else + return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); + } + else + return DecimalUtils::decimalFromComponentsWithMultiplier(std::min(from, MAX_DATETIME64_TIMESTAMP), 0, scale_multiplier); + } +}; + +template +struct ToDateTime64TransformSigned +{ + static constexpr auto name = "toDateTime64"; + + const DateTime64::NativeType scale_multiplier; + + ToDateTime64TransformSigned(UInt32 scale) /// NOLINT + : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) + {} + + NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); + } + from = static_cast(std::max(from, MIN_DATETIME64_TIMESTAMP)); + from = static_cast(std::min(from, MAX_DATETIME64_TIMESTAMP)); + + return DecimalUtils::decimalFromComponentsWithMultiplier(from, 0, scale_multiplier); + } +}; + +template +struct ToDateTime64TransformFloat +{ + static constexpr auto name = "toDateTime64"; + + const UInt32 scale; + + ToDateTime64TransformFloat(UInt32 scale_) /// NOLINT + : scale(scale_) + {} + + NO_SANITIZE_UNDEFINED DateTime64::NativeType execute(FromType from, const DateLUTImpl &) const + { + if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw) + { + if (from < MIN_DATETIME64_TIMESTAMP || from > MAX_DATETIME64_TIMESTAMP) [[unlikely]] + throw Exception(ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "Timestamp value {} is out of bounds of type DateTime64", from); + } + + from = std::max(from, static_cast(MIN_DATETIME64_TIMESTAMP)); + from = std::min(from, static_cast(MAX_DATETIME64_TIMESTAMP)); + return convertToDecimal(from, scale); + } +}; + +struct ToDateTime64Transform +{ + static constexpr auto name = "toDateTime64"; + + const DateTime64::NativeType scale_multiplier; + + ToDateTime64Transform(UInt32 scale) /// NOLINT + : scale_multiplier(DecimalUtils::scaleMultiplier(scale)) + {} + + DateTime64::NativeType execute(UInt16 d, const DateLUTImpl & time_zone) const + { + const auto dt = ToDateTimeImpl<>::execute(d, time_zone); + return execute(dt, time_zone); + } + + DateTime64::NativeType execute(Int32 d, const DateLUTImpl & time_zone) const + { + Int64 dt = static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); + return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); + } + + DateTime64::NativeType execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) const + { + return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); + } +}; + +/** Transformation of numbers, dates, datetimes to strings: through formatting. + */ +template +struct FormatImpl +{ + template + static ReturnType execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType *, const DateLUTImpl *) + { + writeText(x, wb); + return ReturnType(true); + } +}; + +template <> +struct FormatImpl +{ + template + static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl * time_zone) + { + writeDateText(DayNum(x), wb, *time_zone); + return ReturnType(true); + } +}; + +template <> +struct FormatImpl +{ + template + static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) + { + writeDateText(ExtendedDayNum(x), wb, *time_zone); + return ReturnType(true); + } +}; + +template <> +struct FormatImpl +{ + template + static ReturnType execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime *, const DateLUTImpl * time_zone) + { + writeDateTimeText(x, wb, *time_zone); + return ReturnType(true); + } +}; + +template <> +struct FormatImpl +{ + template + static ReturnType execute(const DataTypeDateTime64::FieldType x, WriteBuffer & wb, const DataTypeDateTime64 * type, const DateLUTImpl * time_zone) + { + writeDateTimeText(DateTime64(x), type->getScale(), wb, *time_zone); + return ReturnType(true); + } +}; + + +template +struct FormatImpl> +{ + template + static ReturnType execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum * type, const DateLUTImpl *) + { + static constexpr bool throw_exception = std::is_same_v; + + if constexpr (throw_exception) + { + writeString(type->getNameForValue(x), wb); + } + else + { + StringRef res; + bool is_ok = type->getNameForValue(x, res); + if (is_ok) + writeString(res, wb); + return ReturnType(is_ok); + } + } +}; + +template +struct FormatImpl> +{ + template + static ReturnType execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal * type, const DateLUTImpl *) + { + writeText(x, type->getScale(), wb, false); + return ReturnType(true); + } +}; + +ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col) +{ + ColumnUInt8::MutablePtr null_map = nullptr; + if (const auto * col_nullable = checkAndGetColumn(col.get())) + { + null_map = ColumnUInt8::create(); + null_map->insertRangeFrom(col_nullable->getNullMapColumn(), 0, col_nullable->size()); + } + return null_map; +} + + +/// Generic conversion of any type to String or FixedString via serialization to text. +template +struct ConvertImplGenericToString +{ + static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/, const ContextPtr & context) + { + static_assert(std::is_same_v || std::is_same_v, + "Can be used only to serialize to ColumnString or ColumnFixedString"); + + ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); + + const auto & col_with_type_and_name = columnGetNested(arguments[0]); + const IDataType & type = *col_with_type_and_name.type; + const IColumn & col_from = *col_with_type_and_name.column; + + size_t size = col_from.size(); + auto col_to = removeNullable(result_type)->createColumn(); + + { + ColumnStringHelpers::WriteHelper write_helper( + assert_cast(*col_to), + size); + + auto & write_buffer = write_helper.getWriteBuffer(); + + FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; + auto serialization = type.getDefaultSerialization(); + for (size_t row = 0; row < size; ++row) + { + serialization->serializeText(col_from, row, write_buffer, format_settings); + write_helper.rowWritten(); + } + + write_helper.finalize(); + } + + if (result_type->isNullable() && null_map) + return ColumnNullable::create(std::move(col_to), std::move(null_map)); + return col_to; + } +}; + +/** Conversion of time_t to UInt16, Int32, UInt32 + */ +template +void convertFromTime(typename DataType::FieldType & x, time_t & time) +{ + x = time; +} + +template <> +inline void convertFromTime(DataTypeDateTime::FieldType & x, time_t & time) +{ + if (unlikely(time < 0)) + x = 0; + else if (unlikely(time > MAX_DATETIME_TIMESTAMP)) + x = MAX_DATETIME_TIMESTAMP; + else + x = static_cast(time); +} + +/** Conversion of strings to numbers, dates, datetimes: through parsing. + */ +template +void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) +{ + if constexpr (is_floating_point) + { + if (precise_float_parsing) + readFloatTextPrecise(x, rb); + else + readFloatTextFast(x, rb); + } + else + readText(x, rb); +} + +template <> +inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + DayNum tmp(0); + readDateText(tmp, rb, *time_zone); + x = tmp; +} + +template <> +inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + ExtendedDayNum tmp(0); + readDateText(tmp, rb, *time_zone); + x = tmp; +} + + +// NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code. +template <> +inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + time_t time = 0; + readDateTimeText(time, rb, *time_zone); + convertFromTime(x, time); +} + +template <> +inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + UUID tmp; + readUUIDText(tmp, rb); + x = tmp.toUnderType(); +} + +template <> +inline void parseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + IPv4 tmp; + readIPv4Text(tmp, rb); + x = tmp.toUnderType(); +} + +template <> +inline void parseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + IPv6 tmp; + readIPv6Text(tmp, rb); + x = tmp; +} + +template +bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) +{ + if constexpr (is_floating_point) + { + if (precise_float_parsing) + return tryReadFloatTextPrecise(x, rb); + else + return tryReadFloatTextFast(x, rb); + } + else /*if constexpr (is_integral_v)*/ + return tryReadIntText(x, rb); +} + +template <> +inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + DayNum tmp(0); + if (!tryReadDateText(tmp, rb, *time_zone)) + return false; + x = tmp; + return true; +} + +template <> +inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + ExtendedDayNum tmp(0); + if (!tryReadDateText(tmp, rb, *time_zone)) + return false; + x = tmp; + return true; +} + +template <> +inline bool tryParseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) +{ + time_t time = 0; + if (!tryReadDateTimeText(time, rb, *time_zone)) + return false; + convertFromTime(x, time); + return true; +} + +template <> +inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + UUID tmp; + if (!tryReadUUIDText(tmp, rb)) + return false; + + x = tmp.toUnderType(); + return true; +} + +template <> +inline bool tryParseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + IPv4 tmp; + if (!tryReadIPv4Text(tmp, rb)) + return false; + + x = tmp.toUnderType(); + return true; +} + +template <> +inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) +{ + IPv6 tmp; + if (!tryReadIPv6Text(tmp, rb)) + return false; + + x = tmp; + return true; +} + + +/** Throw exception with verbose message when string value is not parsed completely. + */ +[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, const IDataType & result_type) +{ + WriteBufferFromOwnString message_buf; + message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size()) + << " as " << result_type.getName() + << ": syntax error"; + + if (read_buffer.offset()) + message_buf << " at position " << read_buffer.offset() + << " (parsed just " << quote << String(read_buffer.buffer().begin(), read_buffer.offset()) << ")"; + else + message_buf << " at begin of string"; + + // Currently there are no functions toIPv{4,6}Or{Null,Zero} + if (isNativeNumber(result_type) && !(result_type.getName() == "IPv4" || result_type.getName() == "IPv6")) + message_buf << ". Note: there are to" << result_type.getName() << "OrZero and to" << result_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception."; + + throw Exception(PreformattedMessage{message_buf.str(), "Cannot parse string {} as {}: syntax error {}", {String(read_buffer.buffer().begin(), read_buffer.buffer().size()), result_type.getName()}}, ErrorCodes::CANNOT_PARSE_TEXT); +} + + +enum class ConvertFromStringExceptionMode : uint8_t +{ + Throw, /// Throw exception if value cannot be parsed. + Zero, /// Fill with zero or default if value cannot be parsed. + Null /// Return ColumnNullable with NULLs when value cannot be parsed. +}; + +enum class ConvertFromStringParsingMode : uint8_t +{ + Normal, + BestEffort, /// Only applicable for DateTime. Will use sophisticated method, that is slower. + BestEffortUS +}; + +struct AccurateConvertStrategyAdditions +{ + UInt32 scale { 0 }; +}; + +struct AccurateOrNullConvertStrategyAdditions +{ + UInt32 scale { 0 }; +}; + +template +struct ConvertThroughParsing +{ + static_assert(std::is_same_v || std::is_same_v, + "ConvertThroughParsing is only applicable for String or FixedString data types"); + + static constexpr bool to_datetime64 = std::is_same_v; + + static bool isAllRead(ReadBuffer & in) + { + /// In case of FixedString, skip zero bytes at end. + if constexpr (std::is_same_v) + while (!in.eof() && *in.position() == 0) + ++in.position(); + + if (in.eof()) + return true; + + /// Special case, that allows to parse string with DateTime or DateTime64 as Date or Date32. + if constexpr (std::is_same_v || std::is_same_v) + { + if (!in.eof() && (*in.position() == ' ' || *in.position() == 'T')) + { + if (in.buffer().size() == strlen("YYYY-MM-DD hh:mm:ss")) + return true; + + if (in.buffer().size() >= strlen("YYYY-MM-DD hh:mm:ss.x") + && in.buffer().begin()[19] == '.') + { + in.position() = in.buffer().begin() + 20; + + while (!in.eof() && isNumericASCII(*in.position())) + ++in.position(); + + if (in.eof()) + return true; + } + } + } + + return false; + } + + template + static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, size_t input_rows_count, + Additions additions [[maybe_unused]] = Additions()) + { + using ColVecTo = typename ToDataType::ColumnType; + + const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr; + const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr; + + /// For conversion to Date or DateTime type, second argument with time zone could be specified. + if constexpr (std::is_same_v || to_datetime64) + { + const auto result_type = removeNullable(res_type); + // Time zone is already figured out during result type resolution, no need to do it here. + if (const auto dt_col = checkAndGetDataType(result_type.get())) + local_time_zone = &dt_col->getTimeZone(); + else + local_time_zone = &extractTimeZoneFromFunctionArguments(arguments, 1, 0); + + if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS) + utc_time_zone = &DateLUT::instance("UTC"); + } + else if constexpr (std::is_same_v || std::is_same_v) + { + // Timezone is more or less dummy when parsing Date/Date32 from string. + local_time_zone = &DateLUT::instance(); + utc_time_zone = &DateLUT::instance("UTC"); + } + + const IColumn * col_from = arguments[0].column.get(); + const ColumnString * col_from_string = checkAndGetColumn(col_from); + const ColumnFixedString * col_from_fixed_string = checkAndGetColumn(col_from); + + if (std::is_same_v && !col_from_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + col_from->getName(), Name::name); + + if (std::is_same_v && !col_from_fixed_string) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + col_from->getName(), Name::name); + + size_t size = input_rows_count; + typename ColVecTo::MutablePtr col_to = nullptr; + + if constexpr (IsDataTypeDecimal) + { + UInt32 scale = additions; + if constexpr (to_datetime64) + { + ToDataType check_bounds_in_ctor(scale, local_time_zone ? local_time_zone->getTimeZone() : String{}); + } + else + { + ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale); + } + col_to = ColVecTo::create(size, scale); + } + else + col_to = ColVecTo::create(size); + + typename ColVecTo::Container & vec_to = col_to->getData(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; + if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) + { + col_null_map_to = ColumnUInt8::create(size); + vec_null_map_to = &col_null_map_to->getData(); + } + + const ColumnString::Chars * chars = nullptr; + const IColumn::Offsets * offsets = nullptr; + size_t fixed_string_size = 0; + + if constexpr (std::is_same_v) + { + chars = &col_from_string->getChars(); + offsets = &col_from_string->getOffsets(); + } + else + { + chars = &col_from_fixed_string->getChars(); + fixed_string_size = col_from_fixed_string->getN(); + } + + size_t current_offset = 0; + + bool precise_float_parsing = false; + + if (DB::CurrentThread::isInitialized()) + { + const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + precise_float_parsing = query_context->getSettingsRef()[Setting::precise_float_parsing]; + } + + for (size_t i = 0; i < size; ++i) + { + size_t next_offset = std::is_same_v ? (*offsets)[i] : (current_offset + fixed_string_size); + size_t string_size = std::is_same_v ? next_offset - current_offset - 1 : fixed_string_size; + + ReadBufferFromMemory read_buffer(chars->data() + current_offset, string_size); + + if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw) + { + if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort) + { + if constexpr (to_datetime64) + { + DateTime64 res = 0; + parseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); + vec_to[i] = res; + } + else + { + time_t res; + parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); + convertFromTime(vec_to[i], res); + } + } + else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) + { + if constexpr (to_datetime64) + { + DateTime64 res = 0; + parseDateTime64BestEffortUS(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); + vec_to[i] = res; + } + else + { + time_t res; + parseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); + convertFromTime(vec_to[i], res); + } + } + else + { + if constexpr (to_datetime64) + { + DateTime64 value = 0; + readDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone); + vec_to[i] = value; + } + else if constexpr (IsDataTypeDecimal) + { + SerializationDecimal::readText( + vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); + } + else + { + /// we want to utilize constexpr condition here, which is not mixable with value comparison + do + { + if constexpr (std::is_same_v && std::is_same_v) + { + if (fixed_string_size == IPV6_BINARY_LENGTH) + { + readBinary(vec_to[i], read_buffer); + break; + } + } + if constexpr (std::is_same_v) + { + if (!tryParseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to type {}", TypeName); + } + else + parseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); + } while (false); + } + } + + if (!isAllRead(read_buffer)) + throwExceptionForIncompletelyParsedValue(read_buffer, *res_type); + } + else + { + bool parsed; + + if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort) + { + if constexpr (to_datetime64) + { + DateTime64 res = 0; + parsed = tryParseDateTime64BestEffort(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); + vec_to[i] = res; + } + else + { + time_t res; + parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); + convertFromTime(vec_to[i],res); + } + } + else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) + { + if constexpr (to_datetime64) + { + DateTime64 res = 0; + parsed = tryParseDateTime64BestEffortUS(res, col_to->getScale(), read_buffer, *local_time_zone, *utc_time_zone); + vec_to[i] = res; + } + else + { + time_t res; + parsed = tryParseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); + convertFromTime(vec_to[i],res); + } + } + else + { + if constexpr (to_datetime64) + { + DateTime64 value = 0; + parsed = tryReadDateTime64Text(value, col_to->getScale(), read_buffer, *local_time_zone); + vec_to[i] = value; + } + else if constexpr (IsDataTypeDecimal) + { + parsed = SerializationDecimal::tryReadText( + vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); + } + else if (std::is_same_v && std::is_same_v + && fixed_string_size == IPV6_BINARY_LENGTH) + { + readBinary(vec_to[i], read_buffer); + parsed = true; + } + else + { + parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); + } + } + + if (!isAllRead(read_buffer)) + parsed = false; + + if (!parsed) + { + if constexpr (std::is_same_v) + { + vec_to[i] = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance) + } + else + { + vec_to[i] = static_cast(0); + } + } + + if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) + (*vec_null_map_to)[i] = !parsed; + } + + current_offset = next_offset; + } + + if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) + return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); + else + return col_to; + } +}; + + +/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type. +struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; + +enum class BehaviourOnErrorFromString : uint8_t +{ + ConvertDefaultBehaviorTag, + ConvertReturnNullOnErrorTag, + ConvertReturnZeroOnErrorTag +}; + +/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment. + * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) + */ +template +struct ConvertImpl +{ + template + static ColumnPtr NO_SANITIZE_UNDEFINED execute( + const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type [[maybe_unused]], size_t input_rows_count, + BehaviourOnErrorFromString from_string_tag [[maybe_unused]], Additions additions = Additions()) + { + const ColumnWithTypeAndName & named_from = arguments[0]; + + if constexpr ((std::is_same_v && !FromDataType::is_parametric) + || (std::is_same_v && std::is_same_v) + || (std::is_same_v && std::is_same_v)) + { + /// If types are the same, reuse the columns. + /// Conversions between Enum and the underlying type are also free. + return named_from.column; + } + else if constexpr ((std::is_same_v || std::is_same_v) + && std::is_same_v) + { + /// Conversion of DateTime to Date: throw off time component. + /// Conversion of Date32 to Date. + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (std::is_same_v && std::is_same_v) + { + /// Conversion of DateTime to Date: throw off time component. + return DateTimeTransformImpl::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr ((std::is_same_v || std::is_same_v) + && std::is_same_v) + { + /// Conversion from Date/Date32 to DateTime. + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (std::is_same_v && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + /** Special case of converting Int8, Int16, (U)Int32 or (U)Int64 (and also, for convenience, + * Float32, Float64) to Date. If the + * number is less than 65536, then it is treated as DayNum, and if it's greater or equals to 65536, + * then treated as unix timestamp. If the number exceeds UInt32, saturate to MAX_UINT32 then as DayNum. + * It's a bit illogical, as we actually have two functions in one. + * But allows to support frequent case, + * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. + * (otherwise such usage would be frequent mistake). + */ + else if constexpr (( + std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + /// Special case of converting Int8, Int16, Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to DateTime. + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (std::is_same_v + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count); + } + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + else if constexpr (std::is_same_v + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + else if constexpr (( + std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + /// Conversion of DateTime64 to Date or DateTime: discards fractional part. + else if constexpr (std::is_same_v + && std::is_same_v) + { + return DateTimeTransformImpl>, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + else if constexpr (std::is_same_v + && std::is_same_v) + { + return DateTimeTransformImpl>, false>::template execute( + arguments, result_type, input_rows_count, additions); + } + /// Conversion of Date or DateTime to DateTime64: add zero sub-second part. + else if constexpr (( + std::is_same_v + || std::is_same_v + || std::is_same_v) + && std::is_same_v) + { + return DateTimeTransformImpl::template execute( + arguments, result_type, input_rows_count, additions); + } + else if constexpr (IsDataTypeDateOrDateTime + && std::is_same_v) + { + /// Date or DateTime to String + + using FromFieldType = typename FromDataType::FieldType; + using ColVecType = ColumnVectorOrDecimal; + + auto datetime_arg = arguments[0]; + + const DateLUTImpl * time_zone = nullptr; + const ColumnConst * time_zone_column = nullptr; + + if (arguments.size() == 1) + { + auto non_null_args = createBlockWithNestedColumns(arguments); + time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); + } + else /// When we have a column for timezone + { + datetime_arg.column = datetime_arg.column->convertToFullColumnIfConst(); + + if constexpr (std::is_same_v || std::is_same_v) + time_zone = &DateLUT::instance(); + /// For argument of Date or DateTime type, second argument with time zone could be specified. + if constexpr (std::is_same_v || std::is_same_v) + { + if ((time_zone_column = checkAndGetColumnConst(arguments[1].column.get()))) + { + auto non_null_args = createBlockWithNestedColumns(arguments); + time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); + } + } + } + const auto & col_with_type_and_name = columnGetNested(datetime_arg); + + if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) + { + auto col_to = ColumnString::create(); + + const typename ColVecType::Container & vec_from = col_from->getData(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + size_t size = vec_from.size(); + + if constexpr (std::is_same_v) + data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); + else if constexpr (std::is_same_v) + data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); + else if constexpr (std::is_same_v) + data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1)); + else if constexpr (std::is_same_v) + data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1)); + else + data_to.resize(size * 3); /// Arbitrary + + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + const FromDataType & type = static_cast(*col_with_type_and_name.type); + + ColumnUInt8::MutablePtr null_map = copyNullMap(datetime_arg.column); + + bool cut_trailing_zeros_align_to_groups_of_thousands = false; + if (DB::CurrentThread::isInitialized()) + { + const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + cut_trailing_zeros_align_to_groups_of_thousands = query_context->getSettingsRef()[Setting::date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands]; + } + + if (!null_map && arguments.size() > 1) + null_map = copyNullMap(arguments[1].column->convertToFullColumnIfConst()); + + if (null_map) + { + for (size_t i = 0; i < size; ++i) + { + if (!time_zone_column && arguments.size() > 1) + { + if (!arguments[1].column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); + } + bool is_ok = true; + if constexpr (std::is_same_v) + { + if (cut_trailing_zeros_align_to_groups_of_thousands) + writeDateTimeTextCutTrailingZerosAlignToGroupOfThousands(DateTime64(vec_from[i]), type.getScale(), write_buffer, *time_zone); + else + is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + } + else + { + is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + } + null_map->getData()[i] |= !is_ok; + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + } + else + { + for (size_t i = 0; i < size; ++i) + { + if (!time_zone_column && arguments.size() > 1) + { + if (!arguments[1].column.get()->getDataAt(i).toString().empty()) + time_zone = &DateLUT::instance(arguments[1].column.get()->getDataAt(i).toString()); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Provided time zone must be non-empty"); + } + if constexpr (std::is_same_v) + { + if (cut_trailing_zeros_align_to_groups_of_thousands) + writeDateTimeTextCutTrailingZerosAlignToGroupOfThousands(DateTime64(vec_from[i]), type.getScale(), write_buffer, *time_zone); + else + FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + } + else + { + FormatImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + } + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + } + + write_buffer.finalize(); + + if (null_map) + return ColumnNullable::create(std::move(col_to), std::move(null_map)); + return col_to; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), Name::name); + } + /// Conversion from FixedString to String. + /// Cutting sequences of zero bytes from end of strings. + else if constexpr (std::is_same_v + && std::is_same_v) + { + ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); + const auto & nested = columnGetNested(arguments[0]); + if (const ColumnFixedString * col_from = checkAndGetColumn(nested.column.get())) + { + auto col_to = ColumnString::create(); + + const ColumnFixedString::Chars & data_from = col_from->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + size_t size = col_from->size(); + size_t n = col_from->getN(); + data_to.resize(size * (n + 1)); /// + 1 - zero terminator + offsets_to.resize(size); + + size_t offset_from = 0; + size_t offset_to = 0; + for (size_t i = 0; i < size; ++i) + { + if (!null_map || !null_map->getData()[i]) + { + size_t bytes_to_copy = n; + while (bytes_to_copy > 0 && data_from[offset_from + bytes_to_copy - 1] == 0) + --bytes_to_copy; + + memcpy(&data_to[offset_to], &data_from[offset_from], bytes_to_copy); + offset_to += bytes_to_copy; + } + data_to[offset_to] = 0; + ++offset_to; + offsets_to[i] = offset_to; + offset_from += n; + } + + data_to.resize(offset_to); + if (result_type->isNullable() && null_map) + return ColumnNullable::create(std::move(col_to), std::move(null_map)); + return col_to; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), Name::name); + } + else if constexpr (std::is_same_v) + { + /// Anything else to String. + + using FromFieldType = typename FromDataType::FieldType; + using ColVecType = ColumnVectorOrDecimal; + + ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column); + + const auto & col_with_type_and_name = columnGetNested(arguments[0]); + const auto & type = static_cast(*col_with_type_and_name.type); + + if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) + { + auto col_to = ColumnString::create(); + + const typename ColVecType::Container & vec_from = col_from->getData(); + ColumnString::Chars & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + size_t size = vec_from.size(); + + data_to.resize(size * 3); + offsets_to.resize(size); + + WriteBufferFromVector write_buffer(data_to); + + if (null_map) + { + for (size_t i = 0; i < size; ++i) + { + bool is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); + /// We don't use timezones in this branch + null_map->getData()[i] |= !is_ok; + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + } + else + { + for (size_t i = 0; i < size; ++i) + { + FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + } + + write_buffer.finalize(); + + if (null_map) + return ColumnNullable::create(std::move(col_to), std::move(null_map)); + return col_to; + } + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + arguments[0].column->getName(), Name::name); + } + else if constexpr (std::is_same_v + && std::is_same_v + && std::is_same_v) + { + return ConvertImpl::template execute( + arguments, result_type, input_rows_count, from_string_tag); + } + else if constexpr ((std::is_same_v || std::is_same_v)) + { + switch (from_string_tag) + { + case BehaviourOnErrorFromString::ConvertDefaultBehaviorTag: + return ConvertThroughParsing::execute( + arguments, result_type, input_rows_count, additions); + case BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag: + return ConvertThroughParsing::execute( + arguments, result_type, input_rows_count, additions); + case BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag: + return ConvertThroughParsing::execute( + arguments, result_type, input_rows_count, additions); + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + IntervalKind to = typeid_cast(result_type.get())->getKind(); + IntervalKind from = typeid_cast(arguments[0].type.get())->getKind(); + + if (from == to || arguments[0].column->empty()) + return arguments[0].column; + + Int64 conversion_factor = 1; + Int64 result_value; + + int from_position = static_cast(from.kind); + int to_position = static_cast(to.kind); /// Positions of each interval according to granularity map + + if (from_position < to_position) + { + for (int i = from_position; i < to_position; ++i) + conversion_factor *= interval_conversions[i]; + result_value = arguments[0].column->getInt(0) / conversion_factor; + } + else + { + for (int i = from_position; i > to_position; --i) + conversion_factor *= interval_conversions[i]; + result_value = arguments[0].column->getInt(0) * conversion_factor; + } + + return ColumnConst::create(ColumnInt64::create(1, result_value), input_rows_count); + } + else + { + using FromFieldType = typename FromDataType::FieldType; + using ToFieldType = typename ToDataType::FieldType; + using ColVecFrom = typename FromDataType::ColumnType; + using ColVecTo = typename ToDataType::ColumnType; + + if constexpr ((IsDataTypeDecimal || IsDataTypeDecimal) + && !(std::is_same_v || std::is_same_v) + && (!IsDataTypeDecimalOrNumber || !IsDataTypeDecimalOrNumber)) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + named_from.column->getName(), Name::name); + } + + const ColVecFrom * col_from = checkAndGetColumn(named_from.column.get()); + if (!col_from) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", + named_from.column->getName(), Name::name); + + typename ColVecTo::MutablePtr col_to = nullptr; + + if constexpr (IsDataTypeDecimal) + { + UInt32 scale; + + if constexpr (std::is_same_v + || std::is_same_v) + { + scale = additions.scale; + } + else + { + scale = additions; + } + + col_to = ColVecTo::create(0, scale); + } + else + col_to = ColVecTo::create(); + + const auto & vec_from = col_from->getData(); + auto & vec_to = col_to->getData(); + vec_to.resize(input_rows_count); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; + if constexpr (std::is_same_v) + { + col_null_map_to = ColumnUInt8::create(input_rows_count, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + bool result_is_bool = isBool(result_type); + for (size_t i = 0; i < input_rows_count; ++i) + { + if constexpr (std::is_same_v) + { + if (result_is_bool) + { + vec_to[i] = vec_from[i] != FromFieldType(0); + continue; + } + } + + if constexpr (std::is_same_v && std::is_same_v) + { + static_assert( + std::is_same_v, + "UInt128 and UUID types must be same"); + + vec_to[i].items[1] = vec_from[i].toUnderType().items[0]; + vec_to[i].items[0] = vec_from[i].toUnderType().items[1]; + } + else if constexpr (std::is_same_v && std::is_same_v) + { + static_assert( + std::is_same_v, + "UInt128 and IPv6 types must be same"); + + vec_to[i].items[1] = std::byteswap(vec_from[i].toUnderType().items[0]); + vec_to[i].items[0] = std::byteswap(vec_from[i].toUnderType().items[1]); + } + else if constexpr (std::is_same_v != std::is_same_v) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Conversion between numeric types and UUID is not supported. " + "Probably the passed UUID is unquoted"); + } + else if constexpr ( + (std::is_same_v != std::is_same_v) + && !(is_any_of + || is_any_of)) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", + TypeName, TypeName); + } + else if constexpr (std::is_same_v != std::is_same_v + && !(std::is_same_v || std::is_same_v)) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Conversion between numeric types and IPv6 is not supported. " + "Probably the passed IPv6 is unquoted"); + } + else if constexpr (IsDataTypeDecimal || IsDataTypeDecimal) + { + if constexpr (std::is_same_v) + { + ToFieldType result; + bool convert_result = false; + + if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) + convert_result = tryConvertDecimals(vec_from[i], col_from->getScale(), col_to->getScale(), result); + else if constexpr (IsDataTypeDecimal && IsDataTypeNumber) + convert_result = tryConvertFromDecimal(vec_from[i], col_from->getScale(), result); + else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) + convert_result = tryConvertToDecimal(vec_from[i], col_to->getScale(), result); + + if (convert_result) + vec_to[i] = result; + else + { + vec_to[i] = static_cast(0); + (*vec_null_map_to)[i] = true; + } + } + else + { + if constexpr (IsDataTypeDecimal && IsDataTypeDecimal) + vec_to[i] = convertDecimals(vec_from[i], col_from->getScale(), col_to->getScale()); + else if constexpr (IsDataTypeDecimal && IsDataTypeNumber) + vec_to[i] = convertFromDecimal(vec_from[i], col_from->getScale()); + else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) + vec_to[i] = convertToDecimal(vec_from[i], col_to->getScale()); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Unsupported data type in conversion function"); + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + { + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, named_from.column->getName()); + } + + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + std::memset(dst, '\0', IPV6_BINARY_LENGTH); + dst[10] = dst[11] = 0xff; + + if constexpr (std::endian::native == std::endian::little) + { + dst[12] = src[3]; + dst[13] = src[2]; + dst[14] = src[1]; + dst[15] = src[0]; + } + else + { + dst[12] = src[0]; + dst[13] = src[1]; + dst[14] = src[2]; + dst[15] = src[3]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + vec_to[i] = static_cast(static_cast(vec_from[i])); + } + else if constexpr (std::is_same_v + && (std::is_same_v || std::is_same_v)) + { + vec_to[i] = static_cast(vec_from[i] * DATE_SECONDS_PER_DAY); + } + else + { + /// If From Data is Nan or Inf and we convert to integer type, throw exception + if constexpr (is_floating_point && !is_floating_point) + { + if (!isFinite(vec_from[i])) + { + if constexpr (std::is_same_v) + { + vec_to[i] = 0; + (*vec_null_map_to)[i] = true; + continue; + } + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Unexpected inf or nan to integer conversion"); + } + } + + if constexpr (std::is_same_v + || std::is_same_v) + { + bool convert_result = accurate::convertNumeric(vec_from[i], vec_to[i]); + + if (!convert_result) + { + if (std::is_same_v) + { + vec_to[i] = 0; + (*vec_null_map_to)[i] = true; + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Value in column {} cannot be safely converted into type {}", + named_from.column->getName(), result_type->getName()); + } + } + } + else + { + vec_to[i] = static_cast(vec_from[i]); + } + } + } + + if constexpr (std::is_same_v) + return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); + else + return col_to; + } + } +}; + + +/// Generic conversion of any type from String. Used for complex types: Array and Tuple or types with custom serialization. +template +struct ConvertImplGenericFromString +{ + static ColumnPtr execute(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count, const ContextPtr & context) + { + const IColumn & column_from = *arguments[0].column; + const IDataType & data_type_to = *result_type; + auto res = data_type_to.createColumn(); + auto serialization = data_type_to.getDefaultSerialization(); + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + + executeImpl(column_from, *res, *serialization, input_rows_count, null_map, result_type.get(), context); + return res; + } + + static void executeImpl( + const IColumn & column_from, + IColumn & column_to, + const ISerialization & serialization_from, + size_t input_rows_count, + const PaddedPODArray * null_map, + const IDataType * result_type, + const ContextPtr & context) + { + column_to.reserve(input_rows_count); + + FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; + for (size_t i = 0; i < input_rows_count; ++i) + { + if (null_map && (*null_map)[i]) + { + column_to.insertDefault(); + continue; + } + + const auto & val = column_from.getDataAt(i); + ReadBufferFromMemory read_buffer(val.data, val.size); + try + { + serialization_from.deserializeWholeText(column_to, read_buffer, format_settings); + } + catch (const Exception &) + { + if constexpr (throw_on_error) + throw; + /// Check if exception happened after we inserted the value + /// (deserializeWholeText should not do it, but let's check anyway). + if (column_to.size() > i) + column_to.popBack(column_to.size() - i); + column_to.insertDefault(); + } + + /// Usually deserializeWholeText checks for eof after parsing, but let's check one more time just in case. + if (!read_buffer.eof()) + { + if constexpr (throw_on_error) + { + if (result_type) + throwExceptionForIncompletelyParsedValue(read_buffer, *result_type); + else + throw Exception( + ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse string to column {}. Expected eof", column_to.getName()); + } + else + { + if (column_to.size() > i) + column_to.popBack(column_to.size() - i); + column_to.insertDefault(); + } + } + } + } +}; + + +/// Declared early because used below. +struct NameToDate { static constexpr auto name = "toDate"; }; +struct NameToDate32 { static constexpr auto name = "toDate32"; }; +struct NameToDateTime { static constexpr auto name = "toDateTime"; }; +struct NameToDateTime32 { static constexpr auto name = "toDateTime32"; }; +struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; }; +struct NameToString { static constexpr auto name = "toString"; }; +struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; }; +struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; }; +struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; }; +struct NameToDecimal256 { static constexpr auto name = "toDecimal256"; }; + + +#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \ + struct NameToInterval ## INTERVAL_KIND \ + { \ + static constexpr auto name = "toInterval" #INTERVAL_KIND; \ + static constexpr auto kind = IntervalKind::Kind::INTERVAL_KIND; \ + }; + +DEFINE_NAME_TO_INTERVAL(Nanosecond) +DEFINE_NAME_TO_INTERVAL(Microsecond) +DEFINE_NAME_TO_INTERVAL(Millisecond) +DEFINE_NAME_TO_INTERVAL(Second) +DEFINE_NAME_TO_INTERVAL(Minute) +DEFINE_NAME_TO_INTERVAL(Hour) +DEFINE_NAME_TO_INTERVAL(Day) +DEFINE_NAME_TO_INTERVAL(Week) +DEFINE_NAME_TO_INTERVAL(Month) +DEFINE_NAME_TO_INTERVAL(Quarter) +DEFINE_NAME_TO_INTERVAL(Year) + +#undef DEFINE_NAME_TO_INTERVAL + +struct NameParseDateTimeBestEffort; +struct NameParseDateTimeBestEffortOrZero; +struct NameParseDateTimeBestEffortOrNull; + +template +constexpr bool mightBeDateTime() +{ + if constexpr (std::is_same_v) + return true; + else if constexpr ( + std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v) + return true; + + return false; +} + +template +inline bool isDateTime64(const ColumnsWithTypeAndName & arguments) +{ + if constexpr (std::is_same_v) + return true; + else if constexpr (std::is_same_v || std::is_same_v + || std::is_same_v || std::is_same_v) + { + return (arguments.size() == 2 && isUInt(arguments[1].type)) || arguments.size() == 3; + } + + return false; +} + +template +class FunctionConvert : public IFunction +{ +public: + using Monotonic = MonotonicityImpl; + + static constexpr auto name = Name::name; + static constexpr bool to_datetime64 = std::is_same_v; + static constexpr bool to_decimal = IsDataTypeDecimal && !to_datetime64; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionConvert(ContextPtr context_) : context(context_) {} + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isInjective(const ColumnsWithTypeAndName &) const override { return std::is_same_v; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override + { + return !(IsDataTypeDateOrDateTime && isNumber(*arguments[0].type)); + } + + using DefaultReturnTypeGetter = std::function; + static DataTypePtr getReturnTypeDefaultImplementationForNulls(const ColumnsWithTypeAndName & arguments, const DefaultReturnTypeGetter & getter) + { + NullPresence null_presence = getNullPresense(arguments); + + if (null_presence.has_null_constant) + { + return makeNullable(std::make_shared()); + } + if (null_presence.has_nullable) + { + auto nested_columns = Block(createBlockWithNestedColumns(arguments)); + auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end())); + return makeNullable(return_type); + } + + return getter(arguments); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto getter = [&] (const auto & args) { return getReturnTypeImplRemovedNullable(args); }; + auto res = getReturnTypeDefaultImplementationForNulls(arguments, getter); + to_nullable = res->isNullable(); + checked_return_type = true; + return res; + } + + DataTypePtr getReturnTypeImplRemovedNullable(const ColumnsWithTypeAndName & arguments) const + { + FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, "any type"}}; + FunctionArgumentDescriptors optional_args; + + if constexpr (to_decimal) + { + mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); + } + + if (!to_decimal && isDateTime64(arguments)) + { + mandatory_args.push_back({"scale", static_cast(&isNativeInteger), &isColumnConst, "const Integer"}); + } + + // toString(DateTime or DateTime64, [timezone: String]) + if ((std::is_same_v && !arguments.empty() && (isDateTime64(arguments[0].type) || isDateTime(arguments[0].type))) + // toUnixTimestamp(value[, timezone : String]) + || std::is_same_v + // toDate(value[, timezone : String]) + || std::is_same_v // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below. + // toDate32(value[, timezone : String]) + || std::is_same_v + // toDateTime(value[, timezone: String]) + || std::is_same_v + // toDateTime64(value, scale : Integer[, timezone: String]) + || std::is_same_v) + { + optional_args.push_back({"timezone", static_cast(&isString), nullptr, "String"}); + } + + validateFunctionArguments(*this, arguments, mandatory_args, optional_args); + + if constexpr (std::is_same_v) + { + return std::make_shared(Name::kind); + } + else if constexpr (to_decimal) + { + UInt64 scale = extractToDecimalScale(arguments[1]); + + if constexpr (std::is_same_v) + return createDecimalMaxPrecision(scale); + else if constexpr (std::is_same_v) + return createDecimalMaxPrecision(scale); + else if constexpr (std::is_same_v) + return createDecimalMaxPrecision(scale); + else if constexpr (std::is_same_v) + return createDecimalMaxPrecision(scale); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected branch in code of conversion function: it is a bug."); + } + else + { + // Optional second argument with time zone for DateTime. + UInt8 timezone_arg_position = 1; + UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale; + + // DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first. + if (isDateTime64(arguments)) + { + timezone_arg_position += 1; + scale = static_cast(arguments[1].column->get64(0)); + + if (to_datetime64 || scale != 0) /// toDateTime('xxxx-xx-xx xx:xx:xx', 0) return DateTime + return std::make_shared(scale, + extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); + + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); + } + + if constexpr (std::is_same_v) + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0, false)); + else if constexpr (std::is_same_v) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected branch in code of conversion function: it is a bug."); + else + return std::make_shared(); + } + } + + /// Function actually uses default implementation for nulls, + /// but we need to know if return type is Nullable or not, + /// so we use checked_return_type only to intercept the first call to getReturnTypeImpl(...). + bool useDefaultImplementationForNulls() const override + { + bool to_nullable_string = to_nullable && std::is_same_v; + return checked_return_type && !to_nullable_string; + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override + { + if constexpr (std::is_same_v) + return {}; + else if constexpr (std::is_same_v) + return {2}; + return {1}; + } + bool canBeExecutedOnDefaultArguments() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + try + { + return executeInternal(arguments, result_type, input_rows_count); + } + catch (Exception & e) + { + /// More convenient error message. + if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + { + e.addMessage("Cannot parse " + + result_type->getName() + " from " + + arguments[0].type->getName() + + ", because value is too short"); + } + else if (e.code() == ErrorCodes::CANNOT_PARSE_NUMBER + || e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT + || e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED + || e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING + || e.code() == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE + || e.code() == ErrorCodes::CANNOT_PARSE_DATE + || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME + || e.code() == ErrorCodes::CANNOT_PARSE_UUID + || e.code() == ErrorCodes::CANNOT_PARSE_IPV4 + || e.code() == ErrorCodes::CANNOT_PARSE_IPV6) + { + e.addMessage("Cannot parse " + + result_type->getName() + " from " + + arguments[0].type->getName()); + } + + throw; + } + } + + bool hasInformationAboutMonotonicity() const override + { + return Monotonic::has(); + } + + Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override + { + return Monotonic::get(type, left, right); + } + +private: + ContextPtr context; + mutable bool checked_return_type = false; + mutable bool to_nullable = false; + + ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { + if (arguments.empty()) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least 1 argument", getName()); + + if (result_type->onlyNull()) + return result_type->createColumnConstWithDefaultValue(input_rows_count); + + const DataTypePtr from_type = removeNullable(arguments[0].type); + ColumnPtr result_column; + + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; + + if (context) + date_time_overflow_behavior = context->getSettingsRef()[Setting::date_time_overflow_behavior].value; + + auto call = [&](const auto & types, BehaviourOnErrorFromString from_string_tag) -> bool + { + using Types = std::decay_t; + using LeftDataType = typename Types::LeftType; + using RightDataType = typename Types::RightType; + + if constexpr (IsDataTypeDecimal) + { + if constexpr (std::is_same_v) + { + /// Account for optional timezone argument. + if (arguments.size() != 2 && arguments.size() != 3) + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects 2 or 3 arguments for DataTypeDateTime64.", getName()); + } + else if (arguments.size() != 2) + { + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects 2 arguments for Decimal.", getName()); + } + + const ColumnWithTypeAndName & scale_column = arguments[1]; + UInt32 scale = extractToDecimalScale(scale_column); + + switch (date_time_overflow_behavior) + { + case FormatSettings::DateTimeOverflowBehavior::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); + break; + case FormatSettings::DateTimeOverflowBehavior::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); + break; + case FormatSettings::DateTimeOverflowBehavior::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, scale); + break; + } + } + else if constexpr (IsDataTypeDateOrDateTime && std::is_same_v) + { + const auto * dt64 = assert_cast(arguments[0].type.get()); + switch (date_time_overflow_behavior) + { + case FormatSettings::DateTimeOverflowBehavior::Throw: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); + break; + case FormatSettings::DateTimeOverflowBehavior::Ignore: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); + break; + case FormatSettings::DateTimeOverflowBehavior::Saturate: + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale()); + break; + } + } + else if constexpr ((IsDataTypeNumber + || IsDataTypeDateOrDateTime)&&IsDataTypeDateOrDateTime) + { +#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \ + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column = ConvertImpl::execute( \ + arguments, result_type, input_rows_count, from_string_tag); \ + break; + switch (date_time_overflow_behavior) + { + GENERATE_OVERFLOW_MODE_CASE(Throw) + GENERATE_OVERFLOW_MODE_CASE(Ignore) + GENERATE_OVERFLOW_MODE_CASE(Saturate) + } + +#undef GENERATE_OVERFLOW_MODE_CASE + } + else if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber) + { + using LeftT = typename LeftDataType::FieldType; + using RightT = typename RightDataType::FieldType; + + static constexpr bool bad_left = + is_decimal || is_floating_point || is_big_int_v || is_signed_v; + static constexpr bool bad_right = + is_decimal || is_floating_point || is_big_int_v || is_signed_v; + + /// Disallow int vs UUID conversion (but support int vs UInt128 conversion) + if constexpr ((bad_left && std::is_same_v) || + (bad_right && std::is_same_v)) + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Wrong UUID conversion"); + } + else + { + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, from_string_tag); + } + } + else + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, from_string_tag); + + return true; + }; + + if constexpr (mightBeDateTime()) + { + if (isDateTime64(arguments)) + { + /// For toDateTime('xxxx-xx-xx xx:xx:xx.00', 2[, 'timezone']) we need to it convert to DateTime64 + const ColumnWithTypeAndName & scale_column = arguments[1]; + UInt32 scale = extractToDecimalScale(scale_column); + + if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64 + { + if (!callOnIndexAndDataType( + from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0].type->getName(), + getName()); + + return result_column; + } + } + } + + if constexpr (std::is_same_v) + { + if (from_type->getCustomSerialization()) + return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); + } + + bool done = false; + if constexpr (is_any_of) + { + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); + } + else + { + bool cast_ipv4_ipv6_default_on_conversion_error = false; + if constexpr (is_any_of) + { + if (context && (cast_ipv4_ipv6_default_on_conversion_error = context->getSettingsRef()[Setting::cast_ipv4_ipv6_default_on_conversion_error])) + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag); + } + + if (!cast_ipv4_ipv6_default_on_conversion_error) + { + /// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString) + /// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values. + if (to_nullable && WhichDataType(from_type).isStringOrFixedString()) + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag); + else + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); + } + + if constexpr (std::is_same_v) + if (WhichDataType(from_type).isInterval()) + done = callOnIndexAndDataType(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); + } + + if (!done) + { + /// Generic conversion of any type to String. + if (std::is_same_v) + { + return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", + arguments[0].type->getName(), getName()); + } + + return result_column; + } +}; + + +/** Function toTOrZero (where T is number of date or datetime type): + * try to convert from String to type T through parsing, + * if cannot parse, return default value instead of throwing exception. + * Function toTOrNull will return Nullable type with NULL when cannot parse. + * NOTE Also need to implement tryToUnixTimestamp with timezone. + */ +template +class FunctionConvertFromString : public IFunction +{ +public: + static constexpr auto name = Name::name; + static constexpr bool to_datetime64 = std::is_same_v; + static constexpr bool to_decimal = IsDataTypeDecimal && !to_datetime64; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool canBeExecutedOnDefaultArguments() const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + DataTypePtr res; + + if (isDateTime64(arguments)) + { + validateFunctionArguments(*this, arguments, + FunctionArgumentDescriptors{{"string", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"}}, + // optional + FunctionArgumentDescriptors{ + {"precision", static_cast(&isUInt8), isColumnConst, "const UInt8"}, + {"timezone", static_cast(&isStringOrFixedString), isColumnConst, "const String or FixedString"}, + }); + + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; + if (arguments.size() > 1) + scale = extractToDecimalScale(arguments[1]); + const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false); + + res = scale == 0 ? res = std::make_shared(timezone) : std::make_shared(scale, timezone); + } + else + { + if ((arguments.size() != 1 && arguments.size() != 2) || (to_decimal && arguments.size() != 2)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2. " + "Second argument only make sense for DateTime (time zone, optional) and Decimal (scale).", + getName(), arguments.size()); + + if (!isStringOrFixedString(arguments[0].type)) + { + if (this->getName().find("OrZero") != std::string::npos || + this->getName().find("OrNull") != std::string::npos) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. " + "Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument", + arguments[0].type->getName(), getName()); + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", + arguments[0].type->getName(), getName()); + } + + if (arguments.size() == 2) + { + if constexpr (std::is_same_v) + { + if (!isString(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}", + arguments[1].type->getName(), getName()); + } + else if constexpr (to_decimal) + { + if (!isInteger(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}", + arguments[1].type->getName(), getName()); + if (!arguments[1].column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant", getName()); + } + else + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1. " + "Second argument makes sense only for DateTime and Decimal.", + getName(), arguments.size()); + } + } + + if constexpr (std::is_same_v) + res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); + else if constexpr (std::is_same_v) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); + else if constexpr (to_decimal) + { + UInt64 scale = extractToDecimalScale(arguments[1]); + res = createDecimalMaxPrecision(scale); + if (!res) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Something wrong with toDecimalNNOrZero() or toDecimalNNOrNull()"); + } + else + res = std::make_shared(); + } + + if constexpr (exception_mode == ConvertFromStringExceptionMode::Null) + res = std::make_shared(res); + + return res; + } + + template + ColumnPtr executeInternal(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, UInt32 scale) const + { + const IDataType * from_type = arguments[0].type.get(); + + if (checkAndGetDataType(from_type)) + { + return ConvertThroughParsing::execute( + arguments, result_type, input_rows_count, scale); + } + else if (checkAndGetDataType(from_type)) + { + return ConvertThroughParsing::execute( + arguments, result_type, input_rows_count, scale); + } + + return nullptr; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + ColumnPtr result_column; + + if constexpr (to_decimal) + { + result_column = executeInternal(arguments, result_type, input_rows_count, + assert_cast(*removeNullable(result_type)).getScale()); + } + else if constexpr (mightBeDateTime()) + { + if (isDateTime64(arguments)) + { + UInt64 scale = to_datetime64 ? DataTypeDateTime64::default_scale : 0; + if (arguments.size() > 1) + scale = extractToDecimalScale(arguments[1]); + + if (scale == 0) + { + result_column = executeInternal(arguments, result_type, input_rows_count, 0); + } + else + { + result_column + = executeInternal(arguments, result_type, input_rows_count, static_cast(scale)); + } + } + else + { + result_column = executeInternal(arguments, result_type, input_rows_count, 0); + } + } + else + { + result_column = executeInternal(arguments, result_type, input_rows_count, 0); + } + + if (!result_column) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " + "Only String or FixedString argument is accepted for try-conversion function. For other arguments, " + "use function without 'orZero' or 'orNull'.", arguments[0].type->getName(), getName()); + + return result_column; + } +}; + + +/// Monotonicity. + +struct PositiveMonotonicity +{ + static bool has() { return true; } + static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &) + { + return { .is_monotonic = true }; + } +}; + +struct UnknownMonotonicity +{ + static bool has() { return false; } + static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &) + { + return { }; + } +}; + +template +struct ToNumberMonotonicity +{ + static bool has() { return true; } + + static UInt64 divideByRangeOfType(UInt64 x) + { + if constexpr (sizeof(T) < sizeof(UInt64)) + return x >> (sizeof(T) * 8); + else + return 0; + } + + static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) + { + if (!type.isValueRepresentedByNumber()) + return {}; + + /// If type is same, the conversion is always monotonic. + /// (Enum has separate case, because it is different data type) + if (checkAndGetDataType>(&type) || + checkAndGetDataType>(&type)) + return { .is_monotonic = true, .is_always_monotonic = true }; + + /// Float cases. + + /// When converting to Float, the conversion is always monotonic. + if constexpr (is_floating_point) + return { .is_monotonic = true, .is_always_monotonic = true }; + + const auto * low_cardinality = typeid_cast(&type); + const IDataType * low_cardinality_dictionary_type = nullptr; + if (low_cardinality) + low_cardinality_dictionary_type = low_cardinality->getDictionaryType().get(); + + WhichDataType which_type(type); + WhichDataType which_inner_type = low_cardinality + ? WhichDataType(low_cardinality_dictionary_type) + : WhichDataType(type); + + /// If converting from Float, for monotonicity, arguments must fit in range of result type. + if (which_inner_type.isFloat()) + { + if (left.isNull() || right.isNull()) + return {}; + + Float64 left_float = left.safeGet(); + Float64 right_float = right.safeGet(); + + if (left_float >= static_cast(std::numeric_limits::min()) + && left_float <= static_cast(std::numeric_limits::max()) + && right_float >= static_cast(std::numeric_limits::min()) + && right_float <= static_cast(std::numeric_limits::max())) + return { .is_monotonic = true }; + + return {}; + } + + /// Integer cases. + + /// Only support types represented by native integers. + /// It can be extended to big integers, decimals and DateTime64 later. + /// By the way, NULLs are representing unbounded ranges. + if (!((left.isNull() || left.getType() == Field::Types::UInt64 || left.getType() == Field::Types::Int64) + && (right.isNull() || right.getType() == Field::Types::UInt64 || right.getType() == Field::Types::Int64))) + return {}; + + const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger(); + const bool to_is_unsigned = is_unsigned_v; + + const size_t size_of_from = type.getSizeOfValueInMemory(); + const size_t size_of_to = sizeof(T); + + const bool left_in_first_half = left.isNull() + ? from_is_unsigned + : (left.safeGet() >= 0); + + const bool right_in_first_half = right.isNull() + ? !from_is_unsigned + : (right.safeGet() >= 0); + + /// Size of type is the same. + if (size_of_from == size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return { .is_monotonic = true, .is_always_monotonic = true }; + + if (left_in_first_half == right_in_first_half) + return { .is_monotonic = true }; + + return {}; + } + + /// Size of type is expanded. + if (size_of_from < size_of_to) + { + if (from_is_unsigned == to_is_unsigned) + return { .is_monotonic = true, .is_always_monotonic = true }; + + if (!to_is_unsigned) + return { .is_monotonic = true, .is_always_monotonic = true }; + + /// signed -> unsigned. If arguments from the same half, then function is monotonic. + if (left_in_first_half == right_in_first_half) + return { .is_monotonic = true }; + + return {}; + } + + /// Size of type is shrunk. + if (size_of_from > size_of_to) + { + /// Function cannot be monotonic on unbounded ranges. + if (left.isNull() || right.isNull()) + return {}; + + /// Function cannot be monotonic when left and right are not on the same ranges. + if (divideByRangeOfType(left.safeGet()) != divideByRangeOfType(right.safeGet())) + return {}; + + if (to_is_unsigned) + return { .is_monotonic = true }; + else + { + // If To is signed, it's possible that the signedness is different after conversion. So we check it explicitly. + const bool is_monotonic = (T(left.safeGet()) >= 0) == (T(right.safeGet()) >= 0); + + return { .is_monotonic = is_monotonic }; + } + } + + UNREACHABLE(); + } +}; + +struct ToDateMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) + { + auto which = WhichDataType(type); + if (which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() || which.isInt8() || which.isInt16() || which.isUInt8() + || which.isUInt16()) + { + return {.is_monotonic = true, .is_always_monotonic = true}; + } + else if ( + ((left.getType() == Field::Types::UInt64 || left.isNull()) && (right.getType() == Field::Types::UInt64 || right.isNull()) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) + || ((left.getType() == Field::Types::Int64 || left.isNull()) && (right.getType() == Field::Types::Int64 || right.isNull()) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF))) + || (( + (left.getType() == Field::Types::Float64 || left.isNull()) + && (right.getType() == Field::Types::Float64 || right.isNull()) + && ((left.isNull() || left.safeGet() < 0xFFFF) && (right.isNull() || right.safeGet() >= 0xFFFF)))) + || !isNativeNumber(type)) + { + return {}; + } + else + { + return {.is_monotonic = true, .is_always_monotonic = true}; + } + } +}; + +struct ToDateTimeMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const IDataType & type, const Field &, const Field &) + { + if (type.isValueRepresentedByNumber()) + return {.is_monotonic = true, .is_always_monotonic = true}; + else + return {}; + } +}; + +/** The monotonicity for the `toString` function is mainly determined for test purposes. + * It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`. + */ +struct ToStringMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) + { + IFunction::Monotonicity positive{ .is_monotonic = true }; + IFunction::Monotonicity not_monotonic; + + const auto * type_ptr = &type; + if (const auto * low_cardinality_type = checkAndGetDataType(type_ptr)) + type_ptr = low_cardinality_type->getDictionaryType().get(); + + /// Order on enum values (which is the order on integers) is completely arbitrary in respect to the order on strings. + if (WhichDataType(type).isEnum()) + return not_monotonic; + + /// `toString` function is monotonous if the argument is Date or Date32 or DateTime or String, or non-negative numbers with the same number of symbols. + if (checkDataTypes(type_ptr)) + return positive; + + if (left.isNull() || right.isNull()) + return {}; + + if (left.getType() == Field::Types::UInt64 + && right.getType() == Field::Types::UInt64) + { + return (left.safeGet() == 0 && right.safeGet() == 0) + || (floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) + ? positive : not_monotonic; + } + + if (left.getType() == Field::Types::Int64 + && right.getType() == Field::Types::Int64) + { + return (left.safeGet() == 0 && right.safeGet() == 0) + || (left.safeGet() > 0 && right.safeGet() > 0 && floor(log10(left.safeGet())) == floor(log10(right.safeGet()))) + ? positive : not_monotonic; + } + + return not_monotonic; + } +}; + + +struct NameToUInt8 { static constexpr auto name = "toUInt8"; }; +struct NameToUInt16 { static constexpr auto name = "toUInt16"; }; +struct NameToUInt32 { static constexpr auto name = "toUInt32"; }; +struct NameToUInt64 { static constexpr auto name = "toUInt64"; }; +struct NameToUInt128 { static constexpr auto name = "toUInt128"; }; +struct NameToUInt256 { static constexpr auto name = "toUInt256"; }; +struct NameToInt8 { static constexpr auto name = "toInt8"; }; +struct NameToInt16 { static constexpr auto name = "toInt16"; }; +struct NameToInt32 { static constexpr auto name = "toInt32"; }; +struct NameToInt64 { static constexpr auto name = "toInt64"; }; +struct NameToInt128 { static constexpr auto name = "toInt128"; }; +struct NameToInt256 { static constexpr auto name = "toInt256"; }; +struct NameToBFloat16 { static constexpr auto name = "toBFloat16"; }; +struct NameToFloat32 { static constexpr auto name = "toFloat32"; }; +struct NameToFloat64 { static constexpr auto name = "toFloat64"; }; +struct NameToUUID { static constexpr auto name = "toUUID"; }; +struct NameToIPv4 { static constexpr auto name = "toIPv4"; }; +struct NameToIPv6 { static constexpr auto name = "toIPv6"; }; + +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; + +extern template class FunctionConvert; + +extern template class FunctionConvert; + +extern template class FunctionConvert; + +extern template class FunctionConvert; + +extern template class FunctionConvert; + +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert>; +extern template class FunctionConvert; +extern template class FunctionConvert>; +extern template class FunctionConvert, NameToDecimal32, UnknownMonotonicity>; +extern template class FunctionConvert, NameToDecimal64, UnknownMonotonicity>; +extern template class FunctionConvert, NameToDecimal128, UnknownMonotonicity>; +extern template class FunctionConvert, NameToDecimal256, UnknownMonotonicity>; + +using FunctionToUInt8 = FunctionConvert>; +using FunctionToUInt16 = FunctionConvert>; +using FunctionToUInt32 = FunctionConvert>; +using FunctionToUInt64 = FunctionConvert>; +using FunctionToUInt128 = FunctionConvert>; +using FunctionToUInt256 = FunctionConvert>; + +using FunctionToInt8 = FunctionConvert>; +using FunctionToInt16 = FunctionConvert>; +using FunctionToInt32 = FunctionConvert>; +using FunctionToInt64 = FunctionConvert>; +using FunctionToInt128 = FunctionConvert>; +using FunctionToInt256 = FunctionConvert>; +using FunctionToBFloat16 = FunctionConvert>; +using FunctionToFloat32 = FunctionConvert>; +using FunctionToFloat64 = FunctionConvert>; + +using FunctionToDate = FunctionConvert; + +using FunctionToDate32 = FunctionConvert; + +using FunctionToDateTime = FunctionConvert; + +using FunctionToDateTime32 = FunctionConvert; + +using FunctionToDateTime64 = FunctionConvert; + +using FunctionToUUID = FunctionConvert>; +using FunctionToIPv4 = FunctionConvert>; +using FunctionToIPv6 = FunctionConvert>; +using FunctionToString = FunctionConvert; +using FunctionToUnixTimestamp = FunctionConvert>; +using FunctionToDecimal32 = FunctionConvert, NameToDecimal32, UnknownMonotonicity>; +using FunctionToDecimal64 = FunctionConvert, NameToDecimal64, UnknownMonotonicity>; +using FunctionToDecimal128 = FunctionConvert, NameToDecimal128, UnknownMonotonicity>; +using FunctionToDecimal256 = FunctionConvert, NameToDecimal256, UnknownMonotonicity>; + +template struct FunctionTo; + +template <> struct FunctionTo { using Type = FunctionToUInt8; }; +template <> struct FunctionTo { using Type = FunctionToUInt16; }; +template <> struct FunctionTo { using Type = FunctionToUInt32; }; +template <> struct FunctionTo { using Type = FunctionToUInt64; }; +template <> struct FunctionTo { using Type = FunctionToUInt128; }; +template <> struct FunctionTo { using Type = FunctionToUInt256; }; +template <> struct FunctionTo { using Type = FunctionToInt8; }; +template <> struct FunctionTo { using Type = FunctionToInt16; }; +template <> struct FunctionTo { using Type = FunctionToInt32; }; +template <> struct FunctionTo { using Type = FunctionToInt64; }; +template <> struct FunctionTo { using Type = FunctionToInt128; }; +template <> struct FunctionTo { using Type = FunctionToInt256; }; +template <> struct FunctionTo { using Type = FunctionToBFloat16; }; +template <> struct FunctionTo { using Type = FunctionToFloat32; }; +template <> struct FunctionTo { using Type = FunctionToFloat64; }; + +template +struct FunctionTo { using Type = FunctionToDate; }; + +template +struct FunctionTo { using Type = FunctionToDate32; }; + +template +struct FunctionTo { using Type = FunctionToDateTime; }; + +template +struct FunctionTo { using Type = FunctionToDateTime64; }; + +template <> struct FunctionTo { using Type = FunctionToUUID; }; +template <> struct FunctionTo { using Type = FunctionToIPv4; }; +template <> struct FunctionTo { using Type = FunctionToIPv6; }; +template <> struct FunctionTo { using Type = FunctionToString; }; +template <> struct FunctionTo { using Type = FunctionToFixedString; }; +template <> struct FunctionTo> { using Type = FunctionToDecimal32; }; +template <> struct FunctionTo> { using Type = FunctionToDecimal64; }; +template <> struct FunctionTo> { using Type = FunctionToDecimal128; }; +template <> struct FunctionTo> { using Type = FunctionToDecimal256; }; + +template struct FunctionTo> + : FunctionTo> +{ +}; + +struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; }; +struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; }; +struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; }; +struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; }; +struct NameToUInt128OrZero { static constexpr auto name = "toUInt128OrZero"; }; +struct NameToUInt256OrZero { static constexpr auto name = "toUInt256OrZero"; }; +struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; }; +struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; }; +struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; }; +struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; }; +struct NameToInt128OrZero { static constexpr auto name = "toInt128OrZero"; }; +struct NameToInt256OrZero { static constexpr auto name = "toInt256OrZero"; }; +struct NameToBFloat16OrZero { static constexpr auto name = "toBFloat16OrZero"; }; +struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; }; +struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; }; +struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; }; +struct NameToDate32OrZero { static constexpr auto name = "toDate32OrZero"; }; +struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; }; +struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; }; +struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; }; +struct NameToDecimal64OrZero { static constexpr auto name = "toDecimal64OrZero"; }; +struct NameToDecimal128OrZero { static constexpr auto name = "toDecimal128OrZero"; }; +struct NameToDecimal256OrZero { static constexpr auto name = "toDecimal256OrZero"; }; +struct NameToUUIDOrZero { static constexpr auto name = "toUUIDOrZero"; }; +struct NameToIPv4OrZero { static constexpr auto name = "toIPv4OrZero"; }; +struct NameToIPv6OrZero { static constexpr auto name = "toIPv6OrZero"; }; + +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>; +extern template class FunctionConvertFromString, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>; +extern template class FunctionConvertFromString, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>; +extern template class FunctionConvertFromString, NameToDecimal256OrZero, ConvertFromStringExceptionMode::Zero>; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; + +using FunctionToUInt8OrZero = FunctionConvertFromString; +using FunctionToUInt16OrZero = FunctionConvertFromString; +using FunctionToUInt32OrZero = FunctionConvertFromString; +using FunctionToUInt64OrZero = FunctionConvertFromString; +using FunctionToUInt128OrZero = FunctionConvertFromString; +using FunctionToUInt256OrZero = FunctionConvertFromString; +using FunctionToInt8OrZero = FunctionConvertFromString; +using FunctionToInt16OrZero = FunctionConvertFromString; +using FunctionToInt32OrZero = FunctionConvertFromString; +using FunctionToInt64OrZero = FunctionConvertFromString; +using FunctionToInt128OrZero = FunctionConvertFromString; +using FunctionToInt256OrZero = FunctionConvertFromString; +using FunctionToBFloat16OrZero = FunctionConvertFromString; +using FunctionToFloat32OrZero = FunctionConvertFromString; +using FunctionToFloat64OrZero = FunctionConvertFromString; +using FunctionToDateOrZero = FunctionConvertFromString; +using FunctionToDate32OrZero = FunctionConvertFromString; +using FunctionToDateTimeOrZero = FunctionConvertFromString; +using FunctionToDateTime64OrZero = FunctionConvertFromString; +using FunctionToDecimal32OrZero = FunctionConvertFromString, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>; +using FunctionToDecimal64OrZero = FunctionConvertFromString, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>; +using FunctionToDecimal128OrZero = FunctionConvertFromString, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>; +using FunctionToDecimal256OrZero = FunctionConvertFromString, NameToDecimal256OrZero, ConvertFromStringExceptionMode::Zero>; +using FunctionToUUIDOrZero = FunctionConvertFromString; +using FunctionToIPv4OrZero = FunctionConvertFromString; +using FunctionToIPv6OrZero = FunctionConvertFromString; + +struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; }; +struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; }; +struct NameToUInt32OrNull { static constexpr auto name = "toUInt32OrNull"; }; +struct NameToUInt64OrNull { static constexpr auto name = "toUInt64OrNull"; }; +struct NameToUInt128OrNull { static constexpr auto name = "toUInt128OrNull"; }; +struct NameToUInt256OrNull { static constexpr auto name = "toUInt256OrNull"; }; +struct NameToInt8OrNull { static constexpr auto name = "toInt8OrNull"; }; +struct NameToInt16OrNull { static constexpr auto name = "toInt16OrNull"; }; +struct NameToInt32OrNull { static constexpr auto name = "toInt32OrNull"; }; +struct NameToInt64OrNull { static constexpr auto name = "toInt64OrNull"; }; +struct NameToInt128OrNull { static constexpr auto name = "toInt128OrNull"; }; +struct NameToInt256OrNull { static constexpr auto name = "toInt256OrNull"; }; +struct NameToBFloat16OrNull { static constexpr auto name = "toBFloat16OrNull"; }; +struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; }; +struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; }; +struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; }; +struct NameToDate32OrNull { static constexpr auto name = "toDate32OrNull"; }; +struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; }; +struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; }; +struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; }; +struct NameToDecimal64OrNull { static constexpr auto name = "toDecimal64OrNull"; }; +struct NameToDecimal128OrNull { static constexpr auto name = "toDecimal128OrNull"; }; +struct NameToDecimal256OrNull { static constexpr auto name = "toDecimal256OrNull"; }; +struct NameToUUIDOrNull { static constexpr auto name = "toUUIDOrNull"; }; +struct NameToIPv4OrNull { static constexpr auto name = "toIPv4OrNull"; }; +struct NameToIPv6OrNull { static constexpr auto name = "toIPv6OrNull"; }; + +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>; +extern template class FunctionConvertFromString, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>; +extern template class FunctionConvertFromString, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>; +extern template class FunctionConvertFromString, NameToDecimal256OrNull, ConvertFromStringExceptionMode::Null>; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; +extern template class FunctionConvertFromString; + +using FunctionToUInt8OrNull = FunctionConvertFromString; +using FunctionToUInt16OrNull = FunctionConvertFromString; +using FunctionToUInt32OrNull = FunctionConvertFromString; +using FunctionToUInt64OrNull = FunctionConvertFromString; +using FunctionToUInt128OrNull = FunctionConvertFromString; +using FunctionToUInt256OrNull = FunctionConvertFromString; +using FunctionToInt8OrNull = FunctionConvertFromString; +using FunctionToInt16OrNull = FunctionConvertFromString; +using FunctionToInt32OrNull = FunctionConvertFromString; +using FunctionToInt64OrNull = FunctionConvertFromString; +using FunctionToInt128OrNull = FunctionConvertFromString; +using FunctionToInt256OrNull = FunctionConvertFromString; +using FunctionToBFloat16OrNull = FunctionConvertFromString; +using FunctionToFloat32OrNull = FunctionConvertFromString; +using FunctionToFloat64OrNull = FunctionConvertFromString; +using FunctionToDateOrNull = FunctionConvertFromString; +using FunctionToDate32OrNull = FunctionConvertFromString; +using FunctionToDateTimeOrNull = FunctionConvertFromString; +using FunctionToDateTime64OrNull = FunctionConvertFromString; +using FunctionToDecimal32OrNull = FunctionConvertFromString, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>; +using FunctionToDecimal64OrNull = FunctionConvertFromString, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>; +using FunctionToDecimal128OrNull = FunctionConvertFromString, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>; +using FunctionToDecimal256OrNull = FunctionConvertFromString, NameToDecimal256OrNull, ConvertFromStringExceptionMode::Null>; +using FunctionToUUIDOrNull = FunctionConvertFromString; +using FunctionToIPv4OrNull = FunctionConvertFromString; +using FunctionToIPv6OrNull = FunctionConvertFromString; + +struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; }; +struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; }; +struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; }; +struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; }; +struct NameParseDateTimeBestEffortUSOrZero { static constexpr auto name = "parseDateTimeBestEffortUSOrZero"; }; +struct NameParseDateTimeBestEffortUSOrNull { static constexpr auto name = "parseDateTimeBestEffortUSOrNull"; }; +struct NameParseDateTime32BestEffort { static constexpr auto name = "parseDateTime32BestEffort"; }; +struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = "parseDateTime32BestEffortOrZero"; }; +struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = "parseDateTime32BestEffortOrNull"; }; +struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; }; +struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; }; +struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; }; +struct NameParseDateTime64BestEffortUS { static constexpr auto name = "parseDateTime64BestEffortUS"; }; +struct NameParseDateTime64BestEffortUSOrZero { static constexpr auto name = "parseDateTime64BestEffortUSOrZero"; }; +struct NameParseDateTime64BestEffortUSOrNull { static constexpr auto name = "parseDateTime64BestEffortUSOrNull"; }; + +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +extern template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + +using FunctionParseDateTimeBestEffort = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +using FunctionParseDateTimeBestEffortUS = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +using FunctionParseDateTimeBestEffortUSOrZero = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +using FunctionParseDateTimeBestEffortUSOrNull = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + +using FunctionParseDateTime32BestEffort = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime32BestEffortOrNull = FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +using FunctionParseDateTime64BestEffort = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +using FunctionParseDateTime64BestEffortOrNull = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +using FunctionParseDateTime64BestEffortUS = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +using FunctionParseDateTime64BestEffortUSOrZero = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +using FunctionParseDateTime64BestEffortUSOrNull = FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + + +class ExecutableFunctionCast : public IExecutableFunction +{ +public: + using WrapperType = std::function; + + explicit ExecutableFunctionCast( + WrapperType && wrapper_function_, const char * name_, std::optional diagnostic_) + : wrapper_function(std::move(wrapper_function_)), name(name_), diagnostic(std::move(diagnostic_)) {} + + String getName() const override { return name; } + +protected: + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + /// drop second argument, pass others + ColumnsWithTypeAndName new_arguments{arguments.front()}; + if (arguments.size() > 2) + new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments)); + + try + { + return wrapper_function(new_arguments, result_type, nullptr, input_rows_count); + } + catch (Exception & e) + { + if (diagnostic) + e.addMessage("while converting source column " + backQuoteIfNeed(diagnostic->column_from) + + " to destination column " + backQuoteIfNeed(diagnostic->column_to)); + throw; + } + } + + bool useDefaultImplementationForNulls() const override { return false; } + /// CAST(Nothing, T) -> T + bool useDefaultImplementationForNothing() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + +private: + WrapperType wrapper_function; + const char * name; + std::optional diagnostic; +}; + + +struct FunctionCastName +{ + static constexpr auto name = "CAST"; +}; + +class FunctionCast final : public IFunctionBase +{ +public: + using MonotonicityForRange = std::function; + using WrapperType = std::function; + + FunctionCast(ContextPtr context_ + , const char * cast_name_ + , MonotonicityForRange && monotonicity_for_range_ + , const DataTypes & argument_types_ + , const DataTypePtr & return_type_ + , std::optional diagnostic_ + , CastType cast_type_) + : cast_name(cast_name_), monotonicity_for_range(std::move(monotonicity_for_range_)) + , argument_types(argument_types_), return_type(return_type_), diagnostic(std::move(diagnostic_)) + , cast_type(cast_type_) + , context(context_) + { + } + + const DataTypes & getArgumentTypes() const override { return argument_types; } + const DataTypePtr & getResultType() const override { return return_type; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & /*sample_columns*/) const override + { + try + { + return std::make_unique( + prepareUnpackDictionaries(getArgumentTypes()[0], getResultType()), cast_name, diagnostic); + } + catch (Exception & e) + { + if (diagnostic) + e.addMessage("while converting source column " + backQuoteIfNeed(diagnostic->column_from) + + " to destination column " + backQuoteIfNeed(diagnostic->column_to)); + throw; + } + } + + String getName() const override { return cast_name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool hasInformationAboutMonotonicity() const override + { + return static_cast(monotonicity_for_range); + } + + Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override + { + return monotonicity_for_range(type, left, right); + } + +private: + const char * cast_name; + MonotonicityForRange monotonicity_for_range; + + DataTypes argument_types; + DataTypePtr return_type; + + std::optional diagnostic; + CastType cast_type; + ContextPtr context; + + static WrapperType createFunctionAdaptor(FunctionPtr function, const DataTypePtr & from_type) + { + auto function_adaptor = std::make_unique(function)->build({ColumnWithTypeAndName{nullptr, from_type, ""}}); + + return [function_adaptor]( + ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) + { return function_adaptor->execute(arguments, result_type, input_rows_count, /* dry_run = */ false); }; + } + + static WrapperType createToNullableColumnWrapper() + { + return [] (ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) + { + ColumnPtr res = result_type->createColumn(); + ColumnUInt8::Ptr col_null_map_to = ColumnUInt8::create(input_rows_count, true); + return ColumnNullable::create(res->cloneResized(input_rows_count), std::move(col_null_map_to)); + }; + } + + template + WrapperType createWrapper(const DataTypePtr & from_type, const ToDataType * const to_type, bool requested_result_is_nullable) const + { + TypeIndex from_type_index = from_type->getTypeId(); + WhichDataType which(from_type_index); + TypeIndex to_type_index = to_type->getTypeId(); + WhichDataType to(to_type_index); + bool can_apply_accurate_cast = (cast_type == CastType::accurate || cast_type == CastType::accurateOrNull) + && (which.isInt() || which.isUInt() || which.isFloat()); + can_apply_accurate_cast |= cast_type == CastType::accurate && which.isStringOrFixedString() && to.isNativeInteger(); + + FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior; + if (context) + date_time_overflow_behavior = context->getSettingsRef()[Setting::date_time_overflow_behavior]; + + if (requested_result_is_nullable && checkAndGetDataType(from_type.get())) + { + /// In case when converting to Nullable type, we apply different parsing rule, + /// that will not throw an exception but return NULL in case of malformed input. + FunctionPtr function = FunctionConvertFromString::create(context); + return createFunctionAdaptor(function, from_type); + } + else if (!can_apply_accurate_cast) + { + FunctionPtr function = FunctionTo::Type::create(context); + return createFunctionAdaptor(function, from_type); + } + + return [wrapper_cast_type = cast_type, from_type_index, to_type, date_time_overflow_behavior] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) + { + ColumnPtr result_column; + auto res = callOnIndexAndDataType(from_type_index, [&](const auto & types) -> bool + { + using Types = std::decay_t; + using LeftDataType = typename Types::LeftType; + using RightDataType = typename Types::RightType; + + if constexpr (IsDataTypeNumber) + { + if constexpr (IsDataTypeDateOrDateTime) + { +#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \ + case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \ + result_column \ + = ConvertImpl:: \ + execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, ADDITIONS()); \ + break; + if (wrapper_cast_type == CastType::accurate) + { + switch (date_time_overflow_behavior) + { + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateConvertStrategyAdditions) + } + } + else + { + switch (date_time_overflow_behavior) + { + GENERATE_OVERFLOW_MODE_CASE(Throw, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Ignore, DateTimeAccurateOrNullConvertStrategyAdditions) + GENERATE_OVERFLOW_MODE_CASE(Saturate, DateTimeAccurateOrNullConvertStrategyAdditions) + } + } +#undef GENERATE_OVERFLOW_MODE_CASE + + return true; + } + else if constexpr (IsDataTypeNumber) + { + if (wrapper_cast_type == CastType::accurate) + { + result_column = ConvertImpl::execute( + arguments, + result_type, + input_rows_count, + BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, + AccurateConvertStrategyAdditions()); + } + else + { + result_column = ConvertImpl::execute( + arguments, + result_type, + input_rows_count, + BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, + AccurateOrNullConvertStrategyAdditions()); + } + + return true; + } + } + else if constexpr (IsDataTypeStringOrFixedString) + { + if constexpr (IsDataTypeNumber) + { + chassert(wrapper_cast_type == CastType::accurate); + result_column = ConvertImpl::execute( + arguments, + result_type, + input_rows_count, + BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, + AccurateConvertStrategyAdditions()); + } + return true; + } + + return false; + }); + + /// Additionally check if callOnIndexAndDataType wasn't called at all. + if (!res) + { + if (wrapper_cast_type == CastType::accurateOrNull) + { + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, + "Conversion from {} to {} is not supported", + from_type_index, to_type->getName()); + } + } + + return result_column; + }; + } + + template + WrapperType createBoolWrapper(const DataTypePtr & from_type, const ToDataType * const to_type, bool requested_result_is_nullable) const + { + if (checkAndGetDataType(from_type.get())) + { + if (cast_type == CastType::accurateOrNull) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); + }; + } + + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); + }; + } + + return createWrapper(from_type, to_type, requested_result_is_nullable); + } + + WrapperType createUInt8ToBoolWrapper(const DataTypePtr from_type, const DataTypePtr to_type) const + { + return [from_type, to_type] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) -> ColumnPtr + { + /// Special case when we convert UInt8 column to Bool column. + /// both columns have type UInt8, but we shouldn't use identity wrapper, + /// because Bool column can contain only 0 and 1. + auto res_column = to_type->createColumn(); + const auto & data_from = checkAndGetColumn(*arguments[0].column).getData(); + auto & data_to = assert_cast(res_column.get())->getData(); + data_to.resize(data_from.size()); + for (size_t i = 0; i != data_from.size(); ++i) + data_to[i] = static_cast(data_from[i]); + return res_column; + }; + } + + WrapperType createStringWrapper(const DataTypePtr & from_type) const + { + FunctionPtr function = FunctionToString::create(context); + return createFunctionAdaptor(function, from_type); + } + + WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N) const + { + if (!isStringOrFixedString(from_type)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CAST AS FixedString is only implemented for types String and FixedString"); + + bool exception_mode_null = cast_type == CastType::accurateOrNull; + return [exception_mode_null, N] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) + { + if (exception_mode_null) + return FunctionToFixedString::executeForN(arguments, N); + else + return FunctionToFixedString::executeForN(arguments, N); + }; + } + +#define GENERATE_INTERVAL_CASE(INTERVAL_KIND) \ + case IntervalKind::Kind::INTERVAL_KIND: \ + return createFunctionAdaptor(FunctionConvert::create(context), from_type); + + WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) const + { + switch (kind.kind) + { + GENERATE_INTERVAL_CASE(Nanosecond) + GENERATE_INTERVAL_CASE(Microsecond) + GENERATE_INTERVAL_CASE(Millisecond) + GENERATE_INTERVAL_CASE(Second) + GENERATE_INTERVAL_CASE(Minute) + GENERATE_INTERVAL_CASE(Hour) + GENERATE_INTERVAL_CASE(Day) + GENERATE_INTERVAL_CASE(Week) + GENERATE_INTERVAL_CASE(Month) + GENERATE_INTERVAL_CASE(Quarter) + GENERATE_INTERVAL_CASE(Year) + } + throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion to unexpected IntervalKind: {}", kind.toString()}; + } + +#undef GENERATE_INTERVAL_CASE + + template + requires IsDataTypeDecimal + WrapperType createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const + { + TypeIndex type_index = from_type->getTypeId(); + UInt32 scale = to_type->getScale(); + + WhichDataType which(type_index); + bool ok = which.isNativeInt() || which.isNativeUInt() || which.isDecimal() || which.isFloat() || which.isDateOrDate32() || which.isDateTime() || which.isDateTime64() + || which.isStringOrFixedString(); + if (!ok) + { + if (cast_type == CastType::accurateOrNull) + return createToNullableColumnWrapper(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", + from_type->getName(), to_type->getName()); + } + + auto wrapper_cast_type = cast_type; + + return [wrapper_cast_type, type_index, scale, to_type, requested_result_is_nullable] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *column_nullable, size_t input_rows_count) + { + ColumnPtr result_column; + auto res = callOnIndexAndDataType(type_index, [&](const auto & types) -> bool + { + using Types = std::decay_t; + using LeftDataType = typename Types::LeftType; + using RightDataType = typename Types::RightType; + + if constexpr (IsDataTypeDecimalOrNumber && IsDataTypeDecimalOrNumber && !std::is_same_v) + { + if (wrapper_cast_type == CastType::accurate) + { + AccurateConvertStrategyAdditions additions; + additions.scale = scale; + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions); + + return true; + } + else if (wrapper_cast_type == CastType::accurateOrNull) + { + AccurateOrNullConvertStrategyAdditions additions; + additions.scale = scale; + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions); + + return true; + } + } + else if constexpr (std::is_same_v) + { + if (requested_result_is_nullable) + { + /// Consistent with CAST(Nullable(String) AS Nullable(Numbers)) + /// In case when converting to Nullable type, we apply different parsing rule, + /// that will not throw an exception but return NULL in case of malformed input. + result_column = ConvertImpl::execute( + arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag, scale); + + return true; + } + } + + result_column = ConvertImpl::execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, scale); + + return true; + }); + + /// Additionally check if callOnIndexAndDataType wasn't called at all. + if (!res) + { + if (wrapper_cast_type == CastType::accurateOrNull) + { + auto nullable_column_wrapper = FunctionCast::createToNullableColumnWrapper(); + return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count); + } + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, + "Conversion from {} to {} is not supported", + type_index, to_type->getName()); + } + + return result_column; + }; + } + + WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type) const + { + /// Conversion from String through parsing. + if (checkAndGetDataType(from_type_untyped.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); + }; + } + else if (const auto * agg_type = checkAndGetDataType(from_type_untyped.get())) + { + if (agg_type->getFunction()->haveSameStateRepresentation(*to_type->getFunction())) + { + return [function = to_type->getFunction()]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & /* result_type */, + const ColumnNullable * /* nullable_source */, + size_t /*input_rows_count*/) -> ColumnPtr + { + const auto & argument_column = arguments.front(); + const auto * col_agg = checkAndGetColumn(argument_column.column.get()); + if (col_agg) + { + auto new_col_agg = ColumnAggregateFunction::create(*col_agg); + new_col_agg->set(function); + return new_col_agg; + } + else + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Illegal column {} for function CAST AS AggregateFunction", + argument_column.column->getName()); + } + }; + } + } + + if (cast_type == CastType::accurateOrNull) + return createToNullableColumnWrapper(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", + from_type_untyped->getName(), to_type->getName()); + } + + WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray & to_type) const + { + /// Conversion from String through parsing. + if (checkAndGetDataType(from_type_untyped.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); + }; + } + + DataTypePtr from_type_holder; + const auto * from_type = checkAndGetDataType(from_type_untyped.get()); + const auto * from_type_map = checkAndGetDataType(from_type_untyped.get()); + + /// Convert from Map + if (from_type_map) + { + /// Recreate array of unnamed tuples because otherwise it may work + /// unexpectedly while converting to array of named tuples. + from_type_holder = from_type_map->getNestedTypeWithUnnamedTuple(); + from_type = assert_cast(from_type_holder.get()); + } + + if (!from_type) + { + throw Exception(ErrorCodes::TYPE_MISMATCH, + "CAST AS Array can only be performed between same-dimensional Array, Map or String types"); + } + + DataTypePtr from_nested_type = from_type->getNestedType(); + + /// In query SELECT CAST([] AS Array(Array(String))) from type is Array(Nothing) + bool from_empty_array = isNothing(from_nested_type); + + if (from_type->getNumberOfDimensions() != to_type.getNumberOfDimensions() && !from_empty_array) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "CAST AS Array can only be performed between same-dimensional array types"); + + const DataTypePtr & to_nested_type = to_type.getNestedType(); + + /// Prepare nested type conversion + const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type); + + return [nested_function, from_nested_type, to_nested_type]( + ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr + { + const auto & argument_column = arguments.front(); + + const ColumnArray * col_array = nullptr; + + if (const ColumnMap * col_map = checkAndGetColumn(argument_column.column.get())) + col_array = &col_map->getNestedColumn(); + else + col_array = checkAndGetColumn(argument_column.column.get()); + + if (col_array) + { + /// create columns for converting nested column containing original and result columns + ColumnsWithTypeAndName nested_columns{{ col_array->getDataPtr(), from_nested_type, "" }}; + + /// convert nested column + auto result_column = nested_function(nested_columns, to_nested_type, nullable_source, nested_columns.front().column->size()); + + /// set converted nested column to result + return ColumnArray::create(result_column, col_array->getOffsetsPtr()); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Illegal column {} for function CAST AS Array", + argument_column.column->getName()); + } + }; + } + + using ElementWrappers = std::vector; + + ElementWrappers getElementWrappers(const DataTypes & from_element_types, const DataTypes & to_element_types) const + { + ElementWrappers element_wrappers; + element_wrappers.reserve(from_element_types.size()); + + /// Create conversion wrapper for each element in tuple + for (size_t i = 0; i < from_element_types.size(); ++i) + { + const DataTypePtr & from_element_type = from_element_types[i]; + const DataTypePtr & to_element_type = to_element_types[i]; + element_wrappers.push_back(prepareUnpackDictionaries(from_element_type, to_element_type)); + } + + return element_wrappers; + } + + WrapperType createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type) const + { + /// Conversion from String through parsing. + if (checkAndGetDataType(from_type_untyped.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericFromString::execute(arguments, result_type, column_nullable, input_rows_count, context); + }; + } + + const auto * from_type = checkAndGetDataType(from_type_untyped.get()); + if (!from_type) + throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types or from String.\n" + "Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName()); + + const auto & from_element_types = from_type->getElements(); + const auto & to_element_types = to_type->getElements(); + + std::vector element_wrappers; + std::vector> to_reverse_index; + + /// For named tuples allow conversions for tuples with + /// different sets of elements. If element exists in @to_type + /// and doesn't exist in @to_type it will be filled by default values. + if (from_type->haveExplicitNames() && to_type->haveExplicitNames()) + { + const auto & from_names = from_type->getElementNames(); + std::unordered_map from_positions; + from_positions.reserve(from_names.size()); + for (size_t i = 0; i < from_names.size(); ++i) + from_positions[from_names[i]] = i; + + const auto & to_names = to_type->getElementNames(); + element_wrappers.reserve(to_names.size()); + to_reverse_index.reserve(from_names.size()); + + for (size_t i = 0; i < to_names.size(); ++i) + { + auto it = from_positions.find(to_names[i]); + if (it != from_positions.end()) + { + element_wrappers.emplace_back(prepareUnpackDictionaries(from_element_types[it->second], to_element_types[i])); + to_reverse_index.emplace_back(it->second); + } + else + { + element_wrappers.emplace_back(); + to_reverse_index.emplace_back(); + } + } + } + else + { + if (from_element_types.size() != to_element_types.size()) + throw Exception(ErrorCodes::TYPE_MISMATCH, "CAST AS Tuple can only be performed between tuple types " + "with the same number of elements or from String.\nLeft type: {}, right type: {}", + from_type->getName(), to_type->getName()); + + element_wrappers = getElementWrappers(from_element_types, to_element_types); + to_reverse_index.reserve(to_element_types.size()); + for (size_t i = 0; i < to_element_types.size(); ++i) + to_reverse_index.emplace_back(i); + } + + return [element_wrappers, from_element_types, to_element_types, to_reverse_index] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count) -> ColumnPtr + { + const auto * col = arguments.front().column.get(); + + size_t tuple_size = to_element_types.size(); + const ColumnTuple & column_tuple = typeid_cast(*col); + + Columns converted_columns(tuple_size); + + /// invoke conversion for each element + for (size_t i = 0; i < tuple_size; ++i) + { + if (to_reverse_index[i]) + { + size_t from_idx = *to_reverse_index[i]; + ColumnsWithTypeAndName element = {{column_tuple.getColumns()[from_idx], from_element_types[from_idx], "" }}; + converted_columns[i] = element_wrappers[i](element, to_element_types[i], nullable_source, input_rows_count); + } + else + { + converted_columns[i] = to_element_types[i]->createColumn()->cloneResized(input_rows_count); + } + } + + return ColumnTuple::create(converted_columns); + }; + } + + /// The case of: tuple([key1, key2, ..., key_n], [value1, value2, ..., value_n]) + WrapperType createTupleToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const + { + return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr + { + const auto * col = arguments.front().column.get(); + const auto & column_tuple = assert_cast(*col); + + Columns offsets(2); + Columns converted_columns(2); + for (size_t i = 0; i < 2; ++i) + { + const auto & column_array = assert_cast(column_tuple.getColumn(i)); + ColumnsWithTypeAndName element = {{column_array.getDataPtr(), from_kv_types[i], ""}}; + converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); + offsets[i] = column_array.getOffsetsPtr(); + } + + const auto & keys_offsets = assert_cast(*offsets[0]).getData(); + const auto & values_offsets = assert_cast(*offsets[1]).getData(); + if (keys_offsets != values_offsets) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "CAST AS Map can only be performed from tuple of arrays with equal sizes."); + + return ColumnMap::create(converted_columns[0], converted_columns[1], offsets[0]); + }; + } + + WrapperType createMapToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const + { + return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr + { + const auto * col = arguments.front().column.get(); + const auto & column_map = typeid_cast(*col); + const auto & nested_data = column_map.getNestedData(); + + Columns converted_columns(2); + for (size_t i = 0; i < 2; ++i) + { + ColumnsWithTypeAndName element = {{nested_data.getColumnPtr(i), from_kv_types[i], ""}}; + converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); + } + + return ColumnMap::create(converted_columns[0], converted_columns[1], column_map.getNestedColumn().getOffsetsPtr()); + }; + } + + /// The case of: [(key1, value1), (key2, value2), ...] + WrapperType createArrayToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const + { + return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr + { + const auto * col = arguments.front().column.get(); + const auto & column_array = typeid_cast(*col); + const auto & nested_data = typeid_cast(column_array.getData()); + + Columns converted_columns(2); + for (size_t i = 0; i < 2; ++i) + { + ColumnsWithTypeAndName element = {{nested_data.getColumnPtr(i), from_kv_types[i], ""}}; + converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); + } + + return ColumnMap::create(converted_columns[0], converted_columns[1], column_array.getOffsetsPtr()); + }; + } + + + WrapperType createMapWrapper(const DataTypePtr & from_type_untyped, const DataTypeMap * to_type) const + { + if (const auto * from_tuple = checkAndGetDataType(from_type_untyped.get())) + { + if (from_tuple->getElements().size() != 2) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "CAST AS Map from tuple requires 2 elements. " + "Left type: {}, right type: {}", + from_tuple->getName(), + to_type->getName()); + + DataTypes from_kv_types; + const auto & to_kv_types = to_type->getKeyValueTypes(); + + for (const auto & elem : from_tuple->getElements()) + { + const auto * type_array = checkAndGetDataType(elem.get()); + if (!type_array) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "CAST AS Map can only be performed from tuples of array. Got: {}", from_tuple->getName()); + + from_kv_types.push_back(type_array->getNestedType()); + } + + return createTupleToMapWrapper(from_kv_types, to_kv_types); + } + else if (const auto * from_array = typeid_cast(from_type_untyped.get())) + { + if (typeid_cast(from_array->getNestedType().get())) + return [nested = to_type->getNestedType()](ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t size) + { + return ColumnMap::create(nested->createColumnConstWithDefaultValue(size)->convertToFullColumnIfConst()); + }; + + const auto * nested_tuple = typeid_cast(from_array->getNestedType().get()); + if (!nested_tuple || nested_tuple->getElements().size() != 2) + throw Exception( + ErrorCodes::TYPE_MISMATCH, + "CAST AS Map from array requires nested tuple of 2 elements. " + "Left type: {}, right type: {}", + from_array->getName(), + to_type->getName()); + + return createArrayToMapWrapper(nested_tuple->getElements(), to_type->getKeyValueTypes()); + } + else if (const auto * from_type = checkAndGetDataType(from_type_untyped.get())) + { + return createMapToMapWrapper(from_type->getKeyValueTypes(), to_type->getKeyValueTypes()); + } + else + { + throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported types to CAST AS Map. " + "Left type: {}, right type: {}", from_type_untyped->getName(), to_type->getName()); + } + } + + WrapperType createTupleToObjectDeprecatedWrapper(const DataTypeTuple & from_tuple, bool has_nullable_subcolumns) const + { + if (!from_tuple.haveExplicitNames()) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Cast to Object can be performed only from flatten Named Tuple. Got: {}", from_tuple.getName()); + + PathsInData paths; + DataTypes from_types; + + std::tie(paths, from_types) = flattenTuple(from_tuple.getPtr()); + auto to_types = from_types; + + for (auto & type : to_types) + { + if (isTuple(type) || isNested(type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Cast to Object can be performed only from flatten Named Tuple. Got: {}", + from_tuple.getName()); + + type = recursiveRemoveLowCardinality(type); + } + + return [element_wrappers = getElementWrappers(from_types, to_types), + has_nullable_subcolumns, from_types, to_types, paths] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count) + { + size_t tuple_size = to_types.size(); + auto flattened_column = flattenTuple(arguments.front().column); + const auto & column_tuple = assert_cast(*flattened_column); + + if (tuple_size != column_tuple.getColumns().size()) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Expected tuple with {} subcolumn, but got {} subcolumns", + tuple_size, column_tuple.getColumns().size()); + + auto res = ColumnObjectDeprecated::create(has_nullable_subcolumns); + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnsWithTypeAndName element = {{column_tuple.getColumns()[i], from_types[i], "" }}; + auto converted_column = element_wrappers[i](element, to_types[i], nullable_source, input_rows_count); + res->addSubcolumn(paths[i], converted_column->assumeMutable()); + } + + return res; + }; + } + + WrapperType createMapToObjectDeprecatedWrapper(const DataTypeMap & from_map, bool has_nullable_subcolumns) const + { + auto key_value_types = from_map.getKeyValueTypes(); + + if (!isStringOrFixedString(key_value_types[0])) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Cast to Object from Map can be performed only from Map " + "with String or FixedString key. Got: {}", from_map.getName()); + + const auto & value_type = key_value_types[1]; + auto to_value_type = value_type; + + if (!has_nullable_subcolumns && value_type->isNullable()) + to_value_type = removeNullable(value_type); + + if (has_nullable_subcolumns && !value_type->isNullable()) + to_value_type = makeNullable(value_type); + + DataTypes to_key_value_types{std::make_shared(), std::move(to_value_type)}; + auto element_wrappers = getElementWrappers(key_value_types, to_key_value_types); + + return [has_nullable_subcolumns, element_wrappers, key_value_types, to_key_value_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t) -> ColumnPtr + { + const auto & column_map = assert_cast(*arguments.front().column); + const auto & offsets = column_map.getNestedColumn().getOffsets(); + auto key_value_columns = column_map.getNestedData().getColumnsCopy(); + + for (size_t i = 0; i < 2; ++i) + { + ColumnsWithTypeAndName element{{key_value_columns[i], key_value_types[i], ""}}; + key_value_columns[i] = element_wrappers[i](element, to_key_value_types[i], nullable_source, key_value_columns[i]->size()); + } + + const auto & key_column_str = assert_cast(*key_value_columns[0]); + const auto & value_column = *key_value_columns[1]; + + using SubcolumnsMap = HashMap; + SubcolumnsMap subcolumns; + + for (size_t row = 0; row < offsets.size(); ++row) + { + for (size_t i = offsets[static_cast(row) - 1]; i < offsets[row]; ++i) + { + auto ref = key_column_str.getDataAt(i); + + bool inserted; + SubcolumnsMap::LookupResult it; + subcolumns.emplace(ref, it, inserted); + auto & subcolumn = it->getMapped(); + + if (inserted) + subcolumn = value_column.cloneEmpty()->cloneResized(row); + + /// Map can have duplicated keys. We insert only first one. + if (subcolumn->size() == row) + subcolumn->insertFrom(value_column, i); + } + + /// Insert default values for keys missed in current row. + for (const auto & [_, subcolumn] : subcolumns) + if (subcolumn->size() == row) + subcolumn->insertDefault(); + } + + auto column_object = ColumnObjectDeprecated::create(has_nullable_subcolumns); + for (auto && [key, subcolumn] : subcolumns) + { + PathInData path(key.toView()); + column_object->addSubcolumn(path, std::move(subcolumn)); + } + + return column_object; + }; + } + + WrapperType createObjectDeprecatedWrapper(const DataTypePtr & from_type, const DataTypeObjectDeprecated * to_type) const + { + if (const auto * from_tuple = checkAndGetDataType(from_type.get())) + { + return createTupleToObjectDeprecatedWrapper(*from_tuple, to_type->hasNullableSubcolumns()); + } + else if (const auto * from_map = checkAndGetDataType(from_type.get())) + { + return createMapToObjectDeprecatedWrapper(*from_map, to_type->hasNullableSubcolumns()); + } + else if (checkAndGetDataType(from_type.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) + { + auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context)->assumeMutable(); + res->finalize(); + return res; + }; + } + else if (checkAndGetDataType(from_type.get())) + { + return [is_nullable = to_type->hasNullableSubcolumns()] (ColumnsWithTypeAndName & arguments, const DataTypePtr & , const ColumnNullable * , size_t) -> ColumnPtr + { + const auto & column_object = assert_cast(*arguments.front().column); + auto res = ColumnObjectDeprecated::create(is_nullable); + for (size_t i = 0; i < column_object.size(); i++) + res->insert(column_object[i]); + + res->finalize(); + return res; + }; + } + + throw Exception(ErrorCodes::TYPE_MISMATCH, + "Cast to Object can be performed only from flatten named Tuple, Map or String. Got: {}", from_type->getName()); + } + + + WrapperType createObjectWrapper(const DataTypePtr & from_type, const DataTypeObject * to_object) const + { + if (checkAndGetDataType(from_type.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) + { + return ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count, context); + }; + } + + /// Cast Tuple/Object/Map to JSON type through serializing into JSON string and parsing back into JSON column. + /// Potentially we can do smarter conversion Tuple -> JSON with type preservation, but it's questionable how exactly Tuple should be + /// converted to JSON (for example, should we recursively convert nested Array(Tuple) to Array(JSON) or not, should we infer types from String fields, etc). + if (checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get()) || checkAndGetDataType(from_type.get())) + { + return [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) + { + auto json_string = ColumnString::create(); + ColumnStringHelpers::WriteHelper write_helper(assert_cast(*json_string), input_rows_count); + auto & write_buffer = write_helper.getWriteBuffer(); + FormatSettings format_settings = context ? getFormatSettings(context) : FormatSettings{}; + auto serialization = arguments[0].type->getDefaultSerialization(); + for (size_t i = 0; i < input_rows_count; ++i) + { + serialization->serializeTextJSON(*arguments[0].column, i, write_buffer, format_settings); + write_helper.rowWritten(); + } + write_helper.finalize(); + + ColumnsWithTypeAndName args_with_json_string = {ColumnWithTypeAndName(json_string->getPtr(), std::make_shared(), "")}; + return ConvertImplGenericFromString::execute(args_with_json_string, result_type, nullable_source, input_rows_count, context); + }; + } + + /// TODO: support CAST between JSON types with different parameters + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cast to {} can be performed only from String/Map/Object/Tuple. Got: {}", magic_enum::enum_name(to_object->getSchemaFormat()), from_type->getName()); + } + + WrapperType createVariantToVariantWrapper(const DataTypeVariant & from_variant, const DataTypeVariant & to_variant) const + { + /// We support only extension of variant type, so, only new types can be added. + /// For example: Variant(T1, T2) -> Variant(T1, T2, T3) is supported, but Variant(T1, T2) -> Variant(T1, T3) is not supported. + /// We want to extend Variant type for free without rewriting the data, but we sort data types inside Variant during type creation + /// (we do it because we want Variant(T1, T2) to be the same as Variant(T2, T1)), but after extension the order of variant types + /// (and so their discriminators) can be different. For example: Variant(T1, T3) -> Variant(T1, T2, T3). + /// To avoid full rewrite of discriminators column, ColumnVariant supports it's local order of variant columns (and so local + /// discriminators) and stores mapping global order -> local order. + /// So, to extend Variant with new types for free, we should keep old local order for old variants, append new variants and change + /// mapping global order -> local order according to the new global order. + + /// Create map (new variant type) -> (it's global discriminator in new order). + const auto & new_variants = to_variant.getVariants(); + std::unordered_map new_variant_types_to_new_global_discriminator; + new_variant_types_to_new_global_discriminator.reserve(new_variants.size()); + for (size_t i = 0; i != new_variants.size(); ++i) + new_variant_types_to_new_global_discriminator[new_variants[i]->getName()] = i; + + /// Create set of old variant types. + const auto & old_variants = from_variant.getVariants(); + std::unordered_map old_variant_types_to_old_global_discriminator; + old_variant_types_to_old_global_discriminator.reserve(old_variants.size()); + for (size_t i = 0; i != old_variants.size(); ++i) + old_variant_types_to_old_global_discriminator[old_variants[i]->getName()] = i; + + /// Check that the set of old variants types is a subset of new variant types and collect new global discriminator for each old global discriminator. + std::unordered_map old_global_discriminator_to_new; + old_global_discriminator_to_new.reserve(old_variants.size()); + for (const auto & [old_variant_type, old_discriminator] : old_variant_types_to_old_global_discriminator) + { + auto it = new_variant_types_to_new_global_discriminator.find(old_variant_type); + if (it == new_variant_types_to_new_global_discriminator.end()) + throw Exception( + ErrorCodes::CANNOT_CONVERT_TYPE, + "Cannot convert type {} to {}. Conversion between Variant types is allowed only when new Variant type is an extension " + "of an initial one", from_variant.getName(), to_variant.getName()); + old_global_discriminator_to_new[old_discriminator] = it->second; + } + + /// Collect variant types and their global discriminators that should be added to the old Variant to get the new Variant. + std::vector> variant_types_and_discriminators_to_add; + variant_types_and_discriminators_to_add.reserve(new_variants.size() - old_variants.size()); + for (size_t i = 0; i != new_variants.size(); ++i) + { + if (!old_variant_types_to_old_global_discriminator.contains(new_variants[i]->getName())) + variant_types_and_discriminators_to_add.emplace_back(new_variants[i], i); + } + + return [old_global_discriminator_to_new, variant_types_and_discriminators_to_add] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & column_variant = assert_cast(*arguments.front().column.get()); + size_t num_old_variants = column_variant.getNumVariants(); + Columns new_variant_columns; + new_variant_columns.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); + std::vector new_local_to_global_discriminators; + new_local_to_global_discriminators.reserve(num_old_variants + variant_types_and_discriminators_to_add.size()); + for (size_t i = 0; i != num_old_variants; ++i) + { + new_variant_columns.push_back(column_variant.getVariantPtrByLocalDiscriminator(i)); + new_local_to_global_discriminators.push_back(old_global_discriminator_to_new.at(column_variant.globalDiscriminatorByLocal(i))); + } + + for (const auto & [new_variant_type, new_global_discriminator] : variant_types_and_discriminators_to_add) + { + new_variant_columns.push_back(new_variant_type->createColumn()); + new_local_to_global_discriminators.push_back(new_global_discriminator); + } + + return ColumnVariant::create(column_variant.getLocalDiscriminatorsPtr(), column_variant.getOffsetsPtr(), new_variant_columns, new_local_to_global_discriminators); + }; + } + + /// Create wrapper only if we support this conversion. + WrapperType createWrapperIfCanConvert(const DataTypePtr & from, const DataTypePtr & to) const + { + try + { + /// We can avoid try/catch here if we will implement check that 2 types can be cast, but it + /// requires quite a lot of work. By now let's simply use try/catch. + /// First, check that we can create a wrapper. + WrapperType wrapper = prepareUnpackDictionaries(from, to); + /// Second, check if we can perform a conversion on column with default value. + /// (we cannot just check empty column as we do some checks only during iteration over rows). + auto test_col = from->createColumn(); + test_col->insertDefault(); + ColumnsWithTypeAndName column_from = {{test_col->getPtr(), from, "" }}; + wrapper(column_from, to, nullptr, 1); + return wrapper; + } + catch (const Exception &) + { + return {}; + } + } + + WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const + { + const auto & variant_types = from_variant.getVariants(); + std::vector variant_wrappers; + variant_wrappers.reserve(variant_types.size()); + + /// Create conversion wrapper for each variant. + for (const auto & variant_type : variant_types) + { + WrapperType wrapper; + if (cast_type == CastType::accurateOrNull) + { + /// Create wrapper only if we support conversion from variant to the resulting type. + wrapper = createWrapperIfCanConvert(variant_type, to_type); + } + else + { + wrapper = prepareUnpackDictionaries(variant_type, to_type); + } + variant_wrappers.push_back(wrapper); + } + + return [variant_wrappers, variant_types, to_type] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + const auto & column_variant = assert_cast(*arguments.front().column.get()); + + /// First, cast each variant to the result type. + std::vector cast_variant_columns; + cast_variant_columns.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i); + ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }}; + const auto & variant_wrapper = variant_wrappers[i]; + ColumnPtr cast_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + cast_variant_columns.push_back(std::move(cast_variant)); + } + + /// Second, construct resulting column from cast variant columns according to discriminators. + const auto & local_discriminators = column_variant.getLocalDiscriminators(); + auto res = result_type->createColumn(); + res->reserve(input_rows_count); + for (size_t i = 0; i != input_rows_count; ++i) + { + auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !cast_variant_columns[global_discr]) + res->insertDefault(); + else + res->insertFrom(*cast_variant_columns[global_discr], column_variant.offsetAt(i)); + } + + return res; + }; + } + + static ColumnPtr createVariantFromDescriptorsAndOneNonEmptyVariant(const DataTypes & variant_types, const ColumnPtr & discriminators, const ColumnPtr & variant, ColumnVariant::Discriminator variant_discr) + { + Columns variants; + variants.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + if (i == variant_discr) + variants.emplace_back(variant); + else + variants.push_back(variant_types[i]->createColumn()); + } + + return ColumnVariant::create(discriminators, variants); + } + + WrapperType createStringToVariantWrapper() const + { + return [&](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + auto column = arguments[0].column->convertToFullColumnIfLowCardinality(); + auto args = arguments; + args[0].column = column; + + const ColumnNullable * column_nullable = nullptr; + if (isColumnNullable(*args[0].column)) + { + column_nullable = assert_cast(args[0].column.get()); + args[0].column = column_nullable->getNestedColumnPtr(); + } + + args[0].type = removeNullable(removeLowCardinality(args[0].type)); + + if (cast_type == CastType::accurateOrNull) + return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); + return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); + }; + } + + WrapperType createColumnToVariantWrapper(const DataTypePtr & from_type, const DataTypeVariant & to_variant) const + { + /// We allow converting NULL to Variant(...) as Variant can store NULLs. + if (from_type->onlyNull()) + { + return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + auto result_column = result_type->createColumn(); + result_column->insertManyDefaults(input_rows_count); + return result_column; + }; + } + + auto variant_discr_opt = to_variant.tryGetVariantDiscriminator(removeNullableOrLowCardinalityNullable(from_type)->getName()); + /// Cast String to Variant through parsing if it's not Variant(String). + if (isStringOrFixedString(removeNullable(removeLowCardinality(from_type))) && (!variant_discr_opt || to_variant.getVariants().size() > 1)) + return createStringToVariantWrapper(); + + if (!variant_discr_opt) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert type {} to {}. Conversion to Variant allowed only for types from this Variant", from_type->getName(), to_variant.getName()); + + return [variant_discr = *variant_discr_opt] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & result_variant_type = assert_cast(*result_type); + const auto & variant_types = result_variant_type.getVariants(); + if (const ColumnNullable * col_nullable = typeid_cast(arguments.front().column.get())) + { + const auto & column = col_nullable->getNestedColumnPtr(); + const auto & null_map = col_nullable->getNullMapData(); + IColumn::Filter filter; + filter.reserve(column->size()); + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + auto & discriminators_data = discriminators->getData(); + discriminators_data.reserve(column->size()); + size_t variant_size_hint = 0; + for (size_t i = 0; i != column->size(); ++i) + { + if (null_map[i]) + { + discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); + filter.push_back(0); + } + else + { + discriminators_data.push_back(variant_discr); + filter.push_back(1); + ++variant_size_hint; + } + } + + ColumnPtr variant_column; + /// If there were no NULLs, just use the column. + if (variant_size_hint == column->size()) + variant_column = column; + /// Otherwise we should use filtered column. + else + variant_column = column->filter(filter, variant_size_hint); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), variant_column, variant_discr); + } + else if (isColumnLowCardinalityNullable(*arguments.front().column)) + { + const auto & column = arguments.front().column; + + /// Variant column cannot have LowCardinality(Nullable(...)) variant, as Variant column stores NULLs itself. + /// We should create a null-map, insert NULL_DISCRIMINATOR on NULL values and filter initial column. + const auto & col_lc = assert_cast(*column); + const auto & indexes = col_lc.getIndexes(); + auto null_index = col_lc.getDictionary().getNullValueIndex(); + IColumn::Filter filter; + filter.reserve(col_lc.size()); + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + auto & discriminators_data = discriminators->getData(); + discriminators_data.reserve(col_lc.size()); + size_t variant_size_hint = 0; + for (size_t i = 0; i != col_lc.size(); ++i) + { + if (indexes.getUInt(i) == null_index) + { + discriminators_data.push_back(ColumnVariant::NULL_DISCRIMINATOR); + filter.push_back(0); + } + else + { + discriminators_data.push_back(variant_discr); + filter.push_back(1); + ++variant_size_hint; + } + } + + MutableColumnPtr variant_column; + /// If there were no NULLs, we can just clone the column. + if (variant_size_hint == col_lc.size()) + variant_column = IColumn::mutate(column); + /// Otherwise we should filter column. + else + variant_column = IColumn::mutate(column->filter(filter, variant_size_hint)); + + assert_cast(*variant_column).nestedRemoveNullable(); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), std::move(variant_column), variant_discr); + } + else + { + const auto & column = arguments.front().column; + auto discriminators = ColumnVariant::ColumnDiscriminators::create(); + discriminators->getData().resize_fill(column->size(), variant_discr); + return createVariantFromDescriptorsAndOneNonEmptyVariant(variant_types, std::move(discriminators), column, variant_discr); + } + }; + } + + /// Wrapper for conversion to/from Variant type + WrapperType createVariantWrapper(const DataTypePtr & from_type, const DataTypePtr & to_type) const + { + if (const auto * from_variant = checkAndGetDataType(from_type.get())) + { + if (const auto * to_variant = checkAndGetDataType(to_type.get())) + return createVariantToVariantWrapper(*from_variant, *to_variant); + + return createVariantToColumnWrapper(*from_variant, to_type); + } + + return createColumnToVariantWrapper(from_type, assert_cast(*to_type)); + } + + WrapperType createDynamicToColumnWrapper(const DataTypePtr &) const + { + return [this] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + /// When casting Dynamic to regular column we should cast all variants from current Dynamic column + /// and construct the result based on discriminators. + const auto & column_dynamic = assert_cast(*arguments.front().column.get()); + const auto & variant_column = column_dynamic.getVariantColumn(); + const auto & variant_info = column_dynamic.getVariantInfo(); + + /// First, cast usual variants to result type. + const auto & variant_types = assert_cast(*variant_info.variant_type).getVariants(); + std::vector cast_variant_columns; + cast_variant_columns.reserve(variant_types.size()); + for (size_t i = 0; i != variant_types.size(); ++i) + { + /// Skip shared variant, it will be processed later. + if (i == column_dynamic.getSharedVariantDiscriminator()) + { + cast_variant_columns.push_back(nullptr); + continue; + } + + const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i); + ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}}; + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type); + + ColumnPtr cast_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size()); + cast_variant_columns.push_back(cast_variant); + } + + /// Second, collect all variants stored in shared variant and cast them to result type. + std::vector variant_columns_from_shared_variant; + DataTypes variant_types_from_shared_variant; + /// We will need to know what variant to use when we see discriminator of a shared variant. + /// To do it, we remember what variant was extracted from each row and what was it's offset. + PaddedPODArray shared_variant_indexes; + PaddedPODArray shared_variant_offsets; + std::unordered_map shared_variant_to_index; + const auto & shared_variant = column_dynamic.getSharedVariant(); + const auto shared_variant_discr = column_dynamic.getSharedVariantDiscriminator(); + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + if (!shared_variant.empty()) + { + shared_variant_indexes.reserve(input_rows_count); + shared_variant_offsets.reserve(input_rows_count); + FormatSettings format_settings; + const auto shared_variant_local_discr = variant_column.localDiscriminatorByGlobal(shared_variant_discr); + for (size_t i = 0; i != input_rows_count; ++i) + { + if (local_discriminators[i] == shared_variant_local_discr) + { + auto value = shared_variant.getDataAt(offsets[i]); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + auto type_name = type->getName(); + auto it = shared_variant_to_index.find(type_name); + /// Check if we didn't create column for this variant yet. + if (it == shared_variant_to_index.end()) + { + it = shared_variant_to_index.emplace(type_name, variant_columns_from_shared_variant.size()).first; + variant_columns_from_shared_variant.push_back(type->createColumn()); + variant_types_from_shared_variant.push_back(type); + } + + shared_variant_indexes.push_back(it->second); + shared_variant_offsets.push_back(variant_columns_from_shared_variant[it->second]->size()); + type->getDefaultSerialization()->deserializeBinary(*variant_columns_from_shared_variant[it->second], buf, format_settings); + } + else + { + shared_variant_indexes.emplace_back(); + shared_variant_offsets.emplace_back(); + } + } + } + + /// Cast all extracted variants into result type. + std::vector cast_shared_variant_columns; + cast_shared_variant_columns.reserve(variant_types_from_shared_variant.size()); + for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i) + { + ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}}; + WrapperType variant_wrapper; + if (cast_type == CastType::accurateOrNull) + /// Create wrapper only if we support conversion from variant to the resulting type. + variant_wrapper = createWrapperIfCanConvert(variant_types_from_shared_variant[i], result_type); + else + variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type); + + ColumnPtr cast_variant; + /// Check if we have wrapper for this variant. + if (variant_wrapper) + cast_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size()); + cast_shared_variant_columns.push_back(cast_variant); + } + + /// Construct result column from all cast variants. + auto res = result_type->createColumn(); + res->reserve(input_rows_count); + for (size_t i = 0; i != input_rows_count; ++i) + { + auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { + res->insertDefault(); + } + else if (global_discr == shared_variant_discr) + { + if (cast_shared_variant_columns[shared_variant_indexes[i]]) + res->insertFrom(*cast_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]); + else + res->insertDefault(); + } + else + { + if (cast_variant_columns[global_discr]) + res->insertFrom(*cast_variant_columns[global_discr], offsets[i]); + else + res->insertDefault(); + } + } + + return res; + }; + } + + WrapperType createStringToDynamicThroughParsingWrapper() const + { + return [&](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + auto column = arguments[0].column->convertToFullColumnIfLowCardinality(); + auto args = arguments; + args[0].column = column; + + const ColumnNullable * column_nullable = nullptr; + if (isColumnNullable(*args[0].column)) + { + column_nullable = assert_cast(args[0].column.get()); + args[0].column = column_nullable->getNestedColumnPtr(); + } + + args[0].type = removeNullable(removeLowCardinality(args[0].type)); + + if (cast_type == CastType::accurateOrNull) + return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); + return ConvertImplGenericFromString::execute(args, result_type, column_nullable, input_rows_count, context); + }; + } + + WrapperType createVariantToDynamicWrapper(const DataTypeVariant & from_variant_type, const DataTypeDynamic & dynamic_type) const + { + /// First create extended Variant with shared variant type and cast this Variant to it. + auto variants_for_dynamic = from_variant_type.getVariants(); + size_t number_of_variants = variants_for_dynamic.size(); + variants_for_dynamic.push_back(ColumnDynamic::getSharedVariantDataType()); + const auto & variant_type_for_dynamic = std::make_shared(variants_for_dynamic); + auto old_to_new_variant_wrapper = createVariantToVariantWrapper(from_variant_type, *variant_type_for_dynamic); + auto max_dynamic_types = dynamic_type.getMaxDynamicTypes(); + return [old_to_new_variant_wrapper, variant_type_for_dynamic, number_of_variants, max_dynamic_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + { + auto variant_column_for_dynamic = old_to_new_variant_wrapper(arguments, result_type, col_nullable, input_rows_count); + /// If resulting Dynamic column can contain all variants from this Variant column, just create Dynamic column from it. + if (max_dynamic_types >= number_of_variants) + return ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, max_dynamic_types, max_dynamic_types); + + /// Otherwise some variants should go to the shared variant. Create temporary Dynamic column from this Variant and insert + /// all data to the resulting Dynamic column, this insertion will do all the logic with shared variant. + auto tmp_dynamic_column = ColumnDynamic::create(variant_column_for_dynamic, variant_type_for_dynamic, number_of_variants, number_of_variants); + auto result_dynamic_column = ColumnDynamic::create(max_dynamic_types); + result_dynamic_column->insertRangeFrom(*tmp_dynamic_column, 0, tmp_dynamic_column->size()); + return result_dynamic_column; + }; + } + + WrapperType createColumnToDynamicWrapper(const DataTypePtr & from_type, const DataTypeDynamic & dynamic_type) const + { + if (const auto * variant_type = typeid_cast(from_type.get())) + return createVariantToDynamicWrapper(*variant_type, dynamic_type); + + if (context && context->getSettingsRef()[Setting::cast_string_to_dynamic_use_inference] && isStringOrFixedString(removeNullable(removeLowCardinality(from_type)))) + return createStringToDynamicThroughParsingWrapper(); + + /// First, cast column to Variant with 2 variants - the type of the column we cast and shared variant type. + auto variant_type = std::make_shared(DataTypes{removeNullableOrLowCardinalityNullable(from_type)}); + auto column_to_variant_wrapper = createColumnToVariantWrapper(from_type, *variant_type); + /// Second, cast this Variant to Dynamic. + auto variant_to_dynamic_wrapper = createVariantToDynamicWrapper(*variant_type, dynamic_type); + return [column_to_variant_wrapper, variant_to_dynamic_wrapper, variant_type] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * col_nullable, size_t input_rows_count) -> ColumnPtr + { + auto variant_res = column_to_variant_wrapper(arguments, variant_type, col_nullable, input_rows_count); + ColumnsWithTypeAndName args = {{variant_res, variant_type, ""}}; + return variant_to_dynamic_wrapper(args, result_type, nullptr, input_rows_count); + }; + } + + WrapperType createDynamicToDynamicWrapper(const DataTypeDynamic & from_dynamic, const DataTypeDynamic & to_dynamic) const + { + size_t from_max_types = from_dynamic.getMaxDynamicTypes(); + size_t to_max_types = to_dynamic.getMaxDynamicTypes(); + if (from_max_types == to_max_types) + return createIdentityWrapper(from_dynamic.getPtr()); + + if (to_max_types > from_max_types) + { + return [to_max_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & dynamic_column = assert_cast(*arguments[0].column); + /// We should use the same limit as already used in column and change only global limit. + /// It's needed because shared variant should contain values only when limit is exceeded, + /// so if there are already some data, we cannot increase the limit. + return ColumnDynamic::create(dynamic_column.getVariantColumnPtr(), dynamic_column.getVariantInfo(), dynamic_column.getMaxDynamicTypes(), to_max_types); + }; + } + + return [to_max_types] + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t) -> ColumnPtr + { + const auto & dynamic_column = assert_cast(*arguments[0].column); + /// If real limit in the column is not greater than desired, just use the same variant column. + if (dynamic_column.getMaxDynamicTypes() <= to_max_types) + return ColumnDynamic::create(dynamic_column.getVariantColumnPtr(), dynamic_column.getVariantInfo(), dynamic_column.getMaxDynamicTypes(), to_max_types); + + /// Otherwise some variants should go to the shared variant. We try to keep the most frequent variants. + const auto & variant_info = dynamic_column.getVariantInfo(); + const auto & variants = assert_cast(*variant_info.variant_type).getVariants(); + const auto & statistics = dynamic_column.getStatistics(); + const auto & variant_column = dynamic_column.getVariantColumn(); + auto shared_variant_discr = dynamic_column.getSharedVariantDiscriminator(); + std::vector> variants_with_sizes; + variants_with_sizes.reserve(variant_info.variant_names.size()); + for (const auto & [name, discr] : variant_info.variant_name_to_discriminator) + { + /// Don't include shared variant. + if (discr == shared_variant_discr) + continue; + + size_t size = variant_column.getVariantByGlobalDiscriminator(discr).size(); + /// If column has statistics from the data part, use size from it for consistency. + /// It's important to keep the same dynamic structure of the result column during ALTER. + if (statistics) + { + auto statistics_it = statistics->variants_statistics.find(name); + if (statistics_it != statistics->variants_statistics.end()) + size = statistics_it->second; + } + variants_with_sizes.emplace_back(size, name, variants[discr]); + } + + std::sort(variants_with_sizes.begin(), variants_with_sizes.end(), std::greater()); + DataTypes result_variants; + result_variants.reserve(to_max_types + 1); /// +1 for shared variant. + /// Add new variants from sorted list until we reach to_max_types. + for (const auto & [size, name, type] : variants_with_sizes) + { + if (result_variants.size() < to_max_types) + result_variants.push_back(type); + else + break; + } + + /// Add shared variant. + result_variants.push_back(ColumnDynamic::getSharedVariantDataType()); + /// Create resulting Variant type and Dynamic column. + auto result_variant_type = std::make_shared(result_variants); + auto result_dynamic_column = ColumnDynamic::create(result_variant_type->createColumn(), result_variant_type, to_max_types, to_max_types); + const auto & result_variant_info = result_dynamic_column->getVariantInfo(); + auto & result_variant_column = result_dynamic_column->getVariantColumn(); + auto result_shared_variant_discr = result_dynamic_column->getSharedVariantDiscriminator(); + /// Create mapping from old discriminators to the new ones. + std::vector old_to_new_discriminators; + old_to_new_discriminators.resize(variant_info.variant_name_to_discriminator.size(), result_shared_variant_discr); + for (const auto & [name, discr] : result_variant_info.variant_name_to_discriminator) + { + auto old_discr = variant_info.variant_name_to_discriminator.at(name); + old_to_new_discriminators[old_discr] = discr; + /// Reuse old variant column if it's not shared variant. + if (discr != result_shared_variant_discr) + result_variant_column.getVariantPtrByGlobalDiscriminator(discr) = variant_column.getVariantPtrByGlobalDiscriminator(old_discr); + } + + const auto & local_discriminators = variant_column.getLocalDiscriminators(); + const auto & offsets = variant_column.getOffsets(); + const auto & shared_variant = dynamic_column.getSharedVariant(); + auto & result_local_discriminators = result_variant_column.getLocalDiscriminators(); + result_local_discriminators.reserve(local_discriminators.size()); + auto & result_offsets = result_variant_column.getOffsets(); + result_offsets.reserve(offsets.size()); + auto & result_shared_variant = result_dynamic_column->getSharedVariant(); + for (size_t i = 0; i != local_discriminators.size(); ++i) + { + auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]); + if (global_discr == ColumnVariant::NULL_DISCRIMINATOR) + { + result_local_discriminators.push_back(ColumnVariant::NULL_DISCRIMINATOR); + result_offsets.emplace_back(); + } + else if (global_discr == shared_variant_discr) + { + result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_shared_variant_discr)); + result_offsets.push_back(result_shared_variant.size()); + result_shared_variant.insertFrom(shared_variant, offsets[i]); + } + else + { + auto result_global_discr = old_to_new_discriminators[global_discr]; + if (result_global_discr == result_shared_variant_discr) + { + result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_shared_variant_discr)); + result_offsets.push_back(result_shared_variant.size()); + ColumnDynamic::serializeValueIntoSharedVariant( + result_shared_variant, + variant_column.getVariantByGlobalDiscriminator(global_discr), + variants[global_discr], + variants[global_discr]->getDefaultSerialization(), + offsets[i]); + } + else + { + result_local_discriminators.push_back(result_variant_column.localDiscriminatorByGlobal(result_global_discr)); + result_offsets.push_back(offsets[i]); + } + } + } + + return result_dynamic_column; + }; + } + + /// Wrapper for conversion to/from Dynamic type + WrapperType createDynamicWrapper(const DataTypePtr & from_type, const DataTypePtr & to_type) const + { + if (const auto * from_dynamic = checkAndGetDataType(from_type.get())) + { + if (const auto * to_dynamic = checkAndGetDataType(to_type.get())) + return createDynamicToDynamicWrapper(*from_dynamic, *to_dynamic); + + return createDynamicToColumnWrapper(to_type); + } + + return createColumnToDynamicWrapper(from_type, *checkAndGetDataType(to_type.get())); + } + + template + WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum * to_type) const + { + using EnumType = DataTypeEnum; + using Function = typename FunctionTo::Type; + + if (const auto * from_enum8 = checkAndGetDataType(from_type.get())) + checkEnumToEnumConversion(from_enum8, to_type); + else if (const auto * from_enum16 = checkAndGetDataType(from_type.get())) + checkEnumToEnumConversion(from_enum16, to_type); + + if (checkAndGetDataType(from_type.get())) + return createStringToEnumWrapper(); + else if (checkAndGetDataType(from_type.get())) + return createStringToEnumWrapper(); + else if (isNativeNumber(from_type) || isEnum(from_type)) + { + auto function = Function::create(context); + return createFunctionAdaptor(function, from_type); + } + else + { + if (cast_type == CastType::accurateOrNull) + return createToNullableColumnWrapper(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", + from_type->getName(), to_type->getName()); + } + } + + template + void checkEnumToEnumConversion(const EnumTypeFrom * from_type, const EnumTypeTo * to_type) const + { + const auto & from_values = from_type->getValues(); + const auto & to_values = to_type->getValues(); + + using ValueType = std::common_type_t; + using NameValuePair = std::pair; + using EnumValues = std::vector; + + EnumValues name_intersection; + std::set_intersection(std::begin(from_values), std::end(from_values), + std::begin(to_values), std::end(to_values), std::back_inserter(name_intersection), + [] (auto && from, auto && to) { return from.first < to.first; }); + + for (const auto & name_value : name_intersection) + { + const auto & old_value = name_value.second; + const auto & new_value = to_type->getValue(name_value.first); + if (old_value != new_value) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Enum conversion changes value for element '{}' from {} to {}", + name_value.first, toString(old_value), toString(new_value)); + } + } + + template + WrapperType createStringToEnumWrapper() const + { + const char * function_name = cast_name; + return [function_name] ( + ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) + { + const auto & first_col = arguments.front().column.get(); + const auto & result_type = typeid_cast(*res_type); + + const ColumnStringType * col = typeid_cast(first_col); + + if (col && nullable_col && nullable_col->size() != col->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnNullable is not compatible with original"); + + if (col) + { + const auto size = col->size(); + + auto res = result_type.createColumn(); + auto & out_data = static_cast(*res).getData(); + out_data.resize(size); + + auto default_enum_value = result_type.getValues().front().second; + + if (nullable_col) + { + for (size_t i = 0; i < size; ++i) + { + if (!nullable_col->isNullAt(i)) + out_data[i] = result_type.getValue(col->getDataAt(i)); + else + out_data[i] = default_enum_value; + } + } + else + { + for (size_t i = 0; i < size; ++i) + out_data[i] = result_type.getValue(col->getDataAt(i)); + } + + return res; + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column {} as first argument of function {}", + first_col->getName(), function_name); + }; + } + + template + WrapperType createEnumToStringWrapper() const + { + const char * function_name = cast_name; + return [function_name] ( + ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, const ColumnNullable * nullable_col, size_t /*input_rows_count*/) + { + using ColumnEnumType = typename EnumType::ColumnType; + + const auto & first_col = arguments.front().column.get(); + const auto & first_type = arguments.front().type.get(); + + const ColumnEnumType * enum_col = typeid_cast(first_col); + const EnumType * enum_type = typeid_cast(first_type); + + if (enum_col && nullable_col && nullable_col->size() != enum_col->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnNullable is not compatible with original"); + + if (enum_col && enum_type) + { + const auto size = enum_col->size(); + const auto & enum_data = enum_col->getData(); + + auto res = res_type->createColumn(); + + if (nullable_col) + { + for (size_t i = 0; i < size; ++i) + { + if (!nullable_col->isNullAt(i)) + { + const auto & value = enum_type->getNameForValue(enum_data[i]); + res->insertData(value.data, value.size); + } + else + res->insertDefault(); + } + } + else + { + for (size_t i = 0; i < size; ++i) + { + const auto & value = enum_type->getNameForValue(enum_data[i]); + res->insertData(value.data, value.size); + } + } + + return res; + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column {} as first argument of function {}", + first_col->getName(), function_name); + }; + } + + static WrapperType createIdentityWrapper(const DataTypePtr &) + { + return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable *, size_t /*input_rows_count*/) + { + return arguments.front().column; + }; + } + + static WrapperType createNothingWrapper(const IDataType * to_type) + { + ColumnPtr res = to_type->createColumnConstWithDefaultValue(1); + return [res] (ColumnsWithTypeAndName &, const DataTypePtr &, const ColumnNullable *, size_t input_rows_count) + { + /// Column of Nothing type is trivially convertible to any other column + return res->cloneResized(input_rows_count)->convertToFullColumnIfConst(); + }; + } + + WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const + { + /// Conversion from/to Variant/Dynamic data type is processed in a special way. + /// We don't need to remove LowCardinality/Nullable. + if (isDynamic(to_type) || isDynamic(from_type)) + return createDynamicWrapper(from_type, to_type); + + if (isVariant(to_type) || isVariant(from_type)) + return createVariantWrapper(from_type, to_type); + + const auto * from_low_cardinality = typeid_cast(from_type.get()); + const auto * to_low_cardinality = typeid_cast(to_type.get()); + const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type; + const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type; + + if (from_type->onlyNull()) + { + if (!to_nested->isNullable() && !isVariant(to_type)) + { + if (cast_type == CastType::accurateOrNull) + { + return createToNullableColumnWrapper(); + } + else + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert NULL to a non-nullable type"); + } + } + + return [](ColumnsWithTypeAndName &, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) + { + return result_type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst(); + }; + } + + bool skip_not_null_check = false; + + if (from_low_cardinality && from_nested->isNullable() && !to_nested->isNullable()) + /// Disable check for dictionary. Will check that column doesn't contain NULL in wrapper below. + skip_not_null_check = true; + + auto wrapper = prepareRemoveNullable(from_nested, to_nested, skip_not_null_check); + if (!from_low_cardinality && !to_low_cardinality) + return wrapper; + + return [wrapper, from_low_cardinality, to_low_cardinality, skip_not_null_check] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) -> ColumnPtr + { + ColumnsWithTypeAndName args = {arguments[0]}; + auto & arg = args.front(); + auto res_type = result_type; + + ColumnPtr converted_column; + + ColumnPtr res_indexes; + /// For some types default can't be cast (for example, String to Int). In that case convert column to full. + bool src_converted_to_full_column = false; + + { + auto tmp_rows_count = input_rows_count; + + if (to_low_cardinality) + res_type = to_low_cardinality->getDictionaryType(); + + if (from_low_cardinality) + { + const auto & col_low_cardinality = typeid_cast(*arguments[0].column); + + if (skip_not_null_check && col_low_cardinality.containsNull()) + throw Exception(ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, "Cannot convert NULL value to non-Nullable type"); + + arg.column = col_low_cardinality.getDictionary().getNestedColumn(); + arg.type = from_low_cardinality->getDictionaryType(); + + /// TODO: Make map with defaults conversion. + src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res_type)); + if (src_converted_to_full_column) + arg.column = arg.column->index(col_low_cardinality.getIndexes(), 0); + else + res_indexes = col_low_cardinality.getIndexesPtr(); + + tmp_rows_count = arg.column->size(); + } + + /// Perform the requested conversion. + converted_column = wrapper(args, res_type, nullable_source, tmp_rows_count); + } + + if (to_low_cardinality) + { + auto res_column = to_low_cardinality->createColumn(); + auto & col_low_cardinality = typeid_cast(*res_column); + + if (from_low_cardinality && !src_converted_to_full_column) + col_low_cardinality.insertRangeFromDictionaryEncodedColumn(*converted_column, *res_indexes); + else + col_low_cardinality.insertRangeFromFullColumn(*converted_column, 0, converted_column->size()); + + return res_column; + } + else if (!src_converted_to_full_column) + return converted_column->index(*res_indexes, 0); + else + return converted_column; + }; + } + + WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type, bool skip_not_null_check) const + { + /// Determine whether pre-processing and/or post-processing must take place during conversion. + + bool source_is_nullable = from_type->isNullable(); + bool result_is_nullable = to_type->isNullable(); + + auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable); + + if (result_is_nullable) + { + return [wrapper, source_is_nullable] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + /// Create a temporary columns on which to perform the operation. + const auto & nullable_type = static_cast(*result_type); + const auto & nested_type = nullable_type.getNestedType(); + + ColumnsWithTypeAndName tmp_args; + if (source_is_nullable) + tmp_args = createBlockWithNestedColumns(arguments); + else + tmp_args = arguments; + + const ColumnNullable * nullable_source = nullptr; + + /// Add original ColumnNullable for createStringToEnumWrapper() + if (source_is_nullable) + { + if (arguments.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of arguments"); + nullable_source = typeid_cast(arguments.front().column.get()); + } + + /// Perform the requested conversion. + auto tmp_res = wrapper(tmp_args, nested_type, nullable_source, input_rows_count); + + /// May happen in fuzzy tests. For debug purpose. + if (!tmp_res) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Couldn't convert {} to {} in prepareRemoveNullable wrapper.", + arguments[0].type->getName(), nested_type->getName()); + + return wrapInNullable(tmp_res, arguments, nested_type, input_rows_count); + }; + } + else if (source_is_nullable) + { + /// Conversion from Nullable to non-Nullable. + + return [wrapper, skip_not_null_check] + (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + auto tmp_args = createBlockWithNestedColumns(arguments); + auto nested_type = removeNullable(result_type); + + /// Check that all values are not-NULL. + /// Check can be skipped in case if LowCardinality dictionary is transformed. + /// In that case, correctness will be checked beforehand. + if (!skip_not_null_check) + { + const auto & col = arguments[0].column; + const auto & nullable_col = assert_cast(*col); + const auto & null_map = nullable_col.getNullMapData(); + + if (!memoryIsZero(null_map.data(), 0, null_map.size())) + throw Exception(ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, "Cannot convert NULL value to non-Nullable type"); + } + const ColumnNullable * nullable_source = typeid_cast(arguments.front().column.get()); + return wrapper(tmp_args, nested_type, nullable_source, input_rows_count); + }; + } + else + return wrapper; + } + + /// 'from_type' and 'to_type' are nested types in case of Nullable. + /// 'requested_result_is_nullable' is true if CAST to Nullable type is requested. + WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const + { + if (isUInt8(from_type) && isBool(to_type)) + return createUInt8ToBoolWrapper(from_type, to_type); + + /// We can cast IPv6 into IPv6, IPv4 into IPv4, but we should not allow to cast FixedString(16) into IPv6 as part of identity cast + bool safe_convert_custom_types = true; + + if (const auto * to_type_custom_name = to_type->getCustomName()) + safe_convert_custom_types = from_type->getCustomName() && from_type->getCustomName()->getName() == to_type_custom_name->getName(); + else if (const auto * from_type_custom_name = from_type->getCustomName()) + safe_convert_custom_types = to_type->getCustomName() && from_type_custom_name->getName() == to_type->getCustomName()->getName(); + + if (from_type->equals(*to_type) && safe_convert_custom_types) + { + /// We can only use identity conversion for DataTypeAggregateFunction when they are strictly equivalent. + if (typeid_cast(from_type.get())) + { + if (DataTypeAggregateFunction::strictEquals(from_type, to_type)) + return createIdentityWrapper(from_type); + } + else + return createIdentityWrapper(from_type); + } + else if (WhichDataType(from_type).isNothing()) + return createNothingWrapper(to_type.get()); + + WrapperType ret; + + auto make_default_wrapper = [&](const auto & types) -> bool + { + using Types = std::decay_t; + using ToDataType = typename Types::LeftType; + + if constexpr (is_any_of) + { + ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + return true; + } + if constexpr (std::is_same_v) + { + if (isBool(to_type)) + ret = createBoolWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + else + ret = createWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + return true; + } + if constexpr ( + std::is_same_v || + std::is_same_v) + { + ret = createEnumWrapper(from_type, checkAndGetDataType(to_type.get())); + return true; + } + if constexpr (is_any_of, DataTypeDecimal, + DataTypeDecimal, DataTypeDecimal, + DataTypeDateTime64>) + { + ret = createDecimalWrapper(from_type, checkAndGetDataType(to_type.get()), requested_result_is_nullable); + return true; + } + + return false; + }; + + bool cast_ipv4_ipv6_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::cast_ipv4_ipv6_default_on_conversion_error]; + bool input_format_ipv4_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::input_format_ipv4_default_on_conversion_error]; + bool input_format_ipv6_default_on_conversion_error_value = context && context->getSettingsRef()[Setting::input_format_ipv6_default_on_conversion_error]; + + auto make_custom_serialization_wrapper = [&, cast_ipv4_ipv6_default_on_conversion_error_value, input_format_ipv4_default_on_conversion_error_value, input_format_ipv6_default_on_conversion_error_value](const auto & types) -> bool + { + using Types = std::decay_t; + using ToDataType = typename Types::RightType; + using FromDataType = typename Types::LeftType; + + if constexpr (WhichDataType(FromDataType::type_id).isStringOrFixedString()) + { + if constexpr (std::is_same_v) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, + input_format_ipv4_default_on_conversion_error_value, + requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t) -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv4()) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (requested_result_is_nullable) + return convertToIPv4(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv4_default_on_conversion_error_value) + return convertToIPv4(arguments[0].column, null_map); + else + return convertToIPv4(arguments[0].column, null_map); + }; + + return true; + } + + if constexpr (std::is_same_v) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, + input_format_ipv6_default_on_conversion_error_value, + requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t) -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv6()) + throw Exception( + ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv6", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (requested_result_is_nullable) + return convertToIPv6(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value || input_format_ipv6_default_on_conversion_error_value) + return convertToIPv6(arguments[0].column, null_map); + else + return convertToIPv6(arguments[0].column, null_map); + }; + + return true; + } + + if (to_type->getCustomSerialization() && to_type->getCustomName()) + { + ret = [requested_result_is_nullable, this]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & result_type, + const ColumnNullable * column_nullable, + size_t input_rows_count) -> ColumnPtr + { + auto wrapped_result_type = result_type; + if (requested_result_is_nullable) + wrapped_result_type = makeNullable(result_type); + if (this->cast_type == CastType::accurateOrNull) + return ConvertImplGenericFromString::execute( + arguments, wrapped_result_type, column_nullable, input_rows_count, context); + return ConvertImplGenericFromString::execute( + arguments, wrapped_result_type, column_nullable, input_rows_count, context); + }; + return true; + } + } + else if constexpr (WhichDataType(FromDataType::type_id).isIPv6() && WhichDataType(ToDataType::type_id).isIPv4()) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) + -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv4()) + throw Exception( + ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (requested_result_is_nullable) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else if (cast_ipv4_ipv6_default_on_conversion_error_value) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else + return convertIPv6ToIPv4(arguments[0].column, null_map); + }; + + return true; + } + + if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) + { + if constexpr (WhichDataType(FromDataType::type_id).isEnum()) + { + ret = createEnumToStringWrapper(); + return true; + } + else if (from_type->getCustomSerialization()) + { + ret = [this](ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr + { + return ConvertImplGenericToString::execute(arguments, result_type, input_rows_count, context); + }; + return true; + } + } + + return false; + }; + + if (callOnTwoTypeIndexes(from_type->getTypeId(), to_type->getTypeId(), make_custom_serialization_wrapper)) + return ret; + + if (callOnIndexAndDataType(to_type->getTypeId(), make_default_wrapper)) + return ret; + + switch (to_type->getTypeId()) + { + case TypeIndex::String: + return createStringWrapper(from_type); + case TypeIndex::FixedString: + return createFixedStringWrapper(from_type, checkAndGetDataType(to_type.get())->getN()); + case TypeIndex::Array: + return createArrayWrapper(from_type, static_cast(*to_type)); + case TypeIndex::Tuple: + return createTupleWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::Map: + return createMapWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::ObjectDeprecated: + return createObjectDeprecatedWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::Object: + return createObjectWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::AggregateFunction: + return createAggregateFunctionWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::Interval: + return createIntervalWrapper(from_type, checkAndGetDataType(to_type.get())->getKind()); + default: + break; + } + + if (cast_type == CastType::accurateOrNull) + return createToNullableColumnWrapper(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", + from_type->getName(), to_type->getName()); + } +}; + +} + + +FunctionBasePtr createFunctionBaseCast( + ContextPtr context, + const char * name, + const ColumnsWithTypeAndName & arguments, + const DataTypePtr & return_type, + std::optional diagnostic, + CastType cast_type) +{ + DataTypes data_types(arguments.size()); + + for (size_t i = 0; i < arguments.size(); ++i) + data_types[i] = arguments[i].type; + + FunctionsConversion::FunctionCast::MonotonicityForRange monotonicity; + + if (isEnum(arguments.front().type) + && castTypeToEither(return_type.get(), [&](auto & type) + { + monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + return true; + })) + { + } + else if (castTypeToEither< + DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, + DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, + DataTypeFloat32, DataTypeFloat64, + DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, + DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) + { + monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + return true; + })) + { + } + + return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); +} + +} diff --git a/src/Functions/FunctionsConversion_impl0.cpp b/src/Functions/FunctionsConversion_impl0.cpp new file mode 100644 index 00000000000..5a07f39d7b5 --- /dev/null +++ b/src/Functions/FunctionsConversion_impl0.cpp @@ -0,0 +1,47 @@ +#include + +namespace DB +{ + +namespace FunctionsConversion +{ + +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; + +template class FunctionConvert; + +template class FunctionConvert; + +template class FunctionConvert; + +template class FunctionConvert; + +template class FunctionConvert; + +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert>; +template class FunctionConvert; +template class FunctionConvert>; +template class FunctionConvert, NameToDecimal32, UnknownMonotonicity>; +template class FunctionConvert, NameToDecimal64, UnknownMonotonicity>; +template class FunctionConvert, NameToDecimal128, UnknownMonotonicity>; +template class FunctionConvert, NameToDecimal256, UnknownMonotonicity>; + +} + +} diff --git a/src/Functions/FunctionsConversion_impl1.cpp b/src/Functions/FunctionsConversion_impl1.cpp new file mode 100644 index 00000000000..d2e0f4888b6 --- /dev/null +++ b/src/Functions/FunctionsConversion_impl1.cpp @@ -0,0 +1,38 @@ +#include + +namespace DB +{ + +namespace FunctionsConversion +{ + +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>; +template class FunctionConvertFromString, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>; +template class FunctionConvertFromString, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>; +template class FunctionConvertFromString, NameToDecimal256OrZero, ConvertFromStringExceptionMode::Zero>; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; + +} + +} diff --git a/src/Functions/FunctionsConversion_impl2.cpp b/src/Functions/FunctionsConversion_impl2.cpp new file mode 100644 index 00000000000..6069208e089 --- /dev/null +++ b/src/Functions/FunctionsConversion_impl2.cpp @@ -0,0 +1,38 @@ +#include + +namespace DB +{ + +namespace FunctionsConversion +{ + +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>; +template class FunctionConvertFromString, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>; +template class FunctionConvertFromString, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>; +template class FunctionConvertFromString, NameToDecimal256OrNull, ConvertFromStringExceptionMode::Null>; +template class FunctionConvertFromString; +template class FunctionConvertFromString; +template class FunctionConvertFromString; + +} + +} diff --git a/src/Functions/FunctionsConversion_impl3.cpp b/src/Functions/FunctionsConversion_impl3.cpp new file mode 100644 index 00000000000..e63a0760ddd --- /dev/null +++ b/src/Functions/FunctionsConversion_impl3.cpp @@ -0,0 +1,46 @@ +#include + +namespace DB +{ + +namespace FunctionsConversion +{ + +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTimeBestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime, NameParseDateTime32BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>; +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>; + +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>; +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>; +template class FunctionConvertFromString< + DataTypeDateTime64, NameParseDateTime64BestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>; + +} + +} diff --git a/src/Functions/FunctionsConversion_reg.cpp b/src/Functions/FunctionsConversion_reg.cpp new file mode 100644 index 00000000000..2054f12be1b --- /dev/null +++ b/src/Functions/FunctionsConversion_reg.cpp @@ -0,0 +1,192 @@ +#include + +namespace DB +{ + +REGISTER_FUNCTION(Conversion) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Converts Float32 to BFloat16 with losing the precision. + +Example: +[example:typical] +)", + .examples{ + {"typical", "SELECT toBFloat16(12.3::Float32);", "12.3125"}}, + .categories{"Conversion"}}); + + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + + /// MySQL compatibility alias. Cannot be registered as alias, + /// because we don't want it to be normalized to toDate in queries, + /// otherwise CREATE DICTIONARY query breaks. + factory.registerFunction("DATE", &FunctionsConversion::FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Converts String to BFloat16. + +If the string does not represent a floating point value, the function returns zero. + +The function allows a silent loss of precision while converting from the string representation. In that case, it will return the truncated result. + +Example of successful conversion: +[example:typical] + +Examples of not successful conversion: +[example:invalid1] +[example:invalid2] + +Example of a loss of precision: +[example:precision] +)", + .examples{ + {"typical", "SELECT toBFloat16OrZero('12.3');", "12.3125"}, + {"invalid1", "SELECT toBFloat16OrZero('abc');", "0"}, + {"invalid2", "SELECT toBFloat16OrZero(' 1');", "0"}, + {"precision", "SELECT toBFloat16OrZero('12.3456789');", "12.375"}}, + .categories{"Conversion"}}); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(FunctionDocumentation{.description=R"( +Converts String to Nullable(BFloat16). + +If the string does not represent a floating point value, the function returns NULL. + +The function allows a silent loss of precision while converting from the string representation. In that case, it will return the truncated result. + +Example of successful conversion: +[example:typical] + +Examples of not successful conversion: +[example:invalid1] +[example:invalid2] + +Example of a loss of precision: +[example:precision] +)", + .examples{ + {"typical", "SELECT toBFloat16OrNull('12.3');", "12.3125"}, + {"invalid1", "SELECT toBFloat16OrNull('abc');", "NULL"}, + {"invalid2", "SELECT toBFloat16OrNull(' 1');", "NULL"}, + {"precision", "SELECT toBFloat16OrNull('12.3456789');", "12.375"}}, + .categories{"Conversion"}}); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} From 7f4318d5da1cbe54442a5a69ae4e162cf5e2ca93 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 05:56:59 +0000 Subject: [PATCH 231/433] fix duplicate symbol --- src/Functions/FunctionsConversion.cpp | 76 +++++++++++++++++++++++++++ src/Functions/FunctionsConversion.h | 59 ++------------------- 2 files changed, 79 insertions(+), 56 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index e69de29bb2d..8a7c966b232 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -0,0 +1,76 @@ +#include + +namespace DB +{ + +namespace FunctionsConversion +{ + +UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) +{ + const auto * arg_type = named_column.type.get(); + bool ok = checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type) + || checkAndGetDataType(arg_type); + if (!ok) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of toDecimal() scale {}", named_column.type->getName()); + + Field field; + named_column.column->get(0, field); + return static_cast(field.safeGet()); +} + +ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col) +{ + ColumnUInt8::MutablePtr null_map = nullptr; + if (const auto * col_nullable = checkAndGetColumn(col.get())) + { + null_map = ColumnUInt8::create(); + null_map->insertRangeFrom(col_nullable->getNullMapColumn(), 0, col_nullable->size()); + } + return null_map; +} + +} + +FunctionBasePtr createFunctionBaseCast( + ContextPtr context, + const char * name, + const ColumnsWithTypeAndName & arguments, + const DataTypePtr & return_type, + std::optional diagnostic, + CastType cast_type) +{ + DataTypes data_types(arguments.size()); + + for (size_t i = 0; i < arguments.size(); ++i) + data_types[i] = arguments[i].type; + + FunctionsConversion::FunctionCast::MonotonicityForRange monotonicity; + + if (isEnum(arguments.front().type) + && castTypeToEither(return_type.get(), [&](auto & type) + { + monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + return true; + })) + { + } + else if (castTypeToEither< + DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, + DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, + DataTypeFloat32, DataTypeFloat64, + DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, + DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) + { + monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + return true; + })) + { + } + + return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); +} + +} diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 171c2f2a8ff..1a685a5e134 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -121,20 +121,7 @@ namespace FunctionsConversion * toType - conversion in "natural way"; */ -UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) -{ - const auto * arg_type = named_column.type.get(); - bool ok = checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type) - || checkAndGetDataType(arg_type); - if (!ok) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type of toDecimal() scale {}", named_column.type->getName()); - - Field field; - named_column.column->get(0, field); - return static_cast(field.safeGet()); -} +UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column); /** Conversion of Date to DateTime: adding 00:00:00 time component. @@ -577,16 +564,7 @@ struct FormatImpl> } }; -ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col) -{ - ColumnUInt8::MutablePtr null_map = nullptr; - if (const auto * col_nullable = checkAndGetColumn(col.get())) - { - null_map = ColumnUInt8::create(); - null_map->insertRangeFrom(col_nullable->getNullMapColumn(), 0, col_nullable->size()); - } - return null_map; -} +ColumnUInt8::MutablePtr copyNullMap(ColumnPtr col); /// Generic conversion of any type to String or FixedString via serialization to text. @@ -5531,37 +5509,6 @@ FunctionBasePtr createFunctionBaseCast( const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, std::optional diagnostic, - CastType cast_type) -{ - DataTypes data_types(arguments.size()); - - for (size_t i = 0; i < arguments.size(); ++i) - data_types[i] = arguments[i].type; - - FunctionsConversion::FunctionCast::MonotonicityForRange monotonicity; - - if (isEnum(arguments.front().type) - && castTypeToEither(return_type.get(), [&](auto & type) - { - monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; - return true; - })) - { - } - else if (castTypeToEither< - DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, - DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, - DataTypeFloat32, DataTypeFloat64, - DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, - DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) - { - monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; - return true; - })) - { - } - - return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); -} + CastType cast_type); } From 871e5b32ddc30c04330ee5d591b4f6b82e19e66a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 06:08:20 +0000 Subject: [PATCH 232/433] fix ErrorCodes --- src/Functions/FunctionsConversion.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 8a7c966b232..226f91a62ff 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -3,6 +3,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + namespace FunctionsConversion { From 002f1ee9a7fb7c2769312aee48cc59576e9f0c21 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 07:09:01 +0000 Subject: [PATCH 233/433] fix make --- src/Functions/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 3a57ce42412..3f852334908 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -18,6 +18,10 @@ set(DBMS_FUNCTIONS checkHyperscanRegexp.cpp # checkHyperscanRegexp (Used by MultiMatchAnyImpl.h, multiMatchAny.cpp) CastOverloadResolver.cpp # createInternalCast (Used by WindowTransform.cpp, KeyCondition.cpp...) FunctionsConversion.cpp # createFunctionBaseCast (Used by CastOverloadResolver.cpp) + FunctionsConversion_impl0.cpp + FunctionsConversion_impl1.cpp + FunctionsConversion_impl2.cpp + FunctionsConversion_impl3.cpp extractTimeZoneFromFunctionArguments.cpp # extractTimeZoneFromFunctionArguments (DateTimeTransforms.h, FunctionsConversion.cpp) ) extract_into_parent_list(clickhouse_functions_sources dbms_sources ${DBMS_FUNCTIONS}) From 140f5987da4dbbce3ad082f5a84a501ef3d3e346 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sun, 24 Nov 2024 09:46:38 +0100 Subject: [PATCH 234/433] Perf test for LowCardinality sorting at INSERT --- tests/performance/low_cardinality_insert.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 tests/performance/low_cardinality_insert.xml diff --git a/tests/performance/low_cardinality_insert.xml b/tests/performance/low_cardinality_insert.xml new file mode 100644 index 00000000000..77e62e9f12c --- /dev/null +++ b/tests/performance/low_cardinality_insert.xml @@ -0,0 +1,11 @@ + + DROP TABLE IF EXISTS test_lc_insert + + CREATE TABLE test_lc_insert (k1 LowCardinality(String), k2 LowCardinality(String)) ENGINE = MergeTree order by (k1, k2) + + + + INSERT INTO test_lc_insert(k1, k2) SELECT number%1000 AS k1, number%1001 AS k2 FROM numbers(2000000) + + DROP TABLE IF EXISTS test_lc_insert + From 523945fade269d335c9fa9b1f35e44a91948b023 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 24 Nov 2024 11:58:24 +0100 Subject: [PATCH 235/433] Check arguments --- src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index 3045bc8c930..bf07e8fc93b 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -284,8 +284,11 @@ IAggregateFunction * tryResultTypes(const DataTypes & argument_types, const Type } template -AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string &, const DataTypes & argument_types, const Array &, const Settings *) +AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string & name, const DataTypes & argument_types, const Array &, const Settings *) { + + assertBinary(name, argument_types); + using TypesToCreateSpecializedDataTuple = TypesToCreateSpecializedData; const DataTypePtr & result_type = argument_types[0]; From d478a1dc0e6c4a01c97c4356db5ce9853ef523f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 24 Nov 2024 12:16:47 +0100 Subject: [PATCH 236/433] Small check --- src/Storages/SelectQueryDescription.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index b21711965e0..89189a44c39 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -79,8 +79,11 @@ void checkAllowedQueries(const ASTSelectWithUnionQuery & select) { auto * query = children->as(); + if (query == nullptr) + throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW support query with multiple simple UNION [ALL] only"); + if (query->prewhere() || query->final() || query->sampleSize()) - throw Exception(DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL."); + throw Exception(ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW, "MATERIALIZED VIEW cannot have PREWHERE, SAMPLE or FINAL."); ASTPtr subquery = extractTableExpression(*query, 0); if (!subquery) From 7a3a286f9b5aa80182c5795347d04e2bee7c8cd7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 24 Nov 2024 13:04:46 +0100 Subject: [PATCH 237/433] bump From 530cdbbf13f54b75ee2830ce328f58e1982517d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 24 Nov 2024 14:04:14 +0100 Subject: [PATCH 238/433] Fxi --- src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index bf07e8fc93b..5148e401cd6 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -300,7 +300,7 @@ AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string & name, auto convert_date_type = [] (TypeIndex type_index) { if (type_index == TypeIndex::Date) - return TypeToTypeIndex; + return TypeToTypeIndex; else if (type_index == TypeIndex::DateTime) return TypeToTypeIndex; else From bf102d2a00df6012f5c3cfa55c581a45409ae862 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Sun, 24 Nov 2024 14:23:49 +0000 Subject: [PATCH 239/433] Remove flaky test and extend stable one --- src/Common/FailPoint.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 5 ++ .../test_s3_zero_copy_replication/test.py | 69 ++++--------------- 3 files changed, 22 insertions(+), 54 deletions(-) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 027cc347386..85e8129b668 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -79,6 +79,8 @@ static struct InitFiu REGULAR(zero_copy_lock_zk_fail_after_op) \ REGULAR(plain_object_storage_write_fail_on_directory_create) \ REGULAR(plain_object_storage_write_fail_on_directory_move) \ + REGULAR(zero_copy_unlock_zk_fail_before_op) \ + REGULAR(zero_copy_unlock_zk_fail_after_op) \ namespace FailPoints diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 264644ffd28..d753ee2a175 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -218,6 +218,8 @@ namespace FailPoints extern const char finish_set_quorum_failed_parts[]; extern const char zero_copy_lock_zk_fail_before_op[]; extern const char zero_copy_lock_zk_fail_after_op[]; + extern const char zero_copy_unlock_zk_fail_before_op[]; + extern const char zero_copy_unlock_zk_fail_after_op[]; } namespace ErrorCodes @@ -9852,6 +9854,9 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( LOG_TRACE(logger, "Removing zookeeper lock {} for part {} (files to keep: [{}])", zookeeper_part_replica_node, part_name, fmt::join(files_not_to_remove, ", ")); + fiu_do_on(FailPoints::zero_copy_unlock_zk_fail_before_op, { zookeeper_ptr->forceFailureBeforeOperation(); }); + fiu_do_on(FailPoints::zero_copy_unlock_zk_fail_after_op, { zookeeper_ptr->forceFailureAfterOperation(); }); + if (auto ec = zookeeper_ptr->tryRemove(zookeeper_part_replica_node); ec != Coordination::Error::ZOK) { /// Very complex case. It means that lock already doesn't exist when we tried to remove it. diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index c7d03d4301d..e40a0c7beed 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -693,9 +693,16 @@ def test_s3_zero_copy_keeps_data_after_mutation(started_cluster): @pytest.mark.parametrize( - "failpoint", ["zero_copy_lock_zk_fail_before_op", "zero_copy_lock_zk_fail_after_op"] + "failpoint_lock", + ["zero_copy_lock_zk_fail_before_op", "zero_copy_lock_zk_fail_after_op"], ) -def test_move_shared_lock_fail_once(started_cluster, test_table, failpoint): +@pytest.mark.parametrize( + "failpoint_unlock", + [None, "zero_copy_unlock_zk_fail_before_op", "zero_copy_unlock_zk_fail_after_op"], +) +def test_move_shared_zero_copy_lock_fail( + started_cluster, test_table, failpoint_lock, failpoint_unlock +): node1 = cluster.instances["node1"] node2 = cluster.instances["node2"] @@ -714,7 +721,9 @@ def test_move_shared_lock_fail_once(started_cluster, test_table, failpoint): node1.query(f"INSERT INTO {test_table} VALUES (1, '{date}')") # Try to move and get fail on acquring zero-copy shared lock - node1.query(f"SYSTEM ENABLE FAILPOINT {failpoint}") + node1.query(f"SYSTEM ENABLE FAILPOINT {failpoint_lock}") + if failpoint_unlock: + node1.query(f"SYSTEM ENABLE FAILPOINT {failpoint_unlock}") node1.query_and_get_error( f"ALTER TABLE {test_table} MOVE PARTITION '{date}' TO VOLUME 'external'" ) @@ -729,7 +738,9 @@ def test_move_shared_lock_fail_once(started_cluster, test_table, failpoint): # Try another attempt after zk connection is restored # It should not failed due to leftovers of previous attempt (temporary cloned files) - node1.query(f"SYSTEM DISABLE FAILPOINT {failpoint}") + node1.query(f"SYSTEM DISABLE FAILPOINT {failpoint_lock}") + if failpoint_unlock: + node1.query(f"SYSTEM DISABLE FAILPOINT {failpoint_unlock}") node1.query( f"ALTER TABLE {test_table} MOVE PARTITION '{date}' TO VOLUME 'external'" ) @@ -748,53 +759,3 @@ def test_move_shared_lock_fail_once(started_cluster, test_table, failpoint): node1.query(f"DROP TABLE IF EXISTS {test_table} SYNC") node2.query(f"DROP TABLE IF EXISTS {test_table} SYNC") - - -def test_move_shared_lock_fail_keeper_unavailable(started_cluster, test_table): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - - node1.query( - f""" - CREATE TABLE {test_table} ON CLUSTER test_cluster (num UInt64, date DateTime) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{test_table}', '{{replica}}') - ORDER BY date PARTITION BY date - SETTINGS storage_policy='hybrid' - """ - ) - - date = "2024-10-23" - node2.query(f"SYSTEM STOP FETCHES {test_table}") - - node1.query(f"INSERT INTO {test_table} VALUES (1, '{date}')") - # Pause moving after part cloning, but before swapping - node1.query("SYSTEM ENABLE FAILPOINT stop_moving_part_before_swap_with_active") - - def move(node): - node.query_and_get_error( - f"ALTER TABLE {test_table} MOVE PARTITION '{date}' TO VOLUME 'external'" - ) - - # Start moving - t1 = threading.Thread(target=move, args=[node1]) - t1.start() - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - # Continue moving and try to swap - node1.query("SYSTEM DISABLE FAILPOINT stop_moving_part_before_swap_with_active") - t1.join() - - # Previous MOVE was failed, try another one after zk connection is restored - # It should not failed due to leftovers of previous attempt (temporary cloned files) - node1.query_with_retry( - f"ALTER TABLE {test_table} MOVE PARTITION '{date}' TO VOLUME 'external'" - ) - - # Sanity check - node2.query(f"SYSTEM START FETCHES {test_table}") - wait_for_active_parts(node2, 1, test_table, disk_name="s31") - assert node2.query(f"SELECT sum(num) FROM {test_table}") == "1\n" - - node1.query(f"DROP TABLE IF EXISTS {test_table} SYNC") - node2.query(f"DROP TABLE IF EXISTS {test_table} SYNC") From c6353c43e546333555fde132f9cc0ac818adb763 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 15:16:11 +0000 Subject: [PATCH 240/433] rename implementation details namespace to detail --- src/Functions/FunctionsConversion.cpp | 10 +- src/Functions/FunctionsConversion.h | 2 +- src/Functions/FunctionsConversion_impl0.cpp | 2 +- src/Functions/FunctionsConversion_impl1.cpp | 2 +- src/Functions/FunctionsConversion_impl2.cpp | 2 +- src/Functions/FunctionsConversion_impl3.cpp | 2 +- src/Functions/FunctionsConversion_reg.cpp | 216 ++++++++++---------- 7 files changed, 118 insertions(+), 118 deletions(-) diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 226f91a62ff..0513be5ea44 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -8,7 +8,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -namespace FunctionsConversion +namespace detail { UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column) @@ -52,12 +52,12 @@ FunctionBasePtr createFunctionBaseCast( for (size_t i = 0; i < arguments.size(); ++i) data_types[i] = arguments[i].type; - FunctionsConversion::FunctionCast::MonotonicityForRange monotonicity; + detail::FunctionCast::MonotonicityForRange monotonicity; if (isEnum(arguments.front().type) && castTypeToEither(return_type.get(), [&](auto & type) { - monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + monotonicity = detail::FunctionTo>::Type::Monotonic::get; return true; })) { @@ -69,13 +69,13 @@ FunctionBasePtr createFunctionBaseCast( DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, DataTypeString>(recursiveRemoveLowCardinality(return_type).get(), [&](auto & type) { - monotonicity = FunctionsConversion::FunctionTo>::Type::Monotonic::get; + monotonicity = detail::FunctionTo>::Type::Monotonic::get; return true; })) { } - return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); + return std::make_unique(context, name, std::move(monotonicity), data_types, return_type, diagnostic, cast_type); } } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 1a685a5e134..09640bc1d2b 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -114,7 +114,7 @@ namespace ErrorCodes extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } -namespace FunctionsConversion +namespace detail { /** Type conversion functions. diff --git a/src/Functions/FunctionsConversion_impl0.cpp b/src/Functions/FunctionsConversion_impl0.cpp index 5a07f39d7b5..fa20e00e1cf 100644 --- a/src/Functions/FunctionsConversion_impl0.cpp +++ b/src/Functions/FunctionsConversion_impl0.cpp @@ -3,7 +3,7 @@ namespace DB { -namespace FunctionsConversion +namespace detail { template class FunctionConvert>; diff --git a/src/Functions/FunctionsConversion_impl1.cpp b/src/Functions/FunctionsConversion_impl1.cpp index d2e0f4888b6..724af04d1fd 100644 --- a/src/Functions/FunctionsConversion_impl1.cpp +++ b/src/Functions/FunctionsConversion_impl1.cpp @@ -3,7 +3,7 @@ namespace DB { -namespace FunctionsConversion +namespace detail { template class FunctionConvertFromString; diff --git a/src/Functions/FunctionsConversion_impl2.cpp b/src/Functions/FunctionsConversion_impl2.cpp index 6069208e089..145627f37ea 100644 --- a/src/Functions/FunctionsConversion_impl2.cpp +++ b/src/Functions/FunctionsConversion_impl2.cpp @@ -3,7 +3,7 @@ namespace DB { -namespace FunctionsConversion +namespace detail { template class FunctionConvertFromString; diff --git a/src/Functions/FunctionsConversion_impl3.cpp b/src/Functions/FunctionsConversion_impl3.cpp index e63a0760ddd..300a8c934ce 100644 --- a/src/Functions/FunctionsConversion_impl3.cpp +++ b/src/Functions/FunctionsConversion_impl3.cpp @@ -3,7 +3,7 @@ namespace DB { -namespace FunctionsConversion +namespace detail { template class FunctionConvertFromString< diff --git a/src/Functions/FunctionsConversion_reg.cpp b/src/Functions/FunctionsConversion_reg.cpp index 2054f12be1b..e4abfaaa2a5 100644 --- a/src/Functions/FunctionsConversion_reg.cpp +++ b/src/Functions/FunctionsConversion_reg.cpp @@ -5,20 +5,20 @@ namespace DB REGISTER_FUNCTION(Conversion) { - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(FunctionDocumentation{.description=R"( + factory.registerFunction(FunctionDocumentation{.description=R"( Converts Float32 to BFloat16 with losing the precision. Example: @@ -28,46 +28,46 @@ Example: {"typical", "SELECT toBFloat16(12.3::Float32);", "12.3125"}}, .categories{"Conversion"}}); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionsConversion::FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); + factory.registerFunction("DATE", &detail::FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(FunctionDocumentation{.description=R"( + factory.registerFunction(FunctionDocumentation{.description=R"( Converts String to BFloat16. If the string does not represent a floating point value, the function returns zero. @@ -91,36 +91,36 @@ Example of a loss of precision: {"precision", "SELECT toBFloat16OrZero('12.3456789');", "12.375"}}, .categories{"Conversion"}}); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(FunctionDocumentation{.description=R"( + factory.registerFunction(FunctionDocumentation{.description=R"( Converts String to Nullable(BFloat16). If the string does not represent a floating point value, the function returns NULL. @@ -144,49 +144,49 @@ Example of a loss of precision: {"precision", "SELECT toBFloat16OrNull('12.3456789');", "12.375"}}, .categories{"Conversion"}}); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); } } From a8899eb26d102a3a55e27dc636e19a855cb5f672 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 24 Nov 2024 15:20:12 +0000 Subject: [PATCH 241/433] Docs: update minimum required Clang version --- docs/en/development/build.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index d9fcdec671a..e130e57aeb2 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -44,7 +44,7 @@ sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test For other Linux distributions - check the availability of LLVM's [prebuild packages](https://releases.llvm.org/download.html). -As of March 2024, clang-17 or higher will work. +As of November 2024, clang-18 or higher will work. GCC as a compiler is not supported. To build with a specific Clang version: From f9c07bc6fc9c5f7dd91f24a7a943514961837e7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 24 Nov 2024 20:50:44 +0100 Subject: [PATCH 242/433] Bump From 30a40f89afb56b47794a29253512640550543715 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Nov 2024 12:35:54 +0100 Subject: [PATCH 243/433] Revert "Revert "CI: Stress test with praktika"" --- ci/docker/fasttest/Dockerfile | 9 + ci/docker/stateless-test/Dockerfile | 2 + ci/docker/stateless-test/requirements.txt | 1 + ci/jobs/__init__.py | 0 ci/jobs/build_clickhouse.py | 60 +++- ci/jobs/functional_stateful_tests.py | 5 +- ci/jobs/functional_stateless_tests.py | 4 + ci/jobs/scripts/clickhouse_proc.py | 18 ++ ci/jobs/scripts/clickhouse_version.py | 38 +++ .../functional_tests/setup_ch_cluster.sh | 118 ++++++++ .../functional_tests/setup_log_cluster.sh | 261 ++++++++++++++++++ ci/praktika/_environment.py | 2 +- ci/praktika/artifact.py | 9 + ci/praktika/hook_html.py | 7 +- ci/praktika/json.html | 2 +- ci/praktika/native_jobs.py | 8 +- ci/praktika/result.py | 3 + ci/praktika/runner.py | 37 ++- ci/praktika/s3.py | 13 +- ci/{ => settings}/__init__.py | 0 ci/settings/definitions.py | 1 + ci/workflows/pull_request.py | 114 ++++++-- packages/build | 50 ++-- tests/ci/report.py | 42 +++ tests/ci/stress_check.py | 16 +- tests/config/install.sh | 2 +- 26 files changed, 743 insertions(+), 79 deletions(-) delete mode 100644 ci/jobs/__init__.py create mode 100644 ci/jobs/scripts/clickhouse_version.py create mode 100755 ci/jobs/scripts/functional_tests/setup_ch_cluster.sh create mode 100755 ci/jobs/scripts/functional_tests/setup_log_cluster.sh rename ci/{ => settings}/__init__.py (100%) diff --git a/ci/docker/fasttest/Dockerfile b/ci/docker/fasttest/Dockerfile index 41aeb38619c..f9b536929c2 100644 --- a/ci/docker/fasttest/Dockerfile +++ b/ci/docker/fasttest/Dockerfile @@ -102,5 +102,14 @@ RUN groupadd --system --gid 1000 clickhouse \ && useradd --system --gid 1000 --uid 1000 -m clickhouse \ && mkdir -p /.cache/sccache && chmod 777 /.cache/sccache + +# TODO move nfpm to docker that will do packaging +ARG TARGETARCH +ARG NFPM_VERSION=2.20.0 +RUN arch=${TARGETARCH:-amd64} \ + && curl -Lo /tmp/nfpm.deb "https://github.com/goreleaser/nfpm/releases/download/v${NFPM_VERSION}/nfpm_${arch}.deb" \ + && dpkg -i /tmp/nfpm.deb \ + && rm /tmp/nfpm.deb + ENV PYTHONPATH="/wd" ENV PYTHONUNBUFFERED=1 diff --git a/ci/docker/stateless-test/Dockerfile b/ci/docker/stateless-test/Dockerfile index dcfaa5f6267..4bf5d2788cc 100644 --- a/ci/docker/stateless-test/Dockerfile +++ b/ci/docker/stateless-test/Dockerfile @@ -58,6 +58,7 @@ RUN apt-get update -y \ curl \ wget \ xz-utils \ + ripgrep \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* @@ -114,4 +115,5 @@ RUN curl -L --no-verbose -O 'https://archive.apache.org/dist/hadoop/common/hadoo RUN npm install -g azurite@3.30.0 \ && npm install -g tslib && npm install -g node +ENV PYTHONPATH=".:./ci" USER clickhouse diff --git a/ci/docker/stateless-test/requirements.txt b/ci/docker/stateless-test/requirements.txt index 6f64cc08951..64b06640c31 100644 --- a/ci/docker/stateless-test/requirements.txt +++ b/ci/docker/stateless-test/requirements.txt @@ -4,3 +4,4 @@ requests==2.32.3 pandas==1.5.3 scipy==1.12.0 pyarrow==18.0.0 +grpcio==1.47.0 diff --git a/ci/jobs/__init__.py b/ci/jobs/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/ci/jobs/build_clickhouse.py b/ci/jobs/build_clickhouse.py index ed9fd491fcf..5ab721fb81e 100644 --- a/ci/jobs/build_clickhouse.py +++ b/ci/jobs/build_clickhouse.py @@ -4,11 +4,15 @@ from praktika.result import Result from praktika.settings import Settings from praktika.utils import MetaClasses, Shell, Utils +from ci.jobs.scripts.clickhouse_version import CHVersion + class JobStages(metaclass=MetaClasses.WithIter): CHECKOUT_SUBMODULES = "checkout" CMAKE = "cmake" + UNSHALLOW = "unshallow" BUILD = "build" + PACKAGE = "package" def parse_args(): @@ -33,8 +37,7 @@ CMAKE_CMD = """cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA \ -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON \ {AUX_DEFS} \ -DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 \ --DCOMPILER_CACHE={CACHE_TYPE} \ --DENABLE_BUILD_PROFILING=1 {DIR}""" +-DCOMPILER_CACHE={CACHE_TYPE} -DENABLE_BUILD_PROFILING=1 {DIR}""" def main(): @@ -91,6 +94,27 @@ def main(): res = True results = [] + version = "" + + if res and JobStages.UNSHALLOW in stages: + results.append( + Result.create_from_command_execution( + name="Repo Unshallow", + command="git rev-parse --is-shallow-repository | grep -q true && git fetch --depth 10000 --no-tags --filter=tree:0 origin $(git rev-parse --abbrev-ref HEAD)", + with_log=True, + ) + ) + res = results[-1].is_ok() + if res: + try: + version = CHVersion.get_version() + assert version + print(f"Got version from repo [{version}]") + except Exception as e: + results[-1].set_failed().set_info( + f"Failed to get version from repo, ex [{e}]" + ) + res = False if res and JobStages.CHECKOUT_SUBMODULES in stages: Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}") @@ -127,6 +151,38 @@ def main(): Shell.check(f"ls -l {build_dir}/programs/") res = results[-1].is_ok() + if res and JobStages.PACKAGE in stages: + if "debug" in build_type: + package_type = "debug" + elif "release" in build_type: + package_type = "release" + elif "asan" in build_type: + package_type = "asan" + else: + assert False, "TODO" + + if "amd" in build_type: + deb_arch = "amd64" + else: + deb_arch = "arm64" + + output_dir = "/tmp/praktika/output/" + assert Shell.check(f"rm -f {output_dir}/*.deb") + + results.append( + Result.create_from_command_execution( + name="Build Packages", + command=[ + f"DESTDIR={build_dir}/root ninja programs/install", + f"ln -sf {build_dir}/root {Utils.cwd()}/packages/root", + f"cd {Utils.cwd()}/packages/ && OUTPUT_DIR={output_dir} BUILD_TYPE={package_type} VERSION_STRING={version} DEB_ARCH={deb_arch} ./build --deb", + ], + workdir=build_dir, + with_log=True, + ) + ) + res = results[-1].is_ok() + Result.create_from(results=results, stopwatch=stop_watch).complete_job() diff --git a/ci/jobs/functional_stateful_tests.py b/ci/jobs/functional_stateful_tests.py index b5840fcd45d..4af1ab902ae 100644 --- a/ci/jobs/functional_stateful_tests.py +++ b/ci/jobs/functional_stateful_tests.py @@ -1,5 +1,4 @@ import argparse -import os import time from pathlib import Path @@ -131,6 +130,10 @@ def main(): ) res = res and CH.start() res = res and CH.wait_ready() + # TODO: Use --database-replicated optionally + res = res and Shell.check( + f"./ci/jobs/scripts/functional_tests/setup_ch_cluster.sh" + ) if res: print("ch started") logs_to_attach += [ diff --git a/ci/jobs/functional_stateless_tests.py b/ci/jobs/functional_stateless_tests.py index 0d73312bd9e..676a05fbac1 100644 --- a/ci/jobs/functional_stateless_tests.py +++ b/ci/jobs/functional_stateless_tests.py @@ -101,6 +101,7 @@ def main(): f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-client", f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-compressor", f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-local", + f"ln -sf {ch_path}/clickhouse {ch_path}/clickhouse-disks", f"rm -rf {Settings.TEMP_DIR}/etc/ && mkdir -p {Settings.TEMP_DIR}/etc/clickhouse-client {Settings.TEMP_DIR}/etc/clickhouse-server", f"cp programs/server/config.xml programs/server/users.xml {Settings.TEMP_DIR}/etc/clickhouse-server/", # TODO: find a way to work with Azure secret so it's ok for local tests as well, for now keep azure disabled @@ -114,6 +115,7 @@ def main(): f"for file in /tmp/praktika/etc/clickhouse-server/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>{Settings.TEMP_DIR}/var/log|g; s|>/etc/|>{Settings.TEMP_DIR}/etc/|g' $(readlink -f $file); done", f"for file in /tmp/praktika/etc/clickhouse-server/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|local_disk|{Settings.TEMP_DIR}/local_disk|g' $(readlink -f $file); done", f"clickhouse-server --version", + f"chmod +x /tmp/praktika/input/clickhouse-odbc-bridge", ] results.append( Result.create_from_command_execution( @@ -138,6 +140,7 @@ def main(): res = res and Shell.check( "aws s3 ls s3://test --endpoint-url http://localhost:11111/", verbose=True ) + res = res and CH.log_cluster_config() res = res and CH.start() res = res and CH.wait_ready() if res: @@ -170,6 +173,7 @@ def main(): batch_total=total_batches, test=args.test, ) + CH.log_cluster_stop_replication() results.append(FTResultsProcessor(wd=Settings.OUTPUT_DIR).run()) results[-1].set_timing(stopwatch=stop_watch_) res = results[-1].is_ok() diff --git a/ci/jobs/scripts/clickhouse_proc.py b/ci/jobs/scripts/clickhouse_proc.py index 6108563605f..46efc41ee27 100644 --- a/ci/jobs/scripts/clickhouse_proc.py +++ b/ci/jobs/scripts/clickhouse_proc.py @@ -66,6 +66,24 @@ class ClickHouseProc: print(f"Started setup_minio.sh asynchronously with PID {process.pid}") return True + def log_cluster_config(self): + return Shell.check( + f"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --config-logs-export-cluster /tmp/praktika/etc/clickhouse-server/config.d/system_logs_export.yaml", + verbose=True, + ) + + def log_cluster_setup_replication(self): + return Shell.check( + f"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --setup-logs-replication", + verbose=True, + ) + + def log_cluster_stop_replication(self): + return Shell.check( + f"./ci/jobs/scripts/functional_tests/setup_log_cluster.sh --stop-log-replication", + verbose=True, + ) + def start(self): print("Starting ClickHouse server") Shell.check(f"rm {self.pid_file}") diff --git a/ci/jobs/scripts/clickhouse_version.py b/ci/jobs/scripts/clickhouse_version.py new file mode 100644 index 00000000000..44c2753fb11 --- /dev/null +++ b/ci/jobs/scripts/clickhouse_version.py @@ -0,0 +1,38 @@ +from pathlib import Path + +from praktika.utils import Shell + + +class CHVersion: + FILE_WITH_VERSION_PATH = "./cmake/autogenerated_versions.txt" + + @classmethod + def _get_tweak(cls): + tag = Shell.get_output("git describe --tags --abbrev=0") + assert tag.startswith("v24") + num = Shell.get_output(f"git rev-list --count {tag}..HEAD") + return int(num) + + @classmethod + def get_version(cls): + versions = {} + for line in ( + Path(cls.FILE_WITH_VERSION_PATH).read_text(encoding="utf-8").splitlines() + ): + line = line.strip() + if not line.startswith("SET("): + continue + + name, value = line[4:-1].split(maxsplit=1) + name = name.removeprefix("VERSION_").lower() + try: + value = int(value) + except ValueError: + pass + versions[name] = value + + version_sha = versions["githash"] + tweak = int( + Shell.get_output(f"git rev-list --count {version_sha}..HEAD", verbose=True) + ) + return f"{versions['major']}.{versions['minor']}.{versions['patch']}.{tweak}" diff --git a/ci/jobs/scripts/functional_tests/setup_ch_cluster.sh b/ci/jobs/scripts/functional_tests/setup_ch_cluster.sh new file mode 100755 index 00000000000..0c2d6b11b05 --- /dev/null +++ b/ci/jobs/scripts/functional_tests/setup_ch_cluster.sh @@ -0,0 +1,118 @@ +#!/bin/bash + +set -e -x + +clickhouse-client --query "SHOW DATABASES" +clickhouse-client --query "CREATE DATABASE datasets" +clickhouse-client < ./tests/docker_scripts/create.sql +clickhouse-client --query "SHOW TABLES FROM datasets" + +USE_DATABASE_REPLICATED=0 + +while [[ "$#" -gt 0 ]]; do + case $1 in + --database-replicated) + echo "Setup cluster for testing with Database Replicated" + USE_DATABASE_REPLICATED=1 + ;; + *) + echo "Unknown option: $1" + exit 1 + ;; + esac + shift +done + +if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + clickhouse-client --query "CREATE DATABASE test ON CLUSTER 'test_cluster_database_replicated' + ENGINE=Replicated('/test/clickhouse/db/test', '{shard}', '{replica}')" + + clickhouse-client --query "CREATE TABLE test.hits AS datasets.hits_v1" + clickhouse-client --query "CREATE TABLE test.visits AS datasets.visits_v1" + + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" + + clickhouse-client --query "DROP TABLE datasets.hits_v1" + clickhouse-client --query "DROP TABLE datasets.visits_v1" +else + clickhouse-client --query "CREATE DATABASE test" + clickhouse-client --query "SHOW TABLES FROM test" + if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then + clickhouse-client --query "CREATE TABLE test.hits (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, + EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, + UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, + RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), + URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, + FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, + UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, + MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, + SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, + ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, + SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, + FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, + HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, + GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, + HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, + HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, + FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, + LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, + RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, + ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, + OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, + UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, + URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, + ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), + IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) + ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" + clickhouse-client --query "CREATE TABLE test.visits (CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, + VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, + Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, + EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, + AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), + RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, + SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, + ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, + SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, + UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, + FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, + FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, + Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, + BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), + Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), + WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, + ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, + ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, + ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, + ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, + ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, + OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, + UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, + PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, + PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), + CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, + StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, + OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, + UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, + ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), + Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, + DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) + ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) + SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" + + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + clickhouse-client --max_memory_usage 10G --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1 SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" + clickhouse-client --query "DROP TABLE datasets.visits_v1 SYNC" + clickhouse-client --query "DROP TABLE datasets.hits_v1 SYNC" + else + clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" + clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" + fi + clickhouse-client --query "CREATE TABLE test.hits_s3 (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192, storage_policy='s3_cache'" + # AWS S3 is very inefficient, so increase memory even further: + clickhouse-client --max_memory_usage 30G --max_memory_usage_for_user 30G --query "INSERT INTO test.hits_s3 SELECT * FROM test.hits SETTINGS enable_filesystem_cache_on_write_operations=0, max_insert_threads=16" +fi + +clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "SELECT count() FROM test.hits" +clickhouse-client --query "SELECT count() FROM test.visits" diff --git a/ci/jobs/scripts/functional_tests/setup_log_cluster.sh b/ci/jobs/scripts/functional_tests/setup_log_cluster.sh new file mode 100755 index 00000000000..fc6946b7025 --- /dev/null +++ b/ci/jobs/scripts/functional_tests/setup_log_cluster.sh @@ -0,0 +1,261 @@ +#!/bin/bash + +set -e +# This script sets up export of system log tables to a remote server. +# Remote tables are created if not exist, and augmented with extra columns, +# and their names will contain a hash of the table structure, +# which allows exporting tables from servers of different versions. + +# Config file contains KEY=VALUE pairs with any necessary parameters like: +# CLICKHOUSE_CI_LOGS_HOST - remote host +# CLICKHOUSE_CI_LOGS_USER - password for user +# CLICKHOUSE_CI_LOGS_PASSWORD - password for user +CLICKHOUSE_CI_LOGS_CREDENTIALS=${CLICKHOUSE_CI_LOGS_CREDENTIALS:-/tmp/export-logs-config.sh} +CLICKHOUSE_CI_LOGS_USER=${CLICKHOUSE_CI_LOGS_USER:-ci} + +# Pre-configured destination cluster, where to export the data +CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} + +EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "} +EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"} +EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name"} + +# trace_log needs more columns for symbolization +EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), " +EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), trace)::Array(LowCardinality(String)) AS symbols, arrayMap(x -> addressToLine(x), trace)::Array(LowCardinality(String)) AS lines" + +# coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate) +EXTRA_COLUMNS_COVERAGE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), " +EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayDistinct(arrayMap(x -> demangle(addressToSymbol(x)), coverage))::Array(LowCardinality(String)) AS symbols" + + +function __set_connection_args +{ + # It's impossible to use a generic $CONNECTION_ARGS string, it's unsafe from word splitting perspective. + # That's why we must stick to the generated option + CONNECTION_ARGS=( + --receive_timeout=45 --send_timeout=45 --secure + --user "${CLICKHOUSE_CI_LOGS_USER}" --host "${CLICKHOUSE_CI_LOGS_HOST}" + --password "${CLICKHOUSE_CI_LOGS_PASSWORD}" + ) +} + +function __shadow_credentials +{ + # The function completely screws the output, it shouldn't be used in normal functions, only in () + # The only way to substitute the env as a plain text is using perl 's/\Qsomething\E/another/ + exec &> >(perl -pe ' + s(\Q$ENV{CLICKHOUSE_CI_LOGS_HOST}\E)[CLICKHOUSE_CI_LOGS_HOST]g; + s(\Q$ENV{CLICKHOUSE_CI_LOGS_USER}\E)[CLICKHOUSE_CI_LOGS_USER]g; + s(\Q$ENV{CLICKHOUSE_CI_LOGS_PASSWORD}\E)[CLICKHOUSE_CI_LOGS_PASSWORD]g; + ') +} + +function check_logs_credentials +( + # The function connects with given credentials, and if it's unable to execute the simplest query, returns exit code + + # First check, if all necessary parameters are set + set +x + for parameter in CLICKHOUSE_CI_LOGS_HOST CLICKHOUSE_CI_LOGS_USER CLICKHOUSE_CI_LOGS_PASSWORD; do + export -p | grep -q "$parameter" || { + echo "Credentials parameter $parameter is unset" + return 1 + } + done + + __shadow_credentials + __set_connection_args + local code + # Catch both success and error to not fail on `set -e` + clickhouse-client "${CONNECTION_ARGS[@]}" -q 'SELECT 1 FORMAT Null' && return 0 || code=$? + if [ "$code" != 0 ]; then + echo 'Failed to connect to CI Logs cluster' + return $code + fi +) + +function config_logs_export_cluster +( + # The function is launched in a separate shell instance to not expose the + # exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS + set +x + if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then + echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup" + return + fi + set -a + # shellcheck disable=SC1090 + source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" + set +a + __shadow_credentials + echo "Checking if the credentials work" + check_logs_credentials || return 0 + cluster_config="${1:-/etc/clickhouse-server/config.d/system_logs_export.yaml}" + mkdir -p "$(dirname "$cluster_config")" + echo "remote_servers: + ${CLICKHOUSE_CI_LOGS_CLUSTER}: + shard: + replica: + secure: 1 + user: '${CLICKHOUSE_CI_LOGS_USER}' + host: '${CLICKHOUSE_CI_LOGS_HOST}' + port: 9440 + password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' +" > "$cluster_config" + echo "Cluster ${CLICKHOUSE_CI_LOGS_CLUSTER} is confugured in ${cluster_config}" +) + +function setup_logs_replication +( + # The function is launched in a separate shell instance to not expose the + # exported values from CLICKHOUSE_CI_LOGS_CREDENTIALS + set +x + # disable output + if ! [ -r "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" ]; then + echo "File $CLICKHOUSE_CI_LOGS_CREDENTIALS does not exist, do not setup" + return 0 + fi + set -a + # shellcheck disable=SC1090 + source "${CLICKHOUSE_CI_LOGS_CREDENTIALS}" + set +a + __shadow_credentials + echo "Checking if the credentials work" + check_logs_credentials || return 0 + __set_connection_args + + echo "My hostname is ${HOSTNAME}" + + echo 'Create all configured system logs' + clickhouse-client --query "SYSTEM FLUSH LOGS" + + debug_or_sanitizer_build=$(clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'") + echo "Build is debug or sanitizer: $debug_or_sanitizer_build" + + # We will pre-create a table system.coverage_log. + # It is normally created by clickhouse-test rather than the server, + # so we will create it in advance to make it be picked up by the next commands: + + clickhouse-client --query " + CREATE TABLE IF NOT EXISTS system.coverage_log + ( + time DateTime COMMENT 'The time of test run', + test_name String COMMENT 'The name of the test', + coverage Array(UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run' + ) ENGINE = MergeTree ORDER BY test_name COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster' + " + + # For each system log table: + echo 'Create %_log tables' + clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table + do + if [[ "$table" = "trace_log" ]] + then + EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_TRACE_LOG}" + # Do not try to resolve stack traces in case of debug/sanitizers + # build, since it is too slow (flushing of trace_log can take ~1min + # with such MV attached) + if [[ "$debug_or_sanitizer_build" = 1 ]] + then + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" + else + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" + fi + elif [[ "$table" = "coverage_log" ]] + then + EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_COVERAGE_LOG}" + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG}" + else + EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}" + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" + fi + + # Calculate hash of its structure. Note: 4 is the version of extra columns - increment it if extra columns are changed: + hash=$(clickhouse-client --query " + SELECT sipHash64(9, groupArray((name, type))) + FROM (SELECT name, type FROM system.columns + WHERE database = 'system' AND table = '$table' + ORDER BY position) + ") + + # Create the destination table with adapted name and structure: + statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' + s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; + s/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/; + s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; + /^TTL /d + ') + + echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2 + + echo "::group::${table}" + # there's the only way big "$statement" can be printed without causing EAGAIN error + # cat: write error: Resource temporarily unavailable + statement_print="${statement}" + if [ "${#statement_print}" -gt 4000 ]; then + statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}" + fi + echo -e "$statement_print" + echo "::endgroup::" + + echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ + --distributed_ddl_task_timeout=30 --distributed_ddl_output_mode=throw_only_active \ + "${CONNECTION_ARGS[@]}" || continue + + echo "Creating table system.${table}_sender" >&2 + + # Create Distributed table and materialized view to watch on the original table: + clickhouse-client --query " + CREATE TABLE system.${table}_sender + ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) + SETTINGS flush_on_detach=0 + EMPTY AS + SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * + FROM system.${table} + " || continue + + echo "Creating materialized view system.${table}_watcher" >&2 + + clickhouse-client --query " + CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS + SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * + FROM system.${table} + " || continue + done +) + +function stop_logs_replication +{ + echo "Detach all logs replication" + clickhouse-client --query "select database||'.'||table from system.tables where database = 'system' and (table like '%_sender' or table like '%_watcher')" | { + tee /dev/stderr + } | { + timeout --preserve-status --signal TERM --kill-after 5m 15m xargs -n1 -r -i clickhouse-client --query "drop table {}" + } +} + + +while [[ "$#" -gt 0 ]]; do + case $1 in + --stop-log-replication) + echo "Stopping log replication..." + stop_logs_replication + ;; + --setup-logs-replication) + echo "Setting up log replication..." + setup_logs_replication + ;; + --config-logs-export-cluster) + echo "Configuring logs export for the cluster..." + config_logs_export_cluster "$2" + shift + ;; + *) + echo "Unknown option: $1" + echo "Usage: $0 [--stop-log-replication | --setup-logs-replication | --config-logs-export-cluster ]" + exit 1 + ;; + esac + shift +done \ No newline at end of file diff --git a/ci/praktika/_environment.py b/ci/praktika/_environment.py index 734a4be3176..cef305d1b8b 100644 --- a/ci/praktika/_environment.py +++ b/ci/praktika/_environment.py @@ -179,7 +179,7 @@ class _Environment(MetaClasses.Serializable): if bucket in path: path = path.replace(bucket, endpoint) break - REPORT_URL = f"https://{path}/{Path(settings.HTML_PAGE_FILE).name}?PR={self.PR_NUMBER}&sha={'latest' if latest else self.SHA}&name_0={urllib.parse.quote(self.WORKFLOW_NAME, safe='')}&name_1={urllib.parse.quote(self.JOB_NAME, safe='')}" + REPORT_URL = f"https://{path}/{Path(settings.HTML_PAGE_FILE).name}?PR={self.PR_NUMBER}&sha={'latest' if latest else self.SHA}&name_0={urllib.parse.quote(self.WORKFLOW_NAME, safe='')}" return REPORT_URL def is_local_run(self): diff --git a/ci/praktika/artifact.py b/ci/praktika/artifact.py index ba05f18b9b1..1ae49e62259 100644 --- a/ci/praktika/artifact.py +++ b/ci/praktika/artifact.py @@ -1,3 +1,4 @@ +import copy from dataclasses import dataclass @@ -24,6 +25,14 @@ class Artifact: def is_s3_artifact(self): return self.type == Artifact.Type.S3 + def parametrize(self, names): + res = [] + for name in names: + obj = copy.deepcopy(self) + obj.name = name + res.append(obj) + return res + @classmethod def define_artifact(cls, name, type, path): return cls.Config(name=name, type=type, path=path) diff --git a/ci/praktika/hook_html.py b/ci/praktika/hook_html.py index e2faefb2fa9..5d476fe9d2b 100644 --- a/ci/praktika/hook_html.py +++ b/ci/praktika/hook_html.py @@ -128,6 +128,9 @@ class HtmlRunnerHooks: for job in _workflow.jobs: if job.name not in skip_jobs: result = Result.generate_pending(job.name) + # Preemptively add the general job log to the result directory to ensure + # the post-job handler can upload it, even if the job is terminated unexpectedly + result.set_files([Settings.RUN_LOG]) else: result = Result.generate_skipped(job.name, job_cache_records[job.name]) results.append(result) @@ -137,14 +140,14 @@ class HtmlRunnerHooks: summary_result.start_time = Utils.timestamp() assert _ResultS3.copy_result_to_s3_with_version(summary_result, version=0) - page_url = env.get_report_url(settings=Settings) + page_url = env.get_report_url(settings=Settings, latest=True) print(f"CI Status page url [{page_url}]") res1 = GH.post_commit_status( name=_workflow.name, status=Result.Status.PENDING, description="", - url=env.get_report_url(settings=Settings, latest=True), + url=page_url, ) res2 = GH.post_pr_comment( comment_body=f"Workflow [[{_workflow.name}]({page_url})], commit [{_Environment.get().SHA[:8]}]", diff --git a/ci/praktika/json.html b/ci/praktika/json.html index b11106719cd..717e6d39ac3 100644 --- a/ci/praktika/json.html +++ b/ci/praktika/json.html @@ -601,7 +601,7 @@ td.classList.add('time-column'); td.textContent = value ? formatDuration(value) : ''; } else if (column === 'info') { - td.textContent = value.includes('\n') ? '↵' : (value || ''); + td.textContent = value && value.includes('\n') ? '↵' : (value || ''); td.classList.add('info-column'); } diff --git a/ci/praktika/native_jobs.py b/ci/praktika/native_jobs.py index cff6c851d0e..58cd440283a 100644 --- a/ci/praktika/native_jobs.py +++ b/ci/praktika/native_jobs.py @@ -310,7 +310,7 @@ def _finish_workflow(workflow, job_name): print(env.get_needs_statuses()) print("Check Workflow results") - _ResultS3.copy_result_from_s3( + version = _ResultS3.copy_result_from_s3_with_version( Result.file_name_static(workflow.name), ) workflow_result = Result.from_fs(workflow.name) @@ -333,7 +333,7 @@ def _finish_workflow(workflow, job_name): # dump workflow result after update - to have an updated result in post workflow_result.dump() # add error into env - should apper in the report - env.add_info(ResultInfo.NOT_FINALIZED + f" [{result.name}]") + env.add_info(f"{result.name}: {ResultInfo.NOT_FINALIZED}") update_final_report = True job = workflow.get_job(result.name) if not job or not job.allow_merge_on_failure: @@ -358,9 +358,7 @@ def _finish_workflow(workflow, job_name): env.add_info(ResultInfo.GH_STATUS_ERROR) if update_final_report: - _ResultS3.copy_result_to_s3( - workflow_result, - ) + _ResultS3.copy_result_to_s3_with_version(workflow_result, version + 1) Result.from_fs(job_name).set_status(Result.Status.SUCCESS) diff --git a/ci/praktika/result.py b/ci/praktika/result.py index 8164b1d1295..082807fc9f0 100644 --- a/ci/praktika/result.py +++ b/ci/praktika/result.py @@ -121,6 +121,9 @@ class Result(MetaClasses.Serializable): def set_success(self) -> "Result": return self.set_status(Result.Status.SUCCESS) + def set_failed(self) -> "Result": + return self.set_status(Result.Status.FAILED) + def set_results(self, results: List["Result"]) -> "Result": self.results = results self.dump() diff --git a/ci/praktika/runner.py b/ci/praktika/runner.py index 38112dd5684..4ab7f8eeebc 100644 --- a/ci/praktika/runner.py +++ b/ci/praktika/runner.py @@ -1,3 +1,5 @@ +import glob +import json import os import re import sys @@ -58,6 +60,9 @@ class Runner: workflow_config.digest_dockers[docker.name] = Digest().calc_docker_digest( docker, workflow.dockers ) + + # work around for old clickhouse jobs + os.environ["DOCKER_TAG"] = json.dumps(workflow_config.digest_dockers) workflow_config.dump() Result.generate_pending(job.name).dump() @@ -119,8 +124,21 @@ class Runner: else: prefixes = [env.get_s3_prefix()] * len(required_artifacts) for artifact, prefix in zip(required_artifacts, prefixes): - s3_path = f"{Settings.S3_ARTIFACT_PATH}/{prefix}/{Utils.normalize_string(artifact._provided_by)}/{Path(artifact.path).name}" - assert S3.copy_file_from_s3(s3_path=s3_path, local_path=Settings.INPUT_DIR) + recursive = False + include_pattern = "" + if "*" in artifact.path: + s3_path = f"{Settings.S3_ARTIFACT_PATH}/{prefix}/{Utils.normalize_string(artifact._provided_by)}/" + recursive = True + include_pattern = Path(artifact.path).name + assert "*" in include_pattern + else: + s3_path = f"{Settings.S3_ARTIFACT_PATH}/{prefix}/{Utils.normalize_string(artifact._provided_by)}/{Path(artifact.path).name}" + assert S3.copy_file_from_s3( + s3_path=s3_path, + local_path=Settings.INPUT_DIR, + recursive=recursive, + include_pattern=include_pattern, + ) return 0 @@ -239,9 +257,11 @@ class Runner: info = f"ERROR: {ResultInfo.KILLED}" print(info) result.set_info(info).set_status(Result.Status.ERROR).dump() + else: + # TODO: add setting with different ways of storing general praktika log: always, on error, never. + # now let's store it on error only + result.files = [file for file in result.files if file != Settings.RUN_LOG] - if not result.is_ok(): - result.set_files(files=[Settings.RUN_LOG]) result.update_duration().dump() if run_exit_code == 0: @@ -262,10 +282,11 @@ class Runner: f"ls -l {artifact.path}", verbose=True ), f"Artifact {artifact.path} not found" s3_path = f"{Settings.S3_ARTIFACT_PATH}/{env.get_s3_prefix()}/{Utils.normalize_string(env.JOB_NAME)}" - link = S3.copy_file_to_s3( - s3_path=s3_path, local_path=artifact.path - ) - result.set_link(link) + for file_path in glob.glob(artifact.path): + link = S3.copy_file_to_s3( + s3_path=s3_path, local_path=file_path + ) + result.set_link(link) except Exception as e: error = ( f"ERROR: Failed to upload artifact [{artifact}], ex [{e}]" diff --git a/ci/praktika/s3.py b/ci/praktika/s3.py index 82034b57b80..8202d71b3d8 100644 --- a/ci/praktika/s3.py +++ b/ci/praktika/s3.py @@ -2,6 +2,7 @@ import dataclasses import json from pathlib import Path from typing import Dict +from urllib.parse import quote from praktika._environment import _Environment from praktika.settings import Settings @@ -55,7 +56,7 @@ class S3: bucket = s3_path.split("/")[0] endpoint = Settings.S3_BUCKET_TO_HTTP_ENDPOINT[bucket] assert endpoint - return f"https://{s3_full_path}".replace(bucket, endpoint) + return quote(f"https://{s3_full_path}".replace(bucket, endpoint), safe=":/?&=") @classmethod def put(cls, s3_path, local_path, text=False, metadata=None, if_none_matched=False): @@ -117,15 +118,21 @@ class S3: return res @classmethod - def copy_file_from_s3(cls, s3_path, local_path): + def copy_file_from_s3( + cls, s3_path, local_path, recursive=False, include_pattern="" + ): assert Path(s3_path), f"Invalid S3 Path [{s3_path}]" if Path(local_path).is_dir(): - local_path = Path(local_path) / Path(s3_path).name + pass else: assert Path( local_path ).parent.is_dir(), f"Parent path for [{local_path}] does not exist" cmd = f"aws s3 cp s3://{s3_path} {local_path}" + if recursive: + cmd += " --recursive" + if include_pattern: + cmd += f" --include {include_pattern}" res = cls.run_command_with_retries(cmd) return res diff --git a/ci/__init__.py b/ci/settings/__init__.py similarity index 100% rename from ci/__init__.py rename to ci/settings/__init__.py diff --git a/ci/settings/definitions.py b/ci/settings/definitions.py index 8ebf79231ac..ced1289b950 100644 --- a/ci/settings/definitions.py +++ b/ci/settings/definitions.py @@ -242,3 +242,4 @@ class JobNames: BUILD = "Build" STATELESS = "Stateless tests" STATEFUL = "Stateful tests" + STRESS = "Stress tests" diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 761ab8a6ebc..1d1bcca9fa7 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -16,6 +16,16 @@ class ArtifactNames: CH_ARM_RELEASE = "CH_ARM_RELEASE" CH_ARM_ASAN = "CH_ARM_ASAN" + CH_ODBC_B_AMD_DEBUG = "CH_ODBC_B_AMD_DEBUG" + CH_ODBC_B_AMD_RELEASE = "CH_ODBC_B_AMD_RELEASE" + CH_ODBC_B_ARM_RELEASE = "CH_ODBC_B_ARM_RELEASE" + CH_ODBC_B_ARM_ASAN = "CH_ODBC_B_ARM_ASAN" + + DEB_AMD_DEBUG = "DEB_AMD_DEBUG" + DEB_AMD_RELEASE = "DEB_AMD_RELEASE" + DEB_ARM_RELEASE = "DEB_ARM_RELEASE" + DEB_ARM_ASAN = "DEB_ARM_ASAN" + style_check_job = Job.Config( name=JobNames.STYLE_CHECK, @@ -41,7 +51,7 @@ fast_test_job = Job.Config( build_jobs = Job.Config( name=JobNames.BUILD, runs_on=["...from params..."], - requires=[JobNames.FAST_TEST], + requires=[], command="python3 ./ci/jobs/build_clickhouse.py --build-type {PARAMETER}", run_in_docker="clickhouse/fasttest", timeout=3600 * 2, @@ -63,10 +73,26 @@ build_jobs = Job.Config( ).parametrize( parameter=["amd_debug", "amd_release", "arm_release", "arm_asan"], provides=[ - [ArtifactNames.CH_AMD_DEBUG], - [ArtifactNames.CH_AMD_RELEASE], - [ArtifactNames.CH_ARM_RELEASE], - [ArtifactNames.CH_ARM_ASAN], + [ + ArtifactNames.CH_AMD_DEBUG, + ArtifactNames.DEB_AMD_DEBUG, + ArtifactNames.CH_ODBC_B_AMD_DEBUG, + ], + [ + ArtifactNames.CH_AMD_RELEASE, + ArtifactNames.DEB_AMD_RELEASE, + ArtifactNames.CH_ODBC_B_AMD_RELEASE, + ], + [ + ArtifactNames.CH_ARM_RELEASE, + ArtifactNames.DEB_ARM_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_RELEASE, + ], + [ + ArtifactNames.CH_ARM_ASAN, + ArtifactNames.DEB_ARM_ASAN, + ArtifactNames.CH_ODBC_B_ARM_ASAN, + ], ], runs_on=[ [RunnerLabels.BUILDER_AMD], @@ -105,12 +131,12 @@ stateless_tests_jobs = Job.Config( [RunnerLabels.FUNC_TESTER_ARM], ], requires=[ - [ArtifactNames.CH_AMD_DEBUG], - [ArtifactNames.CH_AMD_DEBUG], - [ArtifactNames.CH_AMD_RELEASE], - [ArtifactNames.CH_AMD_RELEASE], - [ArtifactNames.CH_ARM_ASAN], - [ArtifactNames.CH_ARM_ASAN], + [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], + [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], + [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], + [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], + [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], + [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], ], ) @@ -128,7 +154,7 @@ stateful_tests_jobs = Job.Config( ), ).parametrize( parameter=[ - "amd_debug,parallel", + "amd_release,parallel", ], runs_on=[ [RunnerLabels.BUILDER_AMD], @@ -138,6 +164,29 @@ stateful_tests_jobs = Job.Config( ], ) +# TODO: refactor job to be aligned with praktika style (remove wrappers, run in docker) +stress_test_jobs = Job.Config( + name=JobNames.STRESS, + runs_on=[RunnerLabels.BUILDER_ARM], + command="python3 ./tests/ci/stress_check.py {PARAMETER}", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/functional_stateful_tests.py", + ], + ), +).parametrize( + parameter=[ + "arm_release", + ], + runs_on=[ + [RunnerLabels.FUNC_TESTER_ARM], + ], + requires=[ + [ArtifactNames.DEB_ARM_RELEASE], + ], +) + + workflow = Workflow.Config( name="PR", event=Workflow.Event.PULL_REQUEST, @@ -148,27 +197,52 @@ workflow = Workflow.Config( *build_jobs, *stateless_tests_jobs, *stateful_tests_jobs, + *stress_test_jobs, ], artifacts=[ - Artifact.Config( - name=ArtifactNames.CH_AMD_DEBUG, + *Artifact.Config( + name="...", type=Artifact.Type.S3, path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + ).parametrize( + names=[ + ArtifactNames.CH_AMD_DEBUG, + ArtifactNames.CH_AMD_RELEASE, + ArtifactNames.CH_ARM_RELEASE, + ArtifactNames.CH_ARM_ASAN, + ] + ), + *Artifact.Config( + name="...", + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/build/programs/clickhouse-odbc-bridge", + ).parametrize( + names=[ + ArtifactNames.CH_ODBC_B_AMD_DEBUG, + ArtifactNames.CH_ODBC_B_AMD_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_ASAN, + ] ), Artifact.Config( - name=ArtifactNames.CH_AMD_RELEASE, + name=ArtifactNames.DEB_AMD_DEBUG, type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + path=f"{Settings.TEMP_DIR}/output/*.deb", ), Artifact.Config( - name=ArtifactNames.CH_ARM_RELEASE, + name=ArtifactNames.DEB_AMD_RELEASE, type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + path=f"{Settings.TEMP_DIR}/output/*.deb", ), Artifact.Config( - name=ArtifactNames.CH_ARM_ASAN, + name=ArtifactNames.DEB_ARM_RELEASE, type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + path=f"{Settings.TEMP_DIR}/output/*.deb", + ), + Artifact.Config( + name=ArtifactNames.DEB_ARM_ASAN, + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/output/*.deb", ), ], dockers=DOCKERS, diff --git a/packages/build b/packages/build index b2dd085d9dd..3c46c10948c 100755 --- a/packages/build +++ b/packages/build @@ -5,24 +5,14 @@ set -e # Avoid dependency on locale LC_ALL=C -# Normalize output directory -if [ -n "$OUTPUT_DIR" ]; then - OUTPUT_DIR=$(realpath -m "$OUTPUT_DIR") -fi - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -cd "$CUR_DIR" - -ROOT_DIR=$(readlink -f "$(git rev-parse --show-cdup)") - PKG_ROOT='root' DEB_ARCH=${DEB_ARCH:-amd64} -OUTPUT_DIR=${OUTPUT_DIR:-$ROOT_DIR} -[ -d "${OUTPUT_DIR}" ] || mkdir -p "${OUTPUT_DIR}" SANITIZER=${SANITIZER:-""} SOURCE=${SOURCE:-$PKG_ROOT} +cd "$(dirname "${BASH_SOURCE[0]}")" + HELP="${0} [--test] [--rpm] [-h|--help] --test - adds '+test' prefix to version --apk - build APK packages @@ -40,12 +30,7 @@ Used envs: VERSION_STRING='${VERSION_STRING}' - the package version to overwrite " -if [ -z "${VERSION_STRING}" ]; then - # Get CLICKHOUSE_VERSION_STRING from the current git repo - eval "$("$ROOT_DIR/tests/ci/version_helper.py" -e)" -else - CLICKHOUSE_VERSION_STRING=${VERSION_STRING} -fi +CLICKHOUSE_VERSION_STRING=${VERSION_STRING} export CLICKHOUSE_VERSION_STRING @@ -144,31 +129,32 @@ CLICKHOUSE_VERSION_STRING+=$VERSION_POSTFIX echo -e "\nCurrent version is $CLICKHOUSE_VERSION_STRING" for config in clickhouse*.yaml; do + if [[ $BUILD_TYPE != 'release' ]] && [[ "$config" == "clickhouse-keeper-dbg.yaml" ]]; then + continue + fi if [ -n "$MAKE_DEB" ] || [ -n "$MAKE_TGZ" ]; then echo "Building deb package for $config" - - # Preserve package path - exec 9>&1 - PKG_PATH=$(nfpm package --target "$OUTPUT_DIR" --config "$config" --packager deb | tee /dev/fd/9) - PKG_PATH=${PKG_PATH##*created package: } - exec 9>&- + PKG_PATH=$(nfpm package --target "$OUTPUT_DIR" --config "$config" --packager deb | tee /dev/stderr | grep "created package:" | sed 's/.*created package: //') fi if [ -n "$MAKE_APK" ]; then - echo "Building apk package for $config" - nfpm package --target "$OUTPUT_DIR" --config "$config" --packager apk + echo "Building apk package for $config" + nfpm package --target "$OUTPUT_DIR" --config "$config" --packager apk fi + if [ -n "$MAKE_ARCHLINUX" ]; then - echo "Building archlinux package for $config" - nfpm package --target "$OUTPUT_DIR" --config "$config" --packager archlinux + echo "Building archlinux package for $config" + nfpm package --target "$OUTPUT_DIR" --config "$config" --packager archlinux fi + if [ -n "$MAKE_RPM" ]; then - echo "Building rpm package for $config" - nfpm package --target "$OUTPUT_DIR" --config "$config" --packager rpm + echo "Building rpm package for $config" + nfpm package --target "$OUTPUT_DIR" --config "$config" --packager rpm fi + if [ -n "$MAKE_TGZ" ]; then - echo "Building tarball for $config" - deb2tgz "$PKG_PATH" + echo "Building tarball for $config" + deb2tgz "$PKG_PATH" fi done diff --git a/tests/ci/report.py b/tests/ci/report.py index 30b83b7409f..5f9184cb87a 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -4,6 +4,7 @@ import datetime import json import logging import os +import sys from ast import literal_eval from dataclasses import asdict, dataclass from html import escape @@ -414,6 +415,46 @@ class JobReport: dummy: bool = False exit_code: int = -1 + def to_praktika_result(self, job_name): + sys.path.append("./ci") + + # ugly WA to exclude ci.py file form import + current_dir = os.path.dirname(os.path.abspath(__file__)) + if current_dir in sys.path: + sys.path.remove(current_dir) + from praktika.result import ( # pylint: disable=import-error,import-outside-toplevel + Result, + ) + + if self.start_time: + dt = datetime.datetime.strptime(self.start_time, "%Y-%m-%d %H:%M:%S") + timestamp = dt.timestamp() + else: + timestamp = None + + sub_results = [] + for r in self.test_results: + sub_results.append( + Result( + name=r.name, + status=r.status, + info=r.raw_logs, + links=list(r.log_urls) if r.log_urls else [], + duration=r.time, + ) + ) + + return Result( + name=job_name, + status=self.status, + start_time=timestamp, + duration=self.duration, + results=sub_results, + files=( + [str(f) for f in self.additional_files] if self.additional_files else [] + ), + ) + @staticmethod def get_start_time_from_current(): return datetime.datetime.utcnow().strftime("%Y-%m-%d %H:%M:%S") @@ -474,6 +515,7 @@ class JobReport: to_file = to_file or JOB_REPORT_FILE with open(to_file, "w", encoding="utf-8") as json_file: json.dump(asdict(self), json_file, default=path_converter, indent=2) + return self def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestResults: diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index f9656e60448..b9b4b6886fb 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -9,6 +9,8 @@ import sys from pathlib import Path from typing import List, Tuple +from praktika.utils import Shell # pylint: disable=import-error + from build_download_helper import download_all_deb_packages from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image @@ -154,12 +156,19 @@ def run_stress_test(upgrade_check: bool = False) -> None: pr_info = PRInfo() - docker_image = pull_image(get_docker_image("clickhouse/stress-test")) + docker_image = pull_image(get_docker_image("clickhouse/stateful-test")) packages_path = temp_path / "packages" packages_path.mkdir(parents=True, exist_ok=True) - download_all_deb_packages(check_name, reports_path, packages_path) + if check_name in ("amd_release", "amd_debug", "arm_release"): + # this is praktika based CI + print("Copy input *.deb artifacts") + assert Shell.check( + f"cp /tmp/praktika/input/*.deb {packages_path}", verbose=True + ) + else: + download_all_deb_packages(check_name, reports_path, packages_path) server_log_path = temp_path / "server_log" server_log_path.mkdir(parents=True, exist_ok=True) @@ -201,6 +210,7 @@ def run_stress_test(upgrade_check: bool = False) -> None: result_path, server_log_path, run_log_path ) + Shell.check("pwd", verbose=True) JobReport( description=description, test_results=test_results, @@ -208,7 +218,7 @@ def run_stress_test(upgrade_check: bool = False) -> None: start_time=stopwatch.start_time_str, duration=stopwatch.duration_seconds, additional_files=additional_logs, - ).dump() + ).dump().to_praktika_result(job_name=f"Stress tests ({check_name})").dump() if state == "failure": sys.exit(1) diff --git a/tests/config/install.sh b/tests/config/install.sh index ba25f8bc425..54004ef3071 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -21,7 +21,7 @@ while [[ "$#" -gt 0 ]]; do --fast-test) FAST_TEST=1 ;; --s3-storage) EXPORT_S3_STORAGE_POLICIES=1 ;; --no-azure) NO_AZURE=1 ;; - *) echo "Unknown option: $1" ; exit 1 ;; + *) echo "Unknown option: $1" ;; esac shift done From 4de859a7ea6af9d8aecb319d876f7876f56f63fa Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Nov 2024 12:41:49 +0100 Subject: [PATCH 244/433] fix stress test script --- tests/ci/report.py | 12 ++++++++++-- tests/ci/stress_check.py | 9 ++++----- tests/config/install.sh | 2 +- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 5f9184cb87a..842d4d44423 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -23,6 +23,7 @@ from typing import ( from build_download_helper import APIException, get_gh_api from ci_config import CI +from ci_utils import Shell from env_helper import ( GITHUB_JOB, GITHUB_REPOSITORY, @@ -416,10 +417,11 @@ class JobReport: exit_code: int = -1 def to_praktika_result(self, job_name): - sys.path.append("./ci") - # ugly WA to exclude ci.py file form import + Shell.check("mkdir -p /tmp/praktika/") current_dir = os.path.dirname(os.path.abspath(__file__)) + sys.path.append(current_dir + "/../../ci") + sys.path.append(current_dir + "/../../") if current_dir in sys.path: sys.path.remove(current_dir) from praktika.result import ( # pylint: disable=import-error,import-outside-toplevel @@ -515,6 +517,12 @@ class JobReport: to_file = to_file or JOB_REPORT_FILE with open(to_file, "w", encoding="utf-8") as json_file: json.dump(asdict(self), json_file, default=path_converter, indent=2) + + # temporary WA to ease integration with praktika + check_name = os.getenv("CHECK_NAME", "") + if check_name: + self.to_praktika_result(job_name=check_name).dump() + return self diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b9b4b6886fb..4cfe2e0a99b 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -9,9 +9,8 @@ import sys from pathlib import Path from typing import List, Tuple -from praktika.utils import Shell # pylint: disable=import-error - from build_download_helper import download_all_deb_packages +from ci_utils import Shell from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH @@ -156,8 +155,6 @@ def run_stress_test(upgrade_check: bool = False) -> None: pr_info = PRInfo() - docker_image = pull_image(get_docker_image("clickhouse/stateful-test")) - packages_path = temp_path / "packages" packages_path.mkdir(parents=True, exist_ok=True) @@ -167,8 +164,10 @@ def run_stress_test(upgrade_check: bool = False) -> None: assert Shell.check( f"cp /tmp/praktika/input/*.deb {packages_path}", verbose=True ) + docker_image = pull_image(get_docker_image("clickhouse/stateful-test")) else: download_all_deb_packages(check_name, reports_path, packages_path) + docker_image = pull_image(get_docker_image("clickhouse/stress-test")) server_log_path = temp_path / "server_log" server_log_path.mkdir(parents=True, exist_ok=True) @@ -218,7 +217,7 @@ def run_stress_test(upgrade_check: bool = False) -> None: start_time=stopwatch.start_time_str, duration=stopwatch.duration_seconds, additional_files=additional_logs, - ).dump().to_praktika_result(job_name=f"Stress tests ({check_name})").dump() + ).dump() if state == "failure": sys.exit(1) diff --git a/tests/config/install.sh b/tests/config/install.sh index 54004ef3071..ba25f8bc425 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -21,7 +21,7 @@ while [[ "$#" -gt 0 ]]; do --fast-test) FAST_TEST=1 ;; --s3-storage) EXPORT_S3_STORAGE_POLICIES=1 ;; --no-azure) NO_AZURE=1 ;; - *) echo "Unknown option: $1" ;; + *) echo "Unknown option: $1" ; exit 1 ;; esac shift done From 1fcc70a89c1089dd6af8eff7deddf7202575157c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 24 Nov 2024 21:14:39 +0000 Subject: [PATCH 245/433] Enable clang-tidy bugprone-reserved-identifier No significant difference in compile times anymore: real 58m41.291s user 5676m6.870s sys 122m27.022s vs. real 58m23.603s user 5539m44.135s sys 119m26.956s --- .clang-tidy | 1 - 1 file changed, 1 deletion(-) diff --git a/.clang-tidy b/.clang-tidy index 2a2d416179d..d7e35f16ad5 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -28,7 +28,6 @@ Checks: [ '-bugprone-multi-level-implicit-pointer-conversion', '-bugprone-narrowing-conversions', '-bugprone-not-null-terminated-result', - '-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged '-bugprone-unchecked-optional-access', '-bugprone-crtp-constructor-accessibility', From e3c6689144be01a053bb60dc7223080df2d2df86 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 24 Nov 2024 22:55:56 +0000 Subject: [PATCH 246/433] fix libprotobuf-mutator-cmake --- contrib/libprotobuf-mutator-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt index 9bbd6c17caa..d86aa13ebe0 100644 --- a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -18,7 +18,6 @@ add_library(_protobuf-mutator target_include_directories(_protobuf-mutator BEFORE PUBLIC "${LIBRARY_DIR}/src") # ... which includes target_include_directories(_protobuf-mutator BEFORE PUBLIC "${LIBRARY_DIR}") -target_include_directories(_protobuf-mutator BEFORE PUBLIC "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") target_link_libraries(_protobuf-mutator ch_contrib::protobuf) From 46b663d5037840a780cfcc6d4cd73cdadf1b8910 Mon Sep 17 00:00:00 2001 From: Kirill <71129570+kirillgarbar@users.noreply.github.com> Date: Mon, 25 Nov 2024 12:57:59 +0300 Subject: [PATCH 247/433] Try to disable ordinary db warning --- .../0_stateless/03167_attach_as_replicated_errors.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh index 42aa122709c..caa1c6fb091 100755 --- a/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh +++ b/tests/queries/0_stateless/03167_attach_as_replicated_errors.sh @@ -1,13 +1,13 @@ #!/usr/bin/env bash # Tags: zookeeper, no-replicated-database, no-ordinary-database, no-shared-merge-tree +# Creation of a database with Ordinary engine emits a warning. +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal + CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -# Creation of a database with Ordinary engine emits a warning. -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal - ORDINARY_DB="ordinary_$CLICKHOUSE_DATABASE" ${CLICKHOUSE_CLIENT} --allow_deprecated_database_ordinary=1 -n -q " CREATE DATABASE $ORDINARY_DB ENGINE = Ordinary; From 1579ed83bb61715a580ea6c3b6970f4c24d2944c Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Tue, 26 Nov 2024 02:07:06 +0800 Subject: [PATCH 248/433] optimize the mutex with shared_mutex in MemoryTracker The mutex lock `overcommit_m` has protected while read and write the global variables in the OvercommitTracker of MemoryTracker. But it will cause lots of `native_queued_spin_lock_slowpath` in ClickBench Q8 and Q42 on large cores system. Most of time it will only read, but not update. The patch will replace the mutex lock with read/write lock. Test the patch with clickbench on 256x2 vCPUs system. Overall geomean has got 6.8% performance gain. Q8, 24, Q26 and Q42 have got 77%, 19.5%, 19.5% and 11.4% performance gain. Signed-off-by: Jiebin Sun --- src/Common/OvercommitTracker.cpp | 30 ++++++++++++++++++++++-------- src/Common/OvercommitTracker.h | 5 +++-- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 2a453596dab..e530d486a6a 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -46,7 +46,7 @@ OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int // always called with already acquired global mutex in // ProcessListEntry::~ProcessListEntry(). auto global_lock = process_list->unsafeLock(); - std::unique_lock lk(overcommit_m); + std::unique_lock lk(overcommit_m); size_t id = next_id++; @@ -115,12 +115,19 @@ void OvercommitTracker::tryContinueQueryExecutionAfterFree(Int64 amount) if (OvercommitTrackerBlockerInThread::isBlocked()) return; - std::lock_guard guard(overcommit_m); + std::shared_lock read_lock(overcommit_m); if (cancellation_state != QueryCancellationState::NONE) { - freed_memory += amount; - if (freed_memory >= required_memory) - releaseThreads(); + read_lock.unlock(); + { + std::lock_guard lk(overcommit_m); + if (cancellation_state != QueryCancellationState::NONE) + { + freed_memory += amount; + if (freed_memory >= required_memory) + releaseThreads(); + } + } } } @@ -128,11 +135,18 @@ void OvercommitTracker::onQueryStop(MemoryTracker * tracker) { DENY_ALLOCATIONS_IN_SCOPE; - std::lock_guard lk(overcommit_m); + std::shared_lock read_lock(overcommit_m); if (picked_tracker == tracker) { - reset(); - cv.notify_all(); + read_lock.unlock(); + { + std::lock_guard lk(overcommit_m); + if (picked_tracker == tracker) + { + reset(); + cv.notify_all(); + } + } } } diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 7712ff999ef..d9e657dda5b 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -9,6 +9,7 @@ #include #include #include +#include // This struct is used for the comparison of query memory usage. struct OvercommitRatio @@ -83,8 +84,8 @@ protected: // This mutex is used to disallow concurrent access // to picked_tracker and cancellation_state variables. - std::mutex overcommit_m; - std::condition_variable cv; + DB::SharedMutex overcommit_m; + std::condition_variable_any cv; // Specifies memory tracker of the chosen to stop query. // If soft limit is not set, all the queries which reach hard limit must stop. From d9877280c1bb655d07fcdd869509f59dcc495bf5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 11:50:16 +0100 Subject: [PATCH 249/433] 24.11 changelog missing entries and changes --- CHANGELOG.md | 76 +++++++++++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 73 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b757c680353..cd3a5c83878 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -54,10 +54,10 @@ * Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. [#71086](https://github.com/ClickHouse/ClickHouse/pull/71086) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Do not calculate heavy asynchronous metrics by default. The feature was introduced in [#40332](https://github.com/ClickHouse/ClickHouse/issues/40332), but it isn't good to have a heavy background job that is needed for only a single customer. [#71087](https://github.com/ClickHouse/ClickHouse/pull/71087) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. [#71473](https://github.com/ClickHouse/ClickHouse/pull/71473) ([Pablo Marcos](https://github.com/pamarcos)). +* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). #### Improvement * Higher-order functions with constant arrays and constant captured arguments will return constants. [#58400](https://github.com/ClickHouse/ClickHouse/pull/58400) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. [#62125](https://github.com/ClickHouse/ClickHouse/pull/62125) ([Shichao Jin](https://github.com/jsc0218)). * Query plan step names (`EXPLAIN PLAN json=1`) and pipeline processor names (`EXPLAIN PIPELINE compact=0,graph=1`) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. [#63518](https://github.com/ClickHouse/ClickHouse/pull/63518) ([qhsong](https://github.com/qhsong)). * Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting `check_objects_after_upload`. [#64847](https://github.com/ClickHouse/ClickHouse/pull/64847) ([Smita Kulkarni](https://github.com/SmitaRKulkarni)). * Use `Atomic` database by default in `clickhouse-local`. Address items 1 and 5 from [#50647](https://github.com/ClickHouse/ClickHouse/issues/50647). Closes [#44817](https://github.com/ClickHouse/ClickHouse/issues/44817). [#68024](https://github.com/ClickHouse/ClickHouse/pull/68024) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -89,11 +89,10 @@ * `clickhouse-local` uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. [#71620](https://github.com/ClickHouse/ClickHouse/pull/71620) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * The command line applications will highlight syntax even for multi-statements. [#71622](https://github.com/ClickHouse/ClickHouse/pull/71622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Command-line applications will return non-zero exit codes on errors. In previous versions, the `disks` application returned zero on errors, and other applications returned zero for errors 256 (`PARTITION_ALREADY_EXISTS`) and 512 (`SET_NON_GRANTED_ROLE`). [#71623](https://github.com/ClickHouse/ClickHouse/pull/71623) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. ### Documentation entry for user-facing changes. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* When user/group is given as ID, the `clickhouse su` fails. This patch fixes it to accept `UID:GID` as well. [#71626](https://github.com/ClickHouse/ClickHouse/pull/71626) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * The `Vertical` format (which is also activated when you end your query with `\G`) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. [#71630](https://github.com/ClickHouse/ClickHouse/pull/71630) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Allow to disable memory buffer increase for filesystem cache via setting `filesystem_cache_prefer_bigger_buffer_size`. [#71640](https://github.com/ClickHouse/ClickHouse/pull/71640) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a separate setting `background_download_max_file_segment_size` for background download max file segment size in filesystem cache. [#71648](https://github.com/ClickHouse/ClickHouse/pull/71648) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Changes the default value of `enable_http_compression` from 0 to 1. Closes [#71591](https://github.com/ClickHouse/ClickHouse/issues/71591). [#71774](https://github.com/ClickHouse/ClickHouse/pull/71774) ([Peter Nguyen](https://github.com/petern48)). * Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. [#71785](https://github.com/ClickHouse/ClickHouse/pull/71785) ([Pavel Kruglov](https://github.com/Avogar)). * Previously reading from `system.asynchronous_metrics` would wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. [#71798](https://github.com/ClickHouse/ClickHouse/pull/71798) ([Alexander Gololobov](https://github.com/davenger)). * Set `polling_max_timeout_ms` to 10 minutes, `polling_backoff_ms` to 30 seconds. [#71817](https://github.com/ClickHouse/ClickHouse/pull/71817) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -103,6 +102,77 @@ * Allow_reorder_prewhere_conditions is on by default with old compatibility settings. [#71867](https://github.com/ClickHouse/ClickHouse/pull/71867) ([Raúl Marín](https://github.com/Algunenano)). * On the advanced dashboard HTML page added a dropdown selector for the dashboard from `system.dashboards` table. [#72081](https://github.com/ClickHouse/ClickHouse/pull/72081) ([Sergei Trifonov](https://github.com/serxa)). +#### Bug Fix (user-visible misbehavior in an official stable release) +* The parts deduplicated during `ATTACH PART` query don't get stuck with the `attaching_` prefix anymore. [#65636](https://github.com/ClickHouse/ClickHouse/pull/65636) ([Kirill](https://github.com/kirillgarbar)). +* Fix for the bug when dateTime64 losing precision for the `IN` function. [#67230](https://github.com/ClickHouse/ClickHouse/pull/67230) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix possible logical error when using functions with `IGNORE/RESPECT NULLS` in `ORDER BY ... WITH FILL`, close [#57609](https://github.com/ClickHouse/ClickHouse/issues/57609). [#68234](https://github.com/ClickHouse/ClickHouse/pull/68234) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed rare logical errors in asynchronous inserts with format `Native` in case of reached memory limit. [#68965](https://github.com/ClickHouse/ClickHouse/pull/68965) ([Anton Popov](https://github.com/CurtizJ)). +* Fix COMMENT in CREATE TABLE for EPHEMERAL column. [#70458](https://github.com/ClickHouse/ClickHouse/pull/70458) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix logical error in JSONExtract with LowCardinality(Nullable). [#70549](https://github.com/ClickHouse/ClickHouse/pull/70549) ([Pavel Kruglov](https://github.com/Avogar)). +* Allow system drop replica zkpath when there is another replica with the same zk path. [#70642](https://github.com/ClickHouse/ClickHouse/pull/70642) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix a crash and a leak in AggregateFunctionGroupArraySorted. [#70820](https://github.com/ClickHouse/ClickHouse/pull/70820) ([Michael Kolupaev](https://github.com/al13n321)). +* Add ability to override Content-Type by user headers in the URL engine. [#70859](https://github.com/ClickHouse/ClickHouse/pull/70859) ([Artem Iurin](https://github.com/ortyomka)). +* Fix logical error in `StorageS3Queue` "Cannot create a persistent node in /processed since it already exists". [#70984](https://github.com/ClickHouse/ClickHouse/pull/70984) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed named sessions not being closed and hanging on forever under certain circumstances. [#70998](https://github.com/ClickHouse/ClickHouse/pull/70998) ([Márcio Martins](https://github.com/marcio-absmartly)). +* Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. [#71089](https://github.com/ClickHouse/ClickHouse/pull/71089) ([Shichao Jin](https://github.com/jsc0218)). +* Fix `AT_* is out of range` problem when running on Oracle Linux UEK 6.10. [#71109](https://github.com/ClickHouse/ClickHouse/pull/71109) ([Örjan Fors](https://github.com/op)). +* Fix wrong value in system.query_metric_log due to unexpected race condition. [#71124](https://github.com/ClickHouse/ClickHouse/pull/71124) ([Pablo Marcos](https://github.com/pamarcos)). +* Check if default DB is present after authorization. Fixes [#71097](https://github.com/ClickHouse/ClickHouse/issues/71097). [#71140](https://github.com/ClickHouse/ClickHouse/pull/71140) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. [#71168](https://github.com/ClickHouse/ClickHouse/pull/71168) ([李扬](https://github.com/taiyang-li)). +* Fix bad_weak_ptr exception with Dynamic in functions comparison. [#71183](https://github.com/ClickHouse/ClickHouse/pull/71183) ([Pavel Kruglov](https://github.com/Avogar)). +* Checks that read 7z file is on a local machine. [#71184](https://github.com/ClickHouse/ClickHouse/pull/71184) ([Daniil Ivanik](https://github.com/divanik)). +* Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. [#71186](https://github.com/ClickHouse/ClickHouse/pull/71186) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix ignoring format settings in Native format via HTTP and Async Inserts. [#71193](https://github.com/ClickHouse/ClickHouse/pull/71193) ([Pavel Kruglov](https://github.com/Avogar)). +* SELECT queries run with setting `use_query_cache = 1` are no longer rejected if the name of a system table appears as a literal, e.g. `SELECT * FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;` now works. [#71254](https://github.com/ClickHouse/ClickHouse/pull/71254) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. [#71261](https://github.com/ClickHouse/ClickHouse/pull/71261) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. [#71299](https://github.com/ClickHouse/ClickHouse/pull/71299) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix incomplete cleanup of parallel output format in the client. [#71304](https://github.com/ClickHouse/ClickHouse/pull/71304) ([Raúl Marín](https://github.com/Algunenano)). +* Added missing unescaping in named collections. Without fix clickhouse-server can't start. [#71308](https://github.com/ClickHouse/ClickHouse/pull/71308) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Fix async inserts with empty blocks via native protocol. [#71312](https://github.com/ClickHouse/ClickHouse/pull/71312) ([Anton Popov](https://github.com/CurtizJ)). +* Fix inconsistent AST formatting when granting wrong wildcard grants [#71309](https://github.com/ClickHouse/ClickHouse/issues/71309). [#71332](https://github.com/ClickHouse/ClickHouse/pull/71332) ([pufit](https://github.com/pufit)). +* Add try/catch to data parts destructors to avoid terminate. [#71364](https://github.com/ClickHouse/ClickHouse/pull/71364) ([alesapin](https://github.com/alesapin)). +* Check suspicious and experimental types in JSON type hints. [#71369](https://github.com/ClickHouse/ClickHouse/pull/71369) ([Pavel Kruglov](https://github.com/Avogar)). +* Start memory worker thread on non-Linux OS too (fixes [#71051](https://github.com/ClickHouse/ClickHouse/issues/71051)). [#71384](https://github.com/ClickHouse/ClickHouse/pull/71384) ([Alexandre Snarskii](https://github.com/snar)). +* Fix error Invalid number of rows in Chunk with Variant column. [#71388](https://github.com/ClickHouse/ClickHouse/pull/71388) ([Pavel Kruglov](https://github.com/Avogar)). +* Fix error column "attgenerated" does not exist for old PostgreSQL, fix [#60651](https://github.com/ClickHouse/ClickHouse/issues/60651). [#71396](https://github.com/ClickHouse/ClickHouse/pull/71396) ([0xMihalich](https://github.com/0xMihalich)). +* To avoid spamming the server logs, failing authentication attempts are now logged at level `DEBUG` instead of `ERROR`. [#71405](https://github.com/ClickHouse/ClickHouse/pull/71405) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix crash in `mongodb` table function when passing wrong arguments (e.g. `NULL`). [#71426](https://github.com/ClickHouse/ClickHouse/pull/71426) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix crash with optimize_rewrite_array_exists_to_has. [#71432](https://github.com/ClickHouse/ClickHouse/pull/71432) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed the usage of setting `max_insert_delayed_streams_for_parallel_write` in inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. [#71474](https://github.com/ClickHouse/ClickHouse/pull/71474) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error `Argument for function must be constant` (old analyzer) in case when arrayJoin can apparently appear in `WHERE` condition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. [#71476](https://github.com/ClickHouse/ClickHouse/pull/71476) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent crash in SortCursor with 0 columns (old analyzer). [#71494](https://github.com/ClickHouse/ClickHouse/pull/71494) ([Raúl Marín](https://github.com/Algunenano)). +* Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. [#71500](https://github.com/ClickHouse/ClickHouse/pull/71500) ([李扬](https://github.com/taiyang-li)). +* Fix counting column size in wide part for Dynamic and JSON types. [#71526](https://github.com/ClickHouse/ClickHouse/pull/71526) ([Pavel Kruglov](https://github.com/Avogar)). +* Analyzer fix when query inside materialized view uses IN with CTE. Closes [#65598](https://github.com/ClickHouse/ClickHouse/issues/65598). [#71538](https://github.com/ClickHouse/ClickHouse/pull/71538) ([Maksim Kita](https://github.com/kitaisreal)). +* Avoid crash when using a UDF in a constraint. [#71541](https://github.com/ClickHouse/ClickHouse/pull/71541) ([Raúl Marín](https://github.com/Algunenano)). +* Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. [#71580](https://github.com/ClickHouse/ClickHouse/pull/71580) ([Pablo Marcos](https://github.com/pamarcos)). +* Fix server crashes while using materialized view with certain engines. [#71593](https://github.com/ClickHouse/ClickHouse/pull/71593) ([Pervakov Grigorii](https://github.com/GrigoryPervakov)). +* Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes [#71677](https://github.com/ClickHouse/ClickHouse/issues/71677). [#71678](https://github.com/ClickHouse/ClickHouse/pull/71678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes [#71647](https://github.com/ClickHouse/ClickHouse/issues/71647). [#71679](https://github.com/ClickHouse/ClickHouse/pull/71679) ([Amos Bird](https://github.com/amosbird)). +* Don't transform constant set in predicates over partition columns in case of NOT IN operator. [#71695](https://github.com/ClickHouse/ClickHouse/pull/71695) ([Eduard Karacharov](https://github.com/korowa)). +* Fix docker init script fail log message for more clean understanding. [#71734](https://github.com/ClickHouse/ClickHouse/pull/71734) ([Андрей](https://github.com/andreineustroev)). +* Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error `Bad cast from type DB::ColumnVector to DB::ColumnNullable`. [#71742](https://github.com/ClickHouse/ClickHouse/pull/71742) ([Pavel Kruglov](https://github.com/Avogar)). +* Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. [#71845](https://github.com/ClickHouse/ClickHouse/pull/71845) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. [#71849](https://github.com/ClickHouse/ClickHouse/pull/71849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fixed filling of defaults after parsing into sparse columns. [#71854](https://github.com/ClickHouse/ClickHouse/pull/71854) ([Anton Popov](https://github.com/CurtizJ)). +* Fix GROUPING function error when input is ALIAS on distributed table, close [#68602](https://github.com/ClickHouse/ClickHouse/issues/68602). [#71855](https://github.com/ClickHouse/ClickHouse/pull/71855) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fix possible crash when using `allow_experimental_join_condition`, close [#71693](https://github.com/ClickHouse/ClickHouse/issues/71693). [#71857](https://github.com/ClickHouse/ClickHouse/pull/71857) ([Vladimir Cherkasov](https://github.com/vdimir)). +* Fixed select statements that use `WITH TIES` clause which might not return enough rows. [#71886](https://github.com/ClickHouse/ClickHouse/pull/71886) ([wxybear](https://github.com/wxybear)). +* Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. [#71894](https://github.com/ClickHouse/ClickHouse/pull/71894) ([Udi](https://github.com/udiz)). +* `clickhouse-benchmark` reported wrong metrics for queries taking longer than one second. [#71898](https://github.com/ClickHouse/ClickHouse/pull/71898) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. [#71901](https://github.com/ClickHouse/ClickHouse/pull/71901) ([Julia Kartseva](https://github.com/jkartseva)). +* Use auxiliary keepers for cluster autodiscovery. [#71911](https://github.com/ClickHouse/ClickHouse/pull/71911) ([Anton Ivashkin](https://github.com/ianton-ru)). +* Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes [#69975](https://github.com/ClickHouse/ClickHouse/issues/69975). [#71946](https://github.com/ClickHouse/ClickHouse/pull/71946) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed case when `s3`/`s3Cluster` functions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (like `pattern/*`) and an empty object should exist with the key `pattern/` (such objects automatically created by S3 Console). Also default value for setting `s3_skip_empty_files` changed from `false` to `true` by default. [#71947](https://github.com/ClickHouse/ClickHouse/pull/71947) ([Nikita Taranov](https://github.com/nickitat)). +* Fix a crash in clickhouse-client syntax highlighting. Closes [#71864](https://github.com/ClickHouse/ClickHouse/issues/71864). [#71949](https://github.com/ClickHouse/ClickHouse/pull/71949) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `Illegal type` error for `MergeTree` tables with binary monotonic function in `ORDER BY` when the first argument is constant. Fixes [#71941](https://github.com/ClickHouse/ClickHouse/issues/71941). [#71966](https://github.com/ClickHouse/ClickHouse/pull/71966) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or `Inconsistent AST formatting`. [#71982](https://github.com/ClickHouse/ClickHouse/pull/71982) ([Pavel Kruglov](https://github.com/Avogar)). +* When insert a record by `clickhouse-client`, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. [#71991](https://github.com/ClickHouse/ClickHouse/pull/71991) ([Han Fei](https://github.com/hanfei1991)). +* Fix formatting of `MOVE PARTITION ... TO TABLE ...` alter commands when `format_alter_commands_with_parentheses` is enabled. [#72080](https://github.com/ClickHouse/ClickHouse/pull/72080) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. [#71162](https://github.com/ClickHouse/ClickHouse/pull/71162) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. [#70900](https://github.com/ClickHouse/ClickHouse/pull/70900) ([filimonov](https://github.com/filimonov)). +* Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. [#70297](https://github.com/ClickHouse/ClickHouse/pull/70297) ([Miсhael Stetsyuk](https://github.com/mstetsyuk)). + ### ClickHouse release 24.10, 2024-10-31 #### Backward Incompatible Change From 690cb514f25645a9da253d0389144e632db491a2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Nov 2024 12:11:06 +0100 Subject: [PATCH 250/433] Less bloat --- .../AggregateFunctionsArgMinArgMax.cpp | 123 ++++++++++++------ 1 file changed, 83 insertions(+), 40 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index 5148e401cd6..931d2f87bad 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -251,68 +251,111 @@ public: } }; -/// If we will add more it will compile forever -using TypesToCreateSpecializedData = std::tuple; - -template -IAggregateFunction * createForPair(const TypeIndex & result_type, const TypeIndex & value_type, const DataTypes & argument_types) +template +IAggregateFunction * createWithTwoTypesSecond(const DataTypes & argument_types) { - if (TypeToTypeIndex == result_type && value_type == TypeToTypeIndex) + const DataTypePtr & value_type = argument_types[1]; + WhichDataType which_value(value_type); + + if (which_value.idx == TypeIndex::UInt8) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt16) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt32) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt64) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Int8) { - using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; return new AggregateFunctionArgMinMax(argument_types); } + if (which_value.idx == TypeIndex::Int16) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Int32) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Int64) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Float32) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Float64) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::Date) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::DateTime) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + return nullptr; } -template -IAggregateFunction * tryValueTypes(const DataTypes & argument_types, const TypeIndex & result_type, const TypeIndex & value_type, std::tuple) +template +IAggregateFunction * createWithTwoTypes(const DataTypes & argument_types) { - IAggregateFunction * result = nullptr; - ((result = result ? result : createForPair(result_type, value_type, argument_types)), ...); - return result; + const DataTypePtr & result_type = argument_types[0]; + WhichDataType which_result(result_type); + + if (which_result.idx == TypeIndex::UInt8) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::UInt16) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::UInt32) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::UInt64) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Int8) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Int16) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Int32) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Int64) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Float32) return createWithTwoTypesSecond(argument_types); + if (which_result.idx == TypeIndex::Float64) return createWithTwoTypesSecond(argument_types); + + return nullptr; } -template -IAggregateFunction * tryResultTypes(const DataTypes & argument_types, const TypeIndex result_idx, const TypeIndex value_idx, std::tuple, std::tuple value_tuple) -{ - IAggregateFunction * result = nullptr; - ((result = result ? result : tryValueTypes(argument_types, result_idx, value_idx, value_tuple)), ...); - return result; -} template AggregateFunctionPtr createAggregateFunctionArgMinMax(const std::string & name, const DataTypes & argument_types, const Array &, const Settings *) { - assertBinary(name, argument_types); - using TypesToCreateSpecializedDataTuple = TypesToCreateSpecializedData; + AggregateFunctionPtr result = AggregateFunctionPtr(createWithTwoTypes(argument_types)); - const DataTypePtr & result_type = argument_types[0]; - const DataTypePtr & value_type = argument_types[1]; - - WhichDataType which_result(result_type); - WhichDataType which_value(value_type); - - auto convert_date_type = [] (TypeIndex type_index) - { - if (type_index == TypeIndex::Date) - return TypeToTypeIndex; - else if (type_index == TypeIndex::DateTime) - return TypeToTypeIndex; - else - return type_index; - }; - - AggregateFunctionPtr result = AggregateFunctionPtr(tryResultTypes(argument_types, convert_date_type(which_result.idx), convert_date_type(which_value.idx), TypesToCreateSpecializedDataTuple{}, TypesToCreateSpecializedDataTuple{})); if (!result) { + const DataTypePtr & value_type = argument_types[1]; WhichDataType which(value_type); #define DISPATCH(TYPE) \ - if (which_value.idx == TypeIndex::TYPE) \ + if (which.idx == TypeIndex::TYPE) \ return AggregateFunctionPtr(new AggregateFunctionArgMinMax>, isMin>(argument_types)); /// NOLINT FOR_SINGLE_VALUE_NUMERIC_TYPES(DISPATCH) #undef DISPATCH From b44c7b0d5b1d2458bc273a52e25a03415350dac8 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 25 Nov 2024 11:15:48 +0000 Subject: [PATCH 251/433] Add one more check --- .../ObjectStorage/DataLakes/IcebergMetadata.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 12a3b93f41f..4eadb158ffa 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -352,7 +352,7 @@ std::pair parseTableSchema( LOG_WARNING( metadata_logger, "Iceberg table schema was parsed using v2 specification, but it was impossible to parse it using v1 specification. Be " - "aware that you Iceberg writing Engine violates Iceberg specification. Error during parsing {}", + "aware that you Iceberg writing engine violates Iceberg specification. Error during parsing {}", first_error.displayText()); } catch (const Exception & second_error) @@ -363,9 +363,6 @@ std::pair parseTableSchema( first_error.displayText(), second_error.displayText()); } - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, e.displayText()); - } } } @@ -563,7 +560,13 @@ Strings IcebergMetadata::getDataFiles() const /// Manifest file should always have table schema in avro file metadata. By now we don't support tables with evolved schema, /// so we should check if all manifest files have the same schema as in table metadata. auto avro_metadata = manifest_file_reader->metadata(); - std::vector schema_json = avro_metadata["schema"]; + auto avro_schema_it = avro_metadata.find("schema"); + if (avro_schema_it == avro_metadata.end()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot read Iceberg table: manifest file {} doesn't have table schema in its metadata", + manifest_file); + std::vector schema_json = avro_schema_it->second; String schema_json_string = String(reinterpret_cast(schema_json.data()), schema_json.size()); Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(schema_json_string); From b1cbb3f2de9dec67e09d2b2a4c7236c2dd5f0d79 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Nov 2024 12:20:10 +0100 Subject: [PATCH 252/433] Fix style --- .../AggregateFunctionsArgMinArgMax.cpp | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp index 931d2f87bad..4d3989dd2d7 100644 --- a/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsArgMinArgMax.cpp @@ -258,23 +258,23 @@ IAggregateFunction * createWithTwoTypesSecond(const DataTypes & argument_types) const DataTypePtr & value_type = argument_types[1]; WhichDataType which_value(value_type); - if (which_value.idx == TypeIndex::UInt8) - { - using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; - return new AggregateFunctionArgMinMax(argument_types); - } - if (which_value.idx == TypeIndex::UInt16) - { - using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; - return new AggregateFunctionArgMinMax(argument_types); - } - if (which_value.idx == TypeIndex::UInt32) - { - using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; - return new AggregateFunctionArgMinMax(argument_types); - } - if (which_value.idx == TypeIndex::UInt64) - { + if (which_value.idx == TypeIndex::UInt8) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt16) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt32) + { + using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; + return new AggregateFunctionArgMinMax(argument_types); + } + if (which_value.idx == TypeIndex::UInt64) + { using Data = AggregateFunctionArgMinMaxData, SingleValueDataFixed>; return new AggregateFunctionArgMinMax(argument_types); } From 856f73a7c26bceeeda7a28ddaf54ec84d993b861 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 25 Nov 2024 12:40:34 +0100 Subject: [PATCH 253/433] Fix typo in intersect.md --- docs/en/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 46eb61a94d6..d3916ee9ee3 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -29,7 +29,7 @@ The condition could be any expression based on your requirements. Here is a simple example that intersects the numbers 1 to 10 with the numbers 3 to 8: ```sql -SELECT number FROM numbers(1,10) INTERSECT SELECT number FROM numbers(3,8); +SELECT number FROM numbers(1,10) INTERSECT SELECT number FROM numbers(3,6); ``` Result: From dcd07e25ac53248e6557eb253c884d293f099915 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 13:20:47 +0100 Subject: [PATCH 254/433] Check proper sorting in debug builds --- src/Interpreters/sortBlock.cpp | 58 ++++++++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 7b19d338ee8..bdf672623da 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -272,6 +272,56 @@ bool isAlreadySortedImpl(size_t rows, Comparator compare) return true; } +template +void checkSortedWithPermutationImpl(size_t rows, Comparator compare, UInt64 limit, const IColumn::Permutation & permutation) +{ + if (limit && limit < rows) + rows = limit; + + for (size_t i = 1; i < rows; ++i) + { + const size_t current_row = permutation[i]; + const size_t previous_row = permutation[i - 1]; + + if (compare(current_row, previous_row)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Rows are not sorted with permuation, position {}, previous_row index {}, current_row index {}", i, previous_row, current_row); + } +} + +void checkSortedWithPermutation(const Block & block, const SortDescription & description, UInt64 limit, const IColumn::Permutation & permutation) +{ + if (!block) + return; + + ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description); + bool is_collation_required = false; + + for (auto & column_with_sort_desc : columns_with_sort_desc) + { + if (isCollationRequired(column_with_sort_desc.description)) + { + is_collation_required = true; + break; + } + } + + size_t rows = block.rows(); + + if (is_collation_required) + { + PartialSortingLessWithCollation less(columns_with_sort_desc); + checkSortedWithPermutationImpl(rows, less, limit, permutation); + return; + } + else + { + PartialSortingLess less(columns_with_sort_desc); + checkSortedWithPermutationImpl(rows, less, limit, permutation); + return; + } +} + } void sortBlock(Block & block, const SortDescription & description, UInt64 limit) @@ -279,6 +329,10 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) IColumn::Permutation permutation; getBlockSortPermutationImpl(block, description, IColumn::PermutationSortStability::Unstable, limit, permutation); +#ifndef NDEBUG + checkSortedWithPermutation(block, description, limit, permutation); +#endif + if (permutation.empty()) return; @@ -303,6 +357,10 @@ void stableGetPermutation(const Block & block, const SortDescription & descripti return; getBlockSortPermutationImpl(block, description, IColumn::PermutationSortStability::Stable, 0, out_permutation); + +#ifndef NDEBUG + checkSortedWithPermutation(block, description, 0, out_permutation); +#endif } bool isAlreadySorted(const Block & block, const SortDescription & description) From b64700cfa8cc7c24f86963dc4cf9a59ee2efe20c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Nov 2024 13:25:55 +0100 Subject: [PATCH 255/433] Not everything at once --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index a028a79c917..9d914149b4a 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -148,7 +148,7 @@ private: struct State { QueryPipeline pipeline; - PullingAsyncPipelineExecutor executor; + PullingPipelineExecutor executor; explicit State(QueryPipeline pipeline_) : pipeline(std::move(pipeline_)) From 46a3e3e7957271988e777ef5c54bd9a2f3f0bf32 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 25 Nov 2024 12:36:44 +0000 Subject: [PATCH 256/433] Docs: Update vector search docs --- .../mergetree-family/annindexes.md | 191 ++++++++---------- 1 file changed, 85 insertions(+), 106 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index fcdc16637e6..27e457f5ec0 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,42 +1,47 @@ # Approximate Nearest Neighbor Search Indexes [experimental] -Nearest neighborhood search is the problem of finding the M closest points for a given point in an N-dimensional vector space. The most -straightforward approach to solve this problem is a brute force search where the distance between all points in the vector space and the -reference point is computed. This method guarantees perfect accuracy, but it is usually too slow for practical applications. Thus, nearest -neighborhood search problems are often solved with [approximative algorithms](https://github.com/erikbern/ann-benchmarks). Approximative -nearest neighborhood search techniques, in conjunction with [embedding -methods](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning) allow to search huge -amounts of media (pictures, songs, articles, etc.) in milliseconds. +Nearest neighborhood search is the problem of finding the M closest vectors to a given vector in an N-dimensional vector space. The most +straightforward approach to solve this problem is an exhaustive (brute-force) search which computes the distance between the reference +vector and all other points in the vector space. While method guarantees a perfectly accurate result, but it is usually too slow for +practical applications. As an alternative, [approximative algorithms](https://github.com/erikbern/ann-benchmarks) use greedy heuristics to +find the M closest vectors much faster. This allows to semantic search of picture, song, text +[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning) in milliseconds. Blogs: - [Vector Search with ClickHouse - Part 1](https://clickhouse.com/blog/vector-search-clickhouse-p1) - [Vector Search with ClickHouse - Part 2](https://clickhouse.com/blog/vector-search-clickhouse-p2) -In terms of SQL, the nearest neighborhood problem can be expressed as follows: +In terms of SQL, a nearest neighborhood search can be expressed as follows: ``` sql -SELECT * -FROM table -ORDER BY Distance(vectors, Point) +SELECT [...] +FROM table, [...] +ORDER BY DistanceFunction(vectors, reference_vector) LIMIT N ``` -`vectors` contains N-dimensional values of type [Array(Float32)](../../../sql-reference/data-types/array.md) or Array(Float64), for example -embeddings. Function `Distance` computes the distance between two vectors. Often, the Euclidean (L2) distance is chosen as distance function -but [other distance functions](/docs/en/sql-reference/functions/distance-functions.md) are also possible. `Point` is the reference point, -e.g. `(0.17, 0.33, ...)`, and `N` limits the number of search results. +where +- `DistanceFunction` computes a distance between two vectors (e.g. the + [L2Distance](../../../sql-referenc/functions/distance-functions.md#L2Distance) or + [cosineDistance](../../../sql-referenc/functions/distance-functions.md#cosineDistance)), +- `vectors` is a column of type [Array(Float64)](../../../sql-reference/data-types/array.md) or + [Array(Float32)](../../../sql-reference/data-types/array.md), or [Array(BFloat16)](../../../sql-reference/data-types/array.md), typically + storing embeddings, +- `reference_vector` is a literal of type [Array(Float64)](../../../sql-reference/data-types/array.md) or + [Array(Float32)](../../../sql-reference/data-types/array.md), or [Array(BFloat16)](../../../sql-reference/data-types/array.md), and +- `N` is a constant integer restricting the number of returned results. -This query returns the top-`N` closest points to the reference point. Parameter `N` limits the number of returned values which is useful for -situations where `MaxDistance` is difficult to determine in advance. +The query returns the `N` closest points in `vectors` to `reference_vector`. -With brute force search, the query is expensive (linear in the number of points) because the distance between all points in `vectors` and -`Point` must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation -of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer much quicker (in sub-linear time). +Exhaustive search computes the distance between `reference_vector` and all vectors in `vectors`. As such, its runtime is linear in the +number of stored vectors. Approximate search relies on special data structures (e.g. graphs, random forrests, etc.) which allow to find the +clostest vectors to a given reference vector quickly (i.e. in sub-linear time). ClickHouse provides such a data structure in the form of +"vector similarity indexes", a type of [skipping index](mergetree.md#table_engine-mergetree-data_skipping-indexes). # Creating and Using Vector Similarity Indexes -Syntax to create a vector similarity index over an [Array(Float32)](../../../sql-reference/data-types/array.md) column: +Syntax to create a vector similarity index ```sql CREATE TABLE table @@ -49,19 +54,26 @@ ENGINE = MergeTree ORDER BY id; ``` -Parameters: -- `method`: Supports currently only `hnsw`. -- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a - line between two points in Euclidean space), or `cosineDistance` (the [cosine - distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors). +:::note +USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_vector_similarity_index = 1`. +::: + +The index can be build on a column of type [Array(Float64)](../../../sql-reference/data-types/array.md), +[Array(Float32)](../../../sql-reference/data-types/array.md), or [Array(BFloat16)](../../../sql-reference/data-types/array.md). + +Index parameters: +- `method`: Currently only `hnsw` is supported. +- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance): the length of a line + between two points in Euclidean space), or `cosineDistance` (the [cosine + distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance): the angle between two non-zero vectors). - `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing vectors with reduced precision (optional, default: `bf16`) - `hnsw_max_connections_per_layer`: the number of neighbors per HNSW graph node, also known as `M` in the [HNSW - paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 32) + paper](https://doi.org/10.1109/TPAMI.2018.2889473). Optional, default: `32`. Value `0` means using the default value. - `hnsw_candidate_list_size_for_construction`: the size of the dynamic candidate list when constructing the HNSW graph, also known as - `ef_construction` in the original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473) (optional, default: 128) + `ef_construction` in the original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473). Optional, default: `128`. Value 0 means using + the default value. -Values 0 for parameters `hnsw_max_connections_per_layer` and `hnsw_candidate_list_size_for_construction` means using the default values of -these parameters. +For normalized data, `L2Distance` is usually the best choice, otherwise `cosineDistance` is recommended to compensate for scale. Example: @@ -76,53 +88,31 @@ ENGINE = MergeTree ORDER BY id; ``` -Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW -algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each point represents a vector and the edges represent -similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the -overall dataset, while still providing 99% recall. This is especially useful when working with high-dimensional vectors, that are expensive -to load and compare. The library also has several hardware-specific SIMD optimizations to accelerate further distance computations on modern -Arm (NEON and SVE) and x86 (AVX2 and AVX-512) CPUs and OS-specific optimizations to allow efficient navigation around immutable persistent -files, without loading them into RAM. - -USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_vector_similarity_index = 1`. - -Vector similarity indexes currently support two distance functions: -- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space - ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). -- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors - ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). - -Vector similarity indexes allows storing the vectors in reduced precision formats. Supported scalar kinds are `f64`, `f32`, `f16`, `bf16`, -and `i8`. If no scalar kind was specified during index creation, `bf16` is used as default. - -For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. - -:::note All arrays must have same length. To avoid errors, you can use a [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints), for example, `CONSTRAINT constraint_name_1 CHECK -length(vectors) = 256`. Also, empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported. -::: +length(vectors) = 256`. Empty `Arrays` and unspecified `Array` values in INSERT statements (i.e. default values) are not supported as well. -:::note -The vector similarity index currently does not work with per-table, non-default `index_granularity` settings (see -[here](https://github.com/ClickHouse/ClickHouse/pull/51325#issuecomment-1605920475)). If necessary, the value must be changed in config.xml. -::: +Vector similarity indexes are based on the [USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW +algorithm](https://arxiv.org/abs/1603.09320), i.e., a hierarchical graph where each node represents a vector and the edges between nodes +represent similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from +the overall dataset, while still providing 99% recall. This is especially useful when working with high-dimensional vectors which are +expensive to load and compare. USearch also utilizes SIMD to accelerate distance computations on modern x86 (AVX2 and AVX-512) and ARM (NEON +and SVE) CPUs. -Vector index creation is known to be slow. To speed the process up, index creation can be parallelized. The maximum number of threads can be -configured using server configuration -setting [max_build_vector_similarity_index_thread_pool_size](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters_max_build_vector_similarity_index_thread_pool_size). +Vector similarity indexes are built during column insertion and merge. The HNSW algorithm is known to provide slow inserts. As a result, +`INSERT` and `OPTIMIZE` statements on tables with vector similarity index will be slower than for ordinary tables. Vector similarity indexes +are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write requests. Three +additional techniques are recommended to speed up index creation: +- Index creation can be parallelized. The maximum number of threads can be configured using server setting + [max_build_vector_similarity_index_thread_pool_size](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters_max_build_vector_similarity_index_thread_pool_size). +- Index creation on newly inserted parts may be disabled using setting `materialize_skip_indexes_on_insert`. Search on such parts will fall + back to exact search but as inserted parts are typically small compared to the total table size, the performance impact is negligible. +- As parts are incrementally merged into bigger parts, and these new parts are merged into even bigger parts ("write amplification"), + vector similarity indexes are possibly build multiple times for the same vectors. To avoid that, you may suppress merges during insert + using statement [`SYSTEM STOP MERGES`](../../../sql-reference/statements/system.md), respectively start merges once all data has been + inserted using `SYSTEM START MERGES`. -ANN indexes are built during column insertion and merge. As a result, `INSERT` and `OPTIMIZE` statements will be slower than for ordinary -tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write -requests. - -:::tip -To reduce the cost of building vector similarity indexes, consider setting `materialize_skip_indexes_on_insert` which disables the -construction of skipping indexes on newly inserted parts. Search would fall back to exact search but as inserted parts are typically small -compared to the total table size, the performance impact of that would be negligible. - -ANN indexes support this type of query: +Vector similarity indexes support this type of query: ``` sql WITH [...] AS reference_vector @@ -134,44 +124,33 @@ LIMIT N SETTINGS enable_analyzer = 0; -- Temporary limitation, will be lifted ``` -:::tip -To avoid writing out large vectors, you can use [query -parameters](/docs/en/interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. - -```bash -clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" -``` -::: - To search using a different value of HNSW parameter `hnsw_candidate_list_size_for_search` (default: 256), also known as `ef_search` in the original [HNSW paper](https://doi.org/10.1109/TPAMI.2018.2889473), run the `SELECT` query with `SETTINGS hnsw_candidate_list_size_for_search = `. -**Restrictions**: Approximate algorithms used to determine the nearest neighbors require a limit, hence queries without `LIMIT` clause -cannot utilize ANN indexes. Also, ANN indexes are only used if the query has a `LIMIT` value smaller than setting -`max_limit_for_ann_queries` (default: 1 million rows). This is a safeguard to prevent large memory allocations by external libraries for -approximate neighbor search. +**Restrictions**: Approximate vector search algorithms require a limit, hence queries without `LIMIT` clause cannot utilize vector +similarity indexes. The limit must also be smaller than setting `max_limit_for_ann_queries` (default: 100). -**Differences to Skip Indexes** Similar to regular [skip indexes](https://clickhouse.com/docs/en/optimize/skipping-indexes), ANN indexes are -constructed over granules and each indexed block consists of `GRANULARITY = [N]`-many granules (`[N]` = 1 by default for normal skip -indexes). For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, -then each indexed block will contain 16384 rows. However, data structures and algorithms for approximate neighborhood search (usually -provided by external libraries) are inherently row-oriented. They store a compact representation of a set of rows and also return rows for -ANN queries. This causes some rather unintuitive differences in the way ANN indexes behave compared to normal skip indexes. +**Differences to Regular Skip Indexes** Similar to regular [skip indexes](https://clickhouse.com/docs/en/optimize/skipping-indexes), vector +similarity indexes are constructed over granules and each indexed block consists of `GRANULARITY = [N]`-many granules (`[N]` = 1 by default +for normal skip indexes). For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and +`GRANULARITY = 2`, then each indexed block will contain 16384 rows. However, data structures and algorithms for approximate neighborhood +search are inherently row-oriented. They store a compact representation of a set of rows and also return rows for vector search queries. +This causes some rather unintuitive differences in the way vector vector similarity indexes behave compared to normal skip indexes. -When a user defines an ANN index on a column, ClickHouse internally creates an ANN "sub-index" for each index block. The sub-index is "local" -in the sense that it only knows about the rows of its containing index block. In the previous example and assuming that a column has 65536 -rows, we obtain four index blocks (spanning eight granules) and an ANN sub-index for each index block. A sub-index is theoretically able to -return the rows with the N closest points within its index block directly. However, since ClickHouse loads data from disk to memory at the -granularity of granules, sub-indexes extrapolate matching rows to granule granularity. This is different from regular skip indexes which -skip data at the granularity of index blocks. +When a user defines an vector similarity index on a column, ClickHouse internally creates an vector similarity "sub-index" for each index +block. The sub-index is "local" in the sense that it only knows about the rows of its containing index block. In the previous example and +assuming that a column has 65536 rows, we obtain four index blocks (spanning eight granules) and an vector similarity sub-index for each +index block. A sub-index is theoretically able to return the rows with the N closest points within its index block directly. However, since +ClickHouse loads data from disk to memory at the granularity of granules, sub-indexes extrapolate matching rows to granule granularity. This +is different from regular skip indexes which skip data at the granularity of index blocks. -The `GRANULARITY` parameter determines how many ANN sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger ANN -sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the sub-index has a -"global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at most -`LIMIT [N]`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by performing a -brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes returns up to -`LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with both cases -equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for ANN indexes and fall -back to a smaller `GRANULARITY` values only in case of problems like excessive memory consumption of the ANN structures. If no `GRANULARITY` -was specified for ANN indexes, the default value is 100 million. +The `GRANULARITY` parameter determines how many vector similarity sub-indexes are created. Bigger `GRANULARITY` values mean fewer but larger +vector similarity sub-indexes, up to the point where a column (or a column's data part) has only a single sub-index. In that case, the +sub-index has a "global" view of all column rows and can directly return all granules of the column (part) with relevant rows (there are at +most `LIMIT [N]`-many such granules). In a second step, ClickHouse will load these granules and identify the actually best rows by +performing a brute-force distance calculation over all rows of the granules. With a small `GRANULARITY` value, each of the sub-indexes +returns up to `LIMIT N`-many granules. As a result, more granules need to be loaded and post-filtered. Note that the search accuracy is with +both cases equally good, only the processing performance differs. It is generally recommended to use a large `GRANULARITY` for vector +similarity indexes and fall back to a smaller `GRANULARITY` values only in case of problems like excessive memory consumption of the vector +similarity structures. If no `GRANULARITY` was specified for vector similarity indexes, the default value is 100 million. From 70103f11c5cf9866207ee0fc3bf9c668cd9109a9 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 25 Nov 2024 13:37:56 +0100 Subject: [PATCH 257/433] fix finalize/cancel write buffers --- src/Storages/ObjectStorage/StorageObjectStorageSink.cpp | 1 + src/Storages/StorageMemory.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 7d9b69dfc09..bd7cee407e6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -74,6 +74,7 @@ void StorageObjectStorageSink::finalizeBuffers() catch (...) { /// Stop ParallelFormattingOutputFormat correctly. + cancelBuffers(); releaseBuffers(); throw; } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 71447889d86..d798e1b4fb5 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -561,7 +561,7 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat temp_data_file.emplace(temporary_disk); auto out = std::make_unique(temp_data_file->getAbsolutePath()); copyData(*in, *out); - out.reset(); + out->finalize(); in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {}); } std::unique_ptr in_from_file{static_cast(in.release())}; From b6cc52410df340a4e0453187dcf5e7457d1252e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 25 Nov 2024 12:59:48 +0000 Subject: [PATCH 258/433] Fix spelling --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 27e457f5ec0..c5b015cddfb 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -35,8 +35,8 @@ where The query returns the `N` closest points in `vectors` to `reference_vector`. Exhaustive search computes the distance between `reference_vector` and all vectors in `vectors`. As such, its runtime is linear in the -number of stored vectors. Approximate search relies on special data structures (e.g. graphs, random forrests, etc.) which allow to find the -clostest vectors to a given reference vector quickly (i.e. in sub-linear time). ClickHouse provides such a data structure in the form of +number of stored vectors. Approximate search relies on special data structures (e.g. graphs, random forests, etc.) which allow to find the +closest vectors to a given reference vector quickly (i.e. in sub-linear time). ClickHouse provides such a data structure in the form of "vector similarity indexes", a type of [skipping index](mergetree.md#table_engine-mergetree-data_skipping-indexes). # Creating and Using Vector Similarity Indexes From 37a534325f3c49cfb608dcdfb28d15138fb4a0fc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Nov 2024 13:00:55 +0000 Subject: [PATCH 259/433] Fix WITH FILL: Invalid number of rows in Chunk --- src/Processors/Chunk.cpp | 8 ++++---- src/Processors/Transforms/FillingTransform.cpp | 16 +++++++++++++++- .../03274_with_fill_dup_sort_bug.reference | 8 ++++++++ .../0_stateless/03274_with_fill_dup_sort_bug.sql | 7 +++++++ 4 files changed, 34 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03274_with_fill_dup_sort_bug.reference create mode 100644 tests/queries/0_stateless/03274_with_fill_dup_sort_bug.sql diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 43eb0df87d4..e27762f53dc 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -62,8 +62,8 @@ void Chunk::checkNumRowsIsConsistent() { auto & column = columns[i]; if (column->size() != num_rows) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of rows in Chunk column {}: expected {}, got {}", - column->getName() + " position " + toString(i), toString(num_rows), toString(column->size())); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of rows in Chunk {} column {} at position {}: expected {}, got {}", + dumpStructure(), column->getName(), i, num_rows, column->size()); } } @@ -100,8 +100,8 @@ void Chunk::addColumn(ColumnPtr column) if (empty()) num_rows = column->size(); else if (column->size() != num_rows) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of rows in Chunk column {}, got {}", - column->getName()+ ": expected " + toString(num_rows), toString(column->size())); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of rows in Chunk {} column {}: expected {}, got {}", + dumpStructure(), column->getName(), num_rows, column->size()); columns.emplace_back(std::move(column)); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index ab782f3e521..0f2a1426c71 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -205,6 +205,20 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & return true; } +SortDescription duduplicateSortDescription(const SortDescription & sort_description) +{ + SortDescription result; + std::unordered_set unique_columns; + for (const auto & desc : sort_description) + { + const auto & [_, inserted] = unique_columns.insert(desc.column_name); + if (!inserted) + continue; + result.push_back(desc); + } + return result; +} + FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, @@ -212,7 +226,7 @@ FillingTransform::FillingTransform( InterpolateDescriptionPtr interpolate_description_, const bool use_with_fill_by_sorting_prefix_) : ISimpleTransform(header_, transformHeader(header_, fill_description_), true) - , sort_description(sort_description_) + , sort_description(duduplicateSortDescription(sort_description_)) , fill_description(fill_description_) , interpolate_description(interpolate_description_) , filling_row(fill_description_) diff --git a/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.reference b/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.reference new file mode 100644 index 00000000000..51ae4b62bec --- /dev/null +++ b/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.reference @@ -0,0 +1,8 @@ +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 diff --git a/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.sql b/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.sql new file mode 100644 index 00000000000..41d7bb84563 --- /dev/null +++ b/tests/queries/0_stateless/03274_with_fill_dup_sort_bug.sql @@ -0,0 +1,7 @@ +SELECT + 1 AS a, + 2 AS b +ORDER BY + a ASC, + 1 ASC, + b ASC WITH FILL TO 10; From 22de29a70f4dd7db92076b25d7b11b078858e429 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 14:04:05 +0100 Subject: [PATCH 260/433] stype fixes --- src/Interpreters/sortBlock.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index bdf672623da..3ebfde8b7a3 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_COLLATION; + extern const int LOGICAL_ERROR; } /// Column with description for sort @@ -285,7 +286,7 @@ void checkSortedWithPermutationImpl(size_t rows, Comparator compare, UInt64 limi if (compare(current_row, previous_row)) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Rows are not sorted with permuation, position {}, previous_row index {}, current_row index {}", i, previous_row, current_row); + "Rows are not sorted with permutation, position {}, previous_row index {}, current_row index {}", i, previous_row, current_row); } } From 26d2ac7631ed11602a57c729a8bd2de25b10120b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Nov 2024 12:41:49 +0100 Subject: [PATCH 261/433] fix stress test script --- tests/ci/stress_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 4cfe2e0a99b..cd1251871d5 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -10,7 +10,6 @@ from pathlib import Path from typing import List, Tuple from build_download_helper import download_all_deb_packages -from ci_utils import Shell from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH @@ -19,6 +18,7 @@ from pr_info import PRInfo from report import ERROR, JobReport, TestResults, read_test_results from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_utils import Shell class SensitiveFormatter(logging.Formatter): From 70c983835d191973ad1fa2efff1ad2e891e2432c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Thu, 21 Nov 2024 22:10:27 +0100 Subject: [PATCH 262/433] CI: Unit tests with praktika --- .github/workflows/pr.yaml | 901 +++++++++++++++++++++++ ci/docker/binary-builder/Dockerfile | 80 ++ ci/docker/stateless-test/Dockerfile | 3 +- ci/docker/style-test/Dockerfile | 1 + ci/jobs/build_clickhouse.py | 85 ++- ci/jobs/check_style.py | 175 +++-- ci/jobs/fast_test.py | 17 +- ci/jobs/functional_stateful_tests.py | 7 +- ci/jobs/functional_stateless_tests.py | 2 +- ci/jobs/scripts/check_style/check_cpp.sh | 4 +- ci/praktika/__main__.py | 11 +- ci/praktika/hook_html.py | 3 - ci/praktika/json.html | 2 +- ci/praktika/mangle.py | 4 +- ci/praktika/result.py | 183 ++++- ci/praktika/runner.py | 9 +- ci/praktika/utils.py | 23 +- ci/praktika/yaml_generator.py | 4 +- ci/settings/__init__.py | 0 ci/settings/definitions.py | 245 ------ ci/settings/settings.py | 13 +- ci/setup.py | 17 + ci/workflows/defs.py | 571 ++++++++++++++ ci/workflows/pull_request.py | 258 +------ tests/ci/report.py | 2 +- 25 files changed, 1975 insertions(+), 645 deletions(-) create mode 100644 .github/workflows/pr.yaml create mode 100644 ci/docker/binary-builder/Dockerfile delete mode 100644 ci/settings/__init__.py delete mode 100644 ci/settings/definitions.py create mode 100644 ci/setup.py create mode 100644 ci/workflows/defs.py diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml new file mode 100644 index 00000000000..a663b3f08ce --- /dev/null +++ b/.github/workflows/pr.yaml @@ -0,0 +1,901 @@ +# generated by praktika + +name: PR + +on: + pull_request: + branches: ['master'] + +# Cancel the previous wf run in PRs. +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + GH_TOKEN: ${{ github.token }} + +# Allow updating GH commit statuses and PR comments to post an actual job reports link +permissions: write-all + +jobs: + + config_workflow: + runs-on: [ci_services] + needs: [] + name: "Config Workflow" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Config Workflow''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Config Workflow''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + docker_builds: + runs-on: [ci_services_ebs] + needs: [config_workflow] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'RG9ja2VyIEJ1aWxkcw==') }} + name: "Docker Builds" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Docker Builds''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Docker Builds''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + style_check: + runs-on: [ci_services] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3R5bGUgQ2hlY2s=') }} + name: "Style Check" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Style Check''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Style Check''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + fast_test: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'RmFzdCB0ZXN0') }} + name: "Fast test" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Fast test''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Fast test''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_debug: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9kZWJ1Zyk=') }} + name: "Build (amd_debug)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_debug)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_debug)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_release: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9yZWxlYXNlKQ==') }} + name: "Build (amd_release)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_asan: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9hc2FuKQ==') }} + name: "Build (amd_asan)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_asan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_asan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_tsan: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF90c2FuKQ==') }} + name: "Build (amd_tsan)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_tsan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_tsan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_msan: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9tc2FuKQ==') }} + name: "Build (amd_msan)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_msan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_msan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_ubsan: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF91YnNhbik=') }} + name: "Build (amd_ubsan)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_ubsan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_ubsan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_amd_binary: + runs-on: [builder] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9iaW5hcnkp') }} + name: "Build (amd_binary)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (amd_binary)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (amd_binary)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_arm_release: + runs-on: [builder-aarch64] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFybV9yZWxlYXNlKQ==') }} + name: "Build (arm_release)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (arm_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (arm_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + build_arm_asan: + runs-on: [builder-aarch64] + needs: [config_workflow, docker_builds] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFybV9hc2FuKQ==') }} + name: "Build (arm_asan)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Build (arm_asan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Build (arm_asan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_amd_debugparallel: + runs-on: [builder] + needs: [config_workflow, docker_builds, build_amd_debug] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfZGVidWcscGFyYWxsZWwp') }} + name: "Stateless tests (amd_debug,parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (amd_debug,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (amd_debug,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_amd_debugnon_parallel: + runs-on: [func-tester] + needs: [config_workflow, docker_builds, build_amd_debug] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfZGVidWcsbm9uLXBhcmFsbGVsKQ==') }} + name: "Stateless tests (amd_debug,non-parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (amd_debug,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (amd_debug,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_amd_releaseparallel: + runs-on: [builder] + needs: [config_workflow, docker_builds, build_amd_release] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfcmVsZWFzZSxwYXJhbGxlbCk=') }} + name: "Stateless tests (amd_release,parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (amd_release,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (amd_release,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_amd_releasenon_parallel: + runs-on: [func-tester] + needs: [config_workflow, docker_builds, build_amd_release] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfcmVsZWFzZSxub24tcGFyYWxsZWwp') }} + name: "Stateless tests (amd_release,non-parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (amd_release,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (amd_release,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_arm_asanparallel: + runs-on: [builder-aarch64] + needs: [config_workflow, docker_builds, build_arm_asan] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhcm1fYXNhbixwYXJhbGxlbCk=') }} + name: "Stateless tests (arm_asan,parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (arm_asan,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (arm_asan,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateless_tests_arm_asannon_parallel: + runs-on: [func-tester-aarch64] + needs: [config_workflow, docker_builds, build_arm_asan] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhcm1fYXNhbixub24tcGFyYWxsZWwp') }} + name: "Stateless tests (arm_asan,non-parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateless tests (arm_asan,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateless tests (arm_asan,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stateful_tests_amd_releaseparallel: + runs-on: [builder] + needs: [config_workflow, docker_builds, build_amd_debug] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVmdWwgdGVzdHMgKGFtZF9yZWxlYXNlLHBhcmFsbGVsKQ==') }} + name: "Stateful tests (amd_release,parallel)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stateful tests (amd_release,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stateful tests (amd_release,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + stress_tests_arm_release: + runs-on: [func-tester-aarch64] + needs: [config_workflow, docker_builds, build_arm_release] + if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RyZXNzIHRlc3RzIChhcm1fcmVsZWFzZSk=') }} + name: "Stress tests (arm_release)" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Stress tests (arm_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Stress tests (arm_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi + + finish_workflow: + runs-on: [ci_services] + needs: [config_workflow, docker_builds, style_check, fast_test, build_amd_debug, build_amd_release, build_amd_asan, build_amd_tsan, build_amd_msan, build_amd_ubsan, build_amd_binary, build_arm_release, build_arm_asan, stateless_tests_amd_debugparallel, stateless_tests_amd_debugnon_parallel, stateless_tests_amd_releaseparallel, stateless_tests_amd_releasenon_parallel, stateless_tests_arm_asanparallel, stateless_tests_arm_asannon_parallel, stateful_tests_amd_releaseparallel, stress_tests_arm_release] + if: ${{ !cancelled() }} + name: "Finish Workflow" + outputs: + data: ${{ steps.run.outputs.DATA }} + steps: + - name: Checkout code + uses: actions/checkout@v4 + with: + ref: ${{ github.head_ref }} + + - name: Prepare env script + run: | + cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' + export PYTHONPATH=./ci:. + + cat > /tmp/praktika/workflow_config_pr.json << 'EOF' + ${{ needs.config_workflow.outputs.data }} + EOF + cat > /tmp/praktika/workflow_status.json << 'EOF' + ${{ toJson(needs) }} + EOF + ENV_SETUP_SCRIPT_EOF + + rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika + mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output + + - name: Run + id: run + run: | + . /tmp/praktika_setup_env.sh + set -o pipefail + if command -v ts &> /dev/null; then + python3 -m praktika run '''Finish Workflow''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + else + python3 -m praktika run '''Finish Workflow''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log + fi diff --git a/ci/docker/binary-builder/Dockerfile b/ci/docker/binary-builder/Dockerfile new file mode 100644 index 00000000000..e52458eab75 --- /dev/null +++ b/ci/docker/binary-builder/Dockerfile @@ -0,0 +1,80 @@ +# docker build -t clickhouse/binary-builder . +ARG FROM_TAG=latest +FROM clickhouse/fasttest:$FROM_TAG +ENV CC=clang-${LLVM_VERSION} +ENV CXX=clang++-${LLVM_VERSION} + +# If the cctools is updated, then first build it in the CI, then update here in a different commit +COPY --from=clickhouse/cctools:d9e3596e706b /cctools /cctools + +# Rust toolchain and libraries +ENV RUSTUP_HOME=/rust/rustup +ENV CARGO_HOME=/rust/cargo +ENV PATH="/rust/cargo/bin:${PATH}" +RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ + chmod 777 -R /rust && \ + rustup toolchain install nightly-2024-04-01 && \ + rustup default nightly-2024-04-01 && \ + rustup toolchain remove stable && \ + rustup component add rust-src && \ + rustup target add x86_64-unknown-linux-gnu && \ + rustup target add aarch64-unknown-linux-gnu && \ + rustup target add x86_64-apple-darwin && \ + rustup target add x86_64-unknown-freebsd && \ + rustup target add aarch64-apple-darwin && \ + rustup target add powerpc64le-unknown-linux-gnu && \ + rustup target add x86_64-unknown-linux-musl && \ + rustup target add aarch64-unknown-linux-musl && \ + rustup target add riscv64gc-unknown-linux-gnu + +# A cross-linker for RISC-V 64 (we need it, because LLVM's LLD does not work): +RUN apt-get update \ + && apt-get install software-properties-common --yes --no-install-recommends --verbose-versions + +RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ + && apt-get update \ + && apt-get install --yes \ + binutils-riscv64-linux-gnu \ + build-essential \ + python3-boto3 \ + yasm \ + zstd \ + zip \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + +# Download toolchain and SDK for Darwin +RUN curl -sL -O https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz + +# Download and install mold 2.0 for s390x build +RUN curl -Lo /tmp/mold.tar.gz "https://github.com/rui314/mold/releases/download/v2.0.0/mold-2.0.0-x86_64-linux.tar.gz" \ + && mkdir /tmp/mold \ + && tar -xzf /tmp/mold.tar.gz -C /tmp/mold \ + && cp -r /tmp/mold/mold*/* /usr \ + && rm -rf /tmp/mold \ + && rm /tmp/mold.tar.gz + +# Architecture of the image when BuildKit/buildx is used +ARG TARGETARCH +ARG NFPM_VERSION=2.20.0 + +RUN arch=${TARGETARCH:-amd64} \ + && curl -Lo /tmp/nfpm.deb "https://github.com/goreleaser/nfpm/releases/download/v${NFPM_VERSION}/nfpm_${arch}.deb" \ + && dpkg -i /tmp/nfpm.deb \ + && rm /tmp/nfpm.deb + +ARG GO_VERSION=1.19.10 +# We needed go for clickhouse-diagnostics (it is not used anymore) +RUN arch=${TARGETARCH:-amd64} \ + && curl -Lo /tmp/go.tgz "https://go.dev/dl/go${GO_VERSION}.linux-${arch}.tar.gz" \ + && tar -xzf /tmp/go.tgz -C /usr/local/ \ + && rm /tmp/go.tgz + +ENV PATH="$PATH:/usr/local/go/bin" +ENV GOPATH=/workdir/go +ENV GOCACHE=/workdir/ + +ARG CLANG_TIDY_SHA1=c191254ea00d47ade11d7170ef82fe038c213774 +RUN curl -Lo /usr/bin/clang-tidy-cache \ + "https://raw.githubusercontent.com/matus-chochlik/ctcache/$CLANG_TIDY_SHA1/clang-tidy-cache" \ + && chmod +x /usr/bin/clang-tidy-cache diff --git a/ci/docker/stateless-test/Dockerfile b/ci/docker/stateless-test/Dockerfile index 4bf5d2788cc..8e99f45f084 100644 --- a/ci/docker/stateless-test/Dockerfile +++ b/ci/docker/stateless-test/Dockerfile @@ -11,7 +11,8 @@ ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/down RUN mkdir /etc/clickhouse-server /etc/clickhouse-keeper /etc/clickhouse-client && chmod 777 /etc/clickhouse-* \ && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server && chmod 777 /var/log/clickhouse-server /var/lib/clickhouse -RUN addgroup --gid 1001 clickhouse && adduser --uid 1001 --gid 1001 --disabled-password clickhouse +RUN addgroup --gid 1000 clickhouse && adduser --uid 1000 --gid 1000 --disabled-password clickhouse +RUN addgroup --gid 1001 clickhouse2 && adduser --uid 1001 --gid 1001 --disabled-password clickhouse2 # moreutils - provides ts fo FT # expect, bzip2 - requried by FT diff --git a/ci/docker/style-test/Dockerfile b/ci/docker/style-test/Dockerfile index 165cdc3dcb1..0c4a67e76cb 100644 --- a/ci/docker/style-test/Dockerfile +++ b/ci/docker/style-test/Dockerfile @@ -6,6 +6,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ libxml2-utils \ python3-pip \ locales \ + ripgrep \ git \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* diff --git a/ci/jobs/build_clickhouse.py b/ci/jobs/build_clickhouse.py index 5ab721fb81e..473027b5779 100644 --- a/ci/jobs/build_clickhouse.py +++ b/ci/jobs/build_clickhouse.py @@ -5,6 +5,7 @@ from praktika.settings import Settings from praktika.utils import MetaClasses, Shell, Utils from ci.jobs.scripts.clickhouse_version import CHVersion +from ci.workflows.defs import CIFiles, ToolSet class JobStages(metaclass=MetaClasses.WithIter): @@ -13,6 +14,7 @@ class JobStages(metaclass=MetaClasses.WithIter): UNSHALLOW = "unshallow" BUILD = "build" PACKAGE = "package" + UNIT = "unit" def parse_args(): @@ -36,12 +38,15 @@ CMAKE_CMD = """cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA \ -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr \ -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON \ {AUX_DEFS} \ --DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 \ +-DCMAKE_C_COMPILER={COMPILER} -DCMAKE_CXX_COMPILER={COMPILER_CPP} \ -DCOMPILER_CACHE={CACHE_TYPE} -DENABLE_BUILD_PROFILING=1 {DIR}""" +# release: cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON -DSPLIT_DEBUG_SYMBOLS=ON -DBUILD_STANDALONE_KEEPER=1 -DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 .. +# binary release: cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 .. +# release coverage: cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DCMAKE_FIND_PACKAGE_NO_PACKAGE_REGISTRY=ON -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_INSTALL_SYSCONFDIR=/etc -DCMAKE_INSTALL_LOCALSTATEDIR=/var -DCMAKE_SKIP_INSTALL_ALL_DEPENDENCY=ON -DCMAKE_C_COMPILER=clang-18 -DCMAKE_CXX_COMPILER=clang++-18 -DSANITIZE_COVERAGE=1 -DBUILD_STANDALONE_KEEPER=0 -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 .. + def main(): - args = parse_args() stop_watch = Utils.Stopwatch() @@ -65,30 +70,52 @@ def main(): BUILD_TYPE = "RelWithDebInfo" SANITIZER = "" - AUX_DEFS = " -DENABLE_TESTS=0 " + AUX_DEFS = " -DENABLE_TESTS=1 " + cmake_cmd = None if "debug" in build_type: print("Build type set: debug") BUILD_TYPE = "Debug" - AUX_DEFS = " -DENABLE_TESTS=1 " + AUX_DEFS = " -DENABLE_TESTS=0 " + package_type = "debug" elif "release" in build_type: print("Build type set: release") AUX_DEFS = ( " -DENABLE_TESTS=0 -DSPLIT_DEBUG_SYMBOLS=ON -DBUILD_STANDALONE_KEEPER=1 " ) + package_type = "release" elif "asan" in build_type: print("Sanitizer set: address") SANITIZER = "address" + package_type = "asan" + elif "tsan" in build_type: + print("Sanitizer set: thread") + SANITIZER = "thread" + package_type = "tsan" + elif "msan" in build_type: + print("Sanitizer set: memory") + SANITIZER = "memory" + package_type = "msan" + elif "ubsan" in build_type: + print("Sanitizer set: undefined") + SANITIZER = "undefined" + package_type = "ubsan" + elif "binary" in build_type: + package_type = "binary" + cmake_cmd = f"cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=None -DSANITIZE= -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_CLICKHOUSE_SELF_EXTRACTING=1 -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} -DCOMPILER_CACHE=sccache -DENABLE_BUILD_PROFILING=1 {Utils.cwd()}" else: assert False - cmake_cmd = CMAKE_CMD.format( - BUILD_TYPE=BUILD_TYPE, - CACHE_TYPE=CACHE_TYPE, - SANITIZER=SANITIZER, - AUX_DEFS=AUX_DEFS, - DIR=Utils.cwd(), - ) + if not cmake_cmd: + cmake_cmd = CMAKE_CMD.format( + BUILD_TYPE=BUILD_TYPE, + CACHE_TYPE=CACHE_TYPE, + SANITIZER=SANITIZER, + AUX_DEFS=AUX_DEFS, + DIR=Utils.cwd(), + COMPILER=ToolSet.COMPILER_C, + COMPILER_CPP=ToolSet.COMPILER_CPP, + ) build_dir = f"{Settings.TEMP_DIR}/build" @@ -98,7 +125,7 @@ def main(): if res and JobStages.UNSHALLOW in stages: results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Repo Unshallow", command="git rev-parse --is-shallow-repository | grep -q true && git fetch --depth 10000 --no-tags --filter=tree:0 origin $(git rev-parse --abbrev-ref HEAD)", with_log=True, @@ -119,7 +146,7 @@ def main(): if res and JobStages.CHECKOUT_SUBMODULES in stages: Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}") results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Checkout Submodules", command=f"git submodule sync --recursive && git submodule init && git submodule update --depth 1 --recursive --jobs {min([Utils.cpu_count(), 20])}", ) @@ -128,7 +155,7 @@ def main(): if res and JobStages.CMAKE in stages: results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Cmake configuration", command=cmake_cmd, workdir=build_dir, @@ -140,7 +167,7 @@ def main(): if res and JobStages.BUILD in stages: Shell.check("sccache --show-stats") results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Build ClickHouse", command="ninja clickhouse-bundle clickhouse-odbc-bridge clickhouse-library-bridge", workdir=build_dir, @@ -149,18 +176,13 @@ def main(): ) Shell.check("sccache --show-stats") Shell.check(f"ls -l {build_dir}/programs/") + Shell.check(f"pwd") + Shell.check(f"find {build_dir} -name unit_tests_dbms") + Shell.check(f"find . -name unit_tests_dbms") res = results[-1].is_ok() - if res and JobStages.PACKAGE in stages: - if "debug" in build_type: - package_type = "debug" - elif "release" in build_type: - package_type = "release" - elif "asan" in build_type: - package_type = "asan" - else: - assert False, "TODO" - + if res and JobStages.PACKAGE in stages and "binary" not in build_type: + assert package_type if "amd" in build_type: deb_arch = "amd64" else: @@ -170,7 +192,7 @@ def main(): assert Shell.check(f"rm -f {output_dir}/*.deb") results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Build Packages", command=[ f"DESTDIR={build_dir}/root ninja programs/install", @@ -183,6 +205,17 @@ def main(): ) res = results[-1].is_ok() + if res and JobStages.UNIT in stages and (SANITIZER or "binary" in build_type): + # TODO: parallel execution + results.append( + Result.from_gtest_run( + name="Unit Tests", + unit_tests_path=CIFiles.UNIT_TESTS_BIN, + with_log=False, + ) + ) + res = results[-1].is_ok() + Result.create_from(results=results, stopwatch=stop_watch).complete_job() diff --git a/ci/jobs/check_style.py b/ci/jobs/check_style.py index d4b81abc92c..55b830b4d20 100644 --- a/ci/jobs/check_style.py +++ b/ci/jobs/check_style.py @@ -1,3 +1,4 @@ +import argparse import math import multiprocessing import os @@ -245,8 +246,18 @@ def check_file_names(files): return "" +def parse_args(): + parser = argparse.ArgumentParser(description="ClickHouse Style Check Job") + # parser.add_argument("--param", help="Optional job start stage", default=None) + parser.add_argument("--test", help="Optional test name pattern", default="") + return parser.parse_args() + + if __name__ == "__main__": results = [] + args = parse_args() + testpattern = args.test + stop_watch = Utils.Stopwatch() all_files = Utils.traverse_paths( @@ -296,87 +307,111 @@ if __name__ == "__main__": ) ) - results.append( - run_check_concurrent( - check_name="Whitespace Check", - check_function=check_whitespaces, - files=cpp_files, + testname = "Whitespace Check" + if testpattern.lower() in testname.lower(): + results.append( + run_check_concurrent( + check_name=testname, + check_function=check_whitespaces, + files=cpp_files, + ) ) - ) - results.append( - run_check_concurrent( - check_name="YamlLint Check", - check_function=check_yamllint, - files=yaml_workflow_files, + testname = "YamlLint Check" + if testpattern.lower() in testname.lower(): + results.append( + run_check_concurrent( + check_name=testname, + check_function=check_yamllint, + files=yaml_workflow_files, + ) ) - ) - results.append( - run_check_concurrent( - check_name="XmlLint Check", - check_function=check_xmllint, - files=xml_files, + testname = "XmlLint Check" + if testpattern.lower() in testname.lower(): + results.append( + run_check_concurrent( + check_name=testname, + check_function=check_xmllint, + files=xml_files, + ) ) - ) - results.append( - run_check_concurrent( - check_name="Functional Tests scripts smoke check", - check_function=check_functional_test_cases, - files=functional_test_files, + testname = "Functional Tests scripts smoke check" + if testpattern.lower() in testname.lower(): + results.append( + run_check_concurrent( + check_name=testname, + check_function=check_functional_test_cases, + files=functional_test_files, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Tests Numbers", - command=check_gaps_in_tests_numbers, - command_args=[functional_test_files], + testname = "Check Tests Numbers" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_gaps_in_tests_numbers, + command_args=[functional_test_files], + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Broken Symlinks", - command=check_broken_links, - command_kwargs={ - "path": "./", - "exclude_paths": ["contrib/", "metadata/", "programs/server/data"], - }, + testname = "Check Broken Symlinks" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_broken_links, + command_kwargs={ + "path": "./", + "exclude_paths": ["contrib/", "metadata/", "programs/server/data"], + }, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check CPP code", - command=check_cpp_code, + testname = "Check CPP code" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_cpp_code, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Submodules", - command=check_repo_submodules, + testname = "Check Submodules" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_repo_submodules, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check File Names", - command=check_file_names, - command_args=[all_files], + testname = "Check File Names" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_file_names, + command_args=[all_files], + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Many Different Things", - command=check_other, + testname = "Check Many Different Things" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_other, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Codespell", - command=check_codespell, + testname = "Check Codespell" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_codespell, + ) ) - ) - results.append( - Result.create_from_command_execution( - name="Check Aspell", - command=check_aspell, + testname = "Check Aspell" + if testpattern.lower() in testname.lower(): + results.append( + Result.from_commands_run( + name=testname, + command=check_aspell, + ) ) - ) Result.create_from(results=results, stopwatch=stop_watch).complete_job() diff --git a/ci/jobs/fast_test.py b/ci/jobs/fast_test.py index 03a4c0cd496..ad1049512c1 100644 --- a/ci/jobs/fast_test.py +++ b/ci/jobs/fast_test.py @@ -6,6 +6,7 @@ from praktika.utils import MetaClasses, Shell, Utils from ci.jobs.scripts.clickhouse_proc import ClickHouseProc from ci.jobs.scripts.functional_tests_results import FTResultsProcessor +from ci.workflows.defs import ToolSet def clone_submodules(): @@ -132,7 +133,7 @@ def main(): if res and JobStages.CHECKOUT_SUBMODULES in stages: Shell.check(f"rm -rf {build_dir} && mkdir -p {build_dir}") results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Checkout Submodules", command=clone_submodules, ) @@ -141,10 +142,12 @@ def main(): if res and JobStages.CMAKE in stages: results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Cmake configuration", - command=f"cmake {current_directory} -DCMAKE_CXX_COMPILER=clang++-18 -DCMAKE_C_COMPILER=clang-18 \ - -DCMAKE_TOOLCHAIN_FILE={current_directory}/cmake/linux/toolchain-x86_64-musl.cmake -DENABLE_LIBRARIES=0 \ + command=f"cmake {current_directory} -DCMAKE_CXX_COMPILER={ToolSet.COMPILER_CPP} \ + -DCMAKE_C_COMPILER={ToolSet.COMPILER_C} \ + -DCMAKE_TOOLCHAIN_FILE={current_directory}/cmake/linux/toolchain-x86_64-musl.cmake \ + -DENABLE_LIBRARIES=0 \ -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DENABLE_THINLTO=0 -DENABLE_NURAFT=1 -DENABLE_SIMDJSON=1 \ -DENABLE_JEMALLOC=1 -DENABLE_LIBURING=1 -DENABLE_YAML_CPP=1 -DCOMPILER_CACHE=sccache", workdir=build_dir, @@ -156,7 +159,7 @@ def main(): if res and JobStages.BUILD in stages: Shell.check("sccache --show-stats") results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Build ClickHouse", command="ninja clickhouse-bundle clickhouse-stripped", workdir=build_dir, @@ -176,7 +179,7 @@ def main(): "clickhouse-test --help", ] results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Check and Compress binary", command=commands, workdir=build_dir, @@ -195,7 +198,7 @@ def main(): update_path_ch_config, ] results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Install ClickHouse Config", command=commands, with_log=True, diff --git a/ci/jobs/functional_stateful_tests.py b/ci/jobs/functional_stateful_tests.py index 4af1ab902ae..e3d0e512b6a 100644 --- a/ci/jobs/functional_stateful_tests.py +++ b/ci/jobs/functional_stateful_tests.py @@ -1,4 +1,5 @@ import argparse +import os import time from pathlib import Path @@ -109,7 +110,7 @@ def main(): f"clickhouse-server --version", ] results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Install ClickHouse", command=commands, with_log=True ) ) @@ -153,6 +154,10 @@ def main(): stop_watch_ = Utils.Stopwatch() step_name = "Tests" print(step_name) + + # TODO: fix tests dependent on this and remove: + os.environ["CLICKHOUSE_TMP"] = "tests/queries/1_stateful" + # assert Shell.check("clickhouse-client -q \"insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')\"", verbose=True) run_test( no_parallel=no_parallel, diff --git a/ci/jobs/functional_stateless_tests.py b/ci/jobs/functional_stateless_tests.py index 676a05fbac1..ed717f0dff9 100644 --- a/ci/jobs/functional_stateless_tests.py +++ b/ci/jobs/functional_stateless_tests.py @@ -118,7 +118,7 @@ def main(): f"chmod +x /tmp/praktika/input/clickhouse-odbc-bridge", ] results.append( - Result.create_from_command_execution( + Result.from_commands_run( name="Install ClickHouse", command=commands, with_log=True ) ) diff --git a/ci/jobs/scripts/check_style/check_cpp.sh b/ci/jobs/scripts/check_style/check_cpp.sh index 2e47b253bac..68de5c87508 100755 --- a/ci/jobs/scripts/check_style/check_cpp.sh +++ b/ci/jobs/scripts/check_style/check_cpp.sh @@ -15,7 +15,7 @@ LC_ALL="en_US.UTF-8" ROOT_PATH="." EXCLUDE='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml' -EXCLUDE_DOCS='Settings\.cpp|FormatFactorySettingsDeclaration\.h' +EXCLUDE_DOCS='Settings\.cpp|FormatFactorySettings\.h' # From [1]: # But since array_to_string_internal() in array.c still loops over array @@ -85,6 +85,8 @@ EXTERN_TYPES_EXCLUDES=( CurrentMetrics::add CurrentMetrics::sub CurrentMetrics::get + CurrentMetrics::getDocumentation + CurrentMetrics::getName CurrentMetrics::set CurrentMetrics::end CurrentMetrics::Increment diff --git a/ci/praktika/__main__.py b/ci/praktika/__main__.py index 3dfdc26d69d..221a9818ffa 100644 --- a/ci/praktika/__main__.py +++ b/ci/praktika/__main__.py @@ -8,12 +8,12 @@ from praktika.yaml_generator import YamlGenerator def create_parser(): - parser = argparse.ArgumentParser(prog="python3 -m praktika") + parser = argparse.ArgumentParser(prog="praktika") subparsers = parser.add_subparsers(dest="command", help="Available subcommands") run_parser = subparsers.add_parser("run", help="Job Runner") - run_parser.add_argument("--job", help="Job Name", type=str, required=True) + run_parser.add_argument("job", help="Job Name", type=str) run_parser.add_argument( "--workflow", help="Workflow Name (required if job name is not uniq per config)", @@ -75,7 +75,8 @@ def create_parser(): return parser -if __name__ == "__main__": +def main(): + sys.path.append(".") parser = create_parser() args = parser.parse_args() @@ -120,3 +121,7 @@ if __name__ == "__main__": else: parser.print_help() sys.exit(1) + + +if __name__ == "__main__": + main() diff --git a/ci/praktika/hook_html.py b/ci/praktika/hook_html.py index 5d476fe9d2b..ed2335a640a 100644 --- a/ci/praktika/hook_html.py +++ b/ci/praktika/hook_html.py @@ -128,9 +128,6 @@ class HtmlRunnerHooks: for job in _workflow.jobs: if job.name not in skip_jobs: result = Result.generate_pending(job.name) - # Preemptively add the general job log to the result directory to ensure - # the post-job handler can upload it, even if the job is terminated unexpectedly - result.set_files([Settings.RUN_LOG]) else: result = Result.generate_skipped(job.name, job_cache_records[job.name]) results.append(result) diff --git a/ci/praktika/json.html b/ci/praktika/json.html index 717e6d39ac3..96df128f641 100644 --- a/ci/praktika/json.html +++ b/ci/praktika/json.html @@ -529,7 +529,7 @@ const columnSymbols = { name: '🗂️', - status: '🧾', + status: '✅', start_time: '🕒', duration: '⏳', info: '📝', diff --git a/ci/praktika/mangle.py b/ci/praktika/mangle.py index f94b11adad5..b7643a297cd 100644 --- a/ci/praktika/mangle.py +++ b/ci/praktika/mangle.py @@ -68,9 +68,7 @@ def _update_workflow_with_native_jobs(workflow): print(f"Enable native job [{_docker_build_job.name}] for [{workflow.name}]") aux_job = copy.deepcopy(_docker_build_job) if workflow.enable_cache: - print( - f"Add automatic digest config for [{aux_job.name}] job since cache is enabled" - ) + print(f"Add automatic digest config for [{aux_job.name}] job") docker_digest_config = Job.CacheDigestConfig() for docker_config in workflow.dockers: docker_digest_config.include_paths.append(docker_config.path) diff --git a/ci/praktika/result.py b/ci/praktika/result.py index 082807fc9f0..fb0166145a7 100644 --- a/ci/praktika/result.py +++ b/ci/praktika/result.py @@ -1,5 +1,6 @@ import dataclasses import datetime +import json import sys from pathlib import Path from typing import Any, Dict, List, Optional, Union @@ -80,12 +81,19 @@ class Result(MetaClasses.Serializable): infos += info if results and not status: for result in results: - if result.status not in (Result.Status.SUCCESS, Result.Status.FAILED): + if result.status not in ( + Result.Status.SUCCESS, + Result.Status.FAILED, + Result.Status.ERROR, + ): Utils.raise_with_error( f"Unexpected result status [{result.status}] for Result.create_from call" ) if result.status != Result.Status.SUCCESS: result_status = Result.Status.FAILED + if result.status == Result.Status.ERROR: + result_status = Result.Status.ERROR + break if results: for result in results: if result.info and with_info_from_results: @@ -166,17 +174,14 @@ class Result(MetaClasses.Serializable): return Result(**obj) def update_duration(self): - if not self.duration and self.start_time: + if self.duration: + return self + if self.start_time: self.duration = datetime.datetime.utcnow().timestamp() - self.start_time else: - if not self.duration: - print( - f"NOTE: duration is set for job [{self.name}] Result - do not update by CI" - ) - else: - print( - f"NOTE: start_time is not set for job [{self.name}] Result - do not update duration" - ) + print( + f"NOTE: start_time is not set for job [{self.name}] Result - do not update duration" + ) return self def set_timing(self, stopwatch: Utils.Stopwatch): @@ -250,7 +255,21 @@ class Result(MetaClasses.Serializable): ) @classmethod - def create_from_command_execution( + def from_gtest_run(cls, name, unit_tests_path, with_log=False): + Shell.check(f"rm {ResultTranslator.GTEST_RESULT_FILE}") + result = Result.from_commands_run( + name=name, + command=[ + f"{unit_tests_path} --gtest_output='json:{ResultTranslator.GTEST_RESULT_FILE}'" + ], + with_log=with_log, + ) + status, results, info = ResultTranslator.from_gtest() + result.set_status(status).set_results(results).set_info(info) + return result + + @classmethod + def from_commands_run( cls, name, command, @@ -507,10 +526,11 @@ class _ResultS3: # return True @classmethod - def upload_result_files_to_s3(cls, result): + def upload_result_files_to_s3(cls, result, s3_subprefix=""): + s3_subprefix = "/".join([s3_subprefix, Utils.normalize_string(result.name)]) if result.results: for result_ in result.results: - cls.upload_result_files_to_s3(result_) + cls.upload_result_files_to_s3(result_, s3_subprefix=s3_subprefix) for file in result.files: if not Path(file).is_file(): print(f"ERROR: Invalid file [{file}] in [{result.name}] - skip upload") @@ -529,7 +549,7 @@ class _ResultS3: file, upload_to_s3=True, text=is_text, - s3_subprefix=Utils.normalize_string(result.name), + s3_subprefix=s3_subprefix, ) result.links.append(file_link) if result.files: @@ -572,3 +592,138 @@ class _ResultS3: return new_status else: return None + + +class ResultTranslator: + GTEST_RESULT_FILE = "/tmp/praktika/gtest.json" + + @classmethod + def from_gtest(cls): + """The json is described by the next proto3 scheme: + (It's wrong, but that's a copy/paste from + https://google.github.io/googletest/advanced.html#generating-a-json-report) + + syntax = "proto3"; + + package googletest; + + import "google/protobuf/timestamp.proto"; + import "google/protobuf/duration.proto"; + + message UnitTest { + int32 tests = 1; + int32 failures = 2; + int32 disabled = 3; + int32 errors = 4; + google.protobuf.Timestamp timestamp = 5; + google.protobuf.Duration time = 6; + string name = 7; + repeated TestCase testsuites = 8; + } + + message TestCase { + string name = 1; + int32 tests = 2; + int32 failures = 3; + int32 disabled = 4; + int32 errors = 5; + google.protobuf.Duration time = 6; + repeated TestInfo testsuite = 7; + } + + message TestInfo { + string name = 1; + string file = 6; + int32 line = 7; + enum Status { + RUN = 0; + NOTRUN = 1; + } + Status status = 2; + google.protobuf.Duration time = 3; + string classname = 4; + message Failure { + string failures = 1; + string type = 2; + } + repeated Failure failures = 5; + }""" + + test_results = [] # type: List[Result] + + if not Path(cls.GTEST_RESULT_FILE).exists(): + print(f"ERROR: No test result file [{cls.GTEST_RESULT_FILE}]") + return ( + Result.Status.ERROR, + test_results, + f"No test result file [{cls.GTEST_RESULT_FILE}]", + ) + + with open(cls.GTEST_RESULT_FILE, "r", encoding="utf-8") as j: + report = json.load(j) + + total_counter = report["tests"] + failed_counter = report["failures"] + error_counter = report["errors"] + + description = "" + SEGFAULT = "Segmentation fault. " + SIGNAL = "Exit on signal. " + for suite in report["testsuites"]: + suite_name = suite["name"] + for test_case in suite["testsuite"]: + case_name = test_case["name"] + test_time = float(test_case["time"][:-1]) + raw_logs = None + if "failures" in test_case: + raw_logs = "" + for failure in test_case["failures"]: + raw_logs += failure[Result.Status.FAILED] + if ( + "Segmentation fault" in raw_logs # type: ignore + and SEGFAULT not in description + ): + description += SEGFAULT + if ( + "received signal SIG" in raw_logs # type: ignore + and SIGNAL not in description + ): + description += SIGNAL + if test_case["status"] == "NOTRUN": + test_status = "SKIPPED" + elif raw_logs is None: + test_status = Result.Status.SUCCESS + else: + test_status = Result.Status.FAILED + + test_results.append( + Result( + f"{suite_name}.{case_name}", + test_status, + duration=test_time, + info=raw_logs, + ) + ) + + check_status = Result.Status.SUCCESS + tests_status = Result.Status.SUCCESS + tests_time = float(report["time"][:-1]) + if failed_counter: + check_status = Result.Status.FAILED + test_status = Result.Status.FAILED + if error_counter: + check_status = Result.Status.ERROR + test_status = Result.Status.ERROR + test_results.append(Result(report["name"], tests_status, duration=tests_time)) + + if not description: + description += ( + f"fail: {failed_counter + error_counter}, " + f"passed: {total_counter - failed_counter - error_counter}" + ) + + return ( + check_status, + test_results, + description, + ) diff --git a/ci/praktika/runner.py b/ci/praktika/runner.py index 4ab7f8eeebc..30c108d5aaa 100644 --- a/ci/praktika/runner.py +++ b/ci/praktika/runner.py @@ -86,6 +86,7 @@ class Runner: print("Read GH Environment") env = _Environment.from_env() env.JOB_NAME = job.name + os.environ["JOB_NAME"] = job.name env.dump() print(env) @@ -200,13 +201,15 @@ class Runner: ResultInfo.TIMEOUT ) elif result.is_running(): - info = f"ERROR: Job terminated with an error, exit code [{exit_code}] - set status to [{Result.Status.ERROR}]" + info = f"ERROR: Job killed, exit code [{exit_code}] - set status to [{Result.Status.ERROR}]" print(info) result.set_status(Result.Status.ERROR).set_info(info) + result.set_files([Settings.RUN_LOG]) else: info = f"ERROR: Invalid status [{result.status}] for exit code [{exit_code}] - switch to [{Result.Status.ERROR}]" print(info) result.set_status(Result.Status.ERROR).set_info(info) + result.set_files([Settings.RUN_LOG]) result.dump() return exit_code @@ -257,10 +260,6 @@ class Runner: info = f"ERROR: {ResultInfo.KILLED}" print(info) result.set_info(info).set_status(Result.Status.ERROR).dump() - else: - # TODO: add setting with different ways of storing general praktika log: always, on error, never. - # now let's store it on error only - result.files = [file for file in result.files if file != Settings.RUN_LOG] result.update_duration().dump() diff --git a/ci/praktika/utils.py b/ci/praktika/utils.py index 2bcc94f2559..0a71074573e 100644 --- a/ci/praktika/utils.py +++ b/ci/praktika/utils.py @@ -227,8 +227,8 @@ class Shell: proc = subprocess.Popen( command, shell=True, - stderr=subprocess.STDOUT, stdout=subprocess.PIPE, + stderr=subprocess.PIPE, stdin=subprocess.PIPE if stdin_str else None, universal_newlines=True, start_new_session=True, # Start a new process group for signal handling @@ -248,11 +248,24 @@ class Shell: proc.stdin.write(stdin_str) proc.stdin.close() - # Process output in real-time - if proc.stdout: - for line in proc.stdout: + # Process both stdout and stderr in real-time + def stream_output(stream, output_fp): + for line in iter(stream.readline, ""): sys.stdout.write(line) - log_fp.write(line) + output_fp.write(line) + + stdout_thread = Thread( + target=stream_output, args=(proc.stdout, log_fp) + ) + stderr_thread = Thread( + target=stream_output, args=(proc.stderr, log_fp) + ) + + stdout_thread.start() + stderr_thread.start() + + stdout_thread.join() + stderr_thread.join() proc.wait() # Wait for the process to finish diff --git a/ci/praktika/yaml_generator.py b/ci/praktika/yaml_generator.py index f56715755e8..746a24bce86 100644 --- a/ci/praktika/yaml_generator.py +++ b/ci/praktika/yaml_generator.py @@ -105,9 +105,9 @@ jobs: . /tmp/praktika_setup_env.sh set -o pipefail if command -v ts &> /dev/null; then - python3 -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log + python3 -m praktika run '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log else - python3 -m praktika run --job '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& tee /tmp/praktika/praktika_run.log + python3 -m praktika run '''{JOB_NAME}''' --workflow "{WORKFLOW_NAME}" --ci |& tee /tmp/praktika/praktika_run.log fi {UPLOADS_GITHUB}\ """ diff --git a/ci/settings/__init__.py b/ci/settings/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/ci/settings/definitions.py b/ci/settings/definitions.py deleted file mode 100644 index ced1289b950..00000000000 --- a/ci/settings/definitions.py +++ /dev/null @@ -1,245 +0,0 @@ -from praktika import Docker, Secret - -S3_BUCKET_NAME = "clickhouse-builds" -S3_BUCKET_HTTP_ENDPOINT = "clickhouse-builds.s3.amazonaws.com" - - -class RunnerLabels: - CI_SERVICES = "ci_services" - CI_SERVICES_EBS = "ci_services_ebs" - BUILDER_AMD = "builder" - BUILDER_ARM = "builder-aarch64" - FUNC_TESTER_AMD = "func-tester" - FUNC_TESTER_ARM = "func-tester-aarch64" - - -BASE_BRANCH = "master" - -azure_secret = Secret.Config( - name="azure_connection_string", - type=Secret.Type.AWS_SSM_VAR, -) - -SECRETS = [ - Secret.Config( - name="dockerhub_robot_password", - type=Secret.Type.AWS_SSM_VAR, - ), - azure_secret, - # Secret.Config( - # name="woolenwolf_gh_app.clickhouse-app-id", - # type=Secret.Type.AWS_SSM_SECRET, - # ), - # Secret.Config( - # name="woolenwolf_gh_app.clickhouse-app-key", - # type=Secret.Type.AWS_SSM_SECRET, - # ), -] - -DOCKERS = [ - # Docker.Config( - # name="clickhouse/binary-builder", - # path="./ci/docker/packager/binary-builder", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/cctools", - # path="./ci/docker/packager/cctools", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/test-old-centos", - # path="./ci/docker/test/compatibility/centos", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/test-old-ubuntu", - # path="./ci/docker/test/compatibility/ubuntu", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/test-util", - # path="./ci/docker/test/util", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/integration-test", - # path="./ci/docker/test/integration/base", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/fuzzer", - # path="./ci/docker/test/fuzzer", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/performance-comparison", - # path="./ci/docker/test/performance-comparison", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - Docker.Config( - name="clickhouse/fasttest", - path="./ci/docker/fasttest", - platforms=Docker.Platforms.arm_amd, - depends_on=[], - ), - # Docker.Config( - # name="clickhouse/test-base", - # path="./ci/docker/test/base", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-util"], - # ), - # Docker.Config( - # name="clickhouse/clickbench", - # path="./ci/docker/test/clickbench", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/keeper-jepsen-test", - # path="./ci/docker/test/keeper-jepsen", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/server-jepsen-test", - # path="./ci/docker/test/server-jepsen", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/sqllogic-test", - # path="./ci/docker/test/sqllogic", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/sqltest", - # path="./ci/docker/test/sqltest", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - Docker.Config( - name="clickhouse/stateless-test", - path="./ci/docker/stateless-test", - platforms=Docker.Platforms.arm_amd, - depends_on=[], - ), - Docker.Config( - name="clickhouse/stateful-test", - path="./ci/docker/stateful-test", - platforms=Docker.Platforms.arm_amd, - depends_on=["clickhouse/stateless-test"], - ), - # Docker.Config( - # name="clickhouse/stress-test", - # path="./ci/docker/test/stress", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/stateful-test"], - # ), - # Docker.Config( - # name="clickhouse/unit-test", - # path="./ci/docker/test/unit", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - # Docker.Config( - # name="clickhouse/integration-tests-runner", - # path="./ci/docker/test/integration/runner", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), - Docker.Config( - name="clickhouse/style-test", - path="./ci/docker/style-test", - platforms=Docker.Platforms.arm_amd, - depends_on=[], - ), - # Docker.Config( - # name="clickhouse/docs-builder", - # path="./ci/docker/docs/builder", - # platforms=Docker.Platforms.arm_amd, - # depends_on=["clickhouse/test-base"], - # ), -] - -# TODO: -# "docker/test/integration/s3_proxy": { -# "name": "clickhouse/s3-proxy", -# "dependent": [] -# }, -# "docker/test/integration/resolver": { -# "name": "clickhouse/python-bottle", -# "dependent": [] -# }, -# "docker/test/integration/helper_container": { -# "name": "clickhouse/integration-helper", -# "dependent": [] -# }, -# "docker/test/integration/mysql_golang_client": { -# "name": "clickhouse/mysql-golang-client", -# "dependent": [] -# }, -# "docker/test/integration/dotnet_client": { -# "name": "clickhouse/dotnet-client", -# "dependent": [] -# }, -# "docker/test/integration/mysql_java_client": { -# "name": "clickhouse/mysql-java-client", -# "dependent": [] -# }, -# "docker/test/integration/mysql_js_client": { -# "name": "clickhouse/mysql-js-client", -# "dependent": [] -# }, -# "docker/test/integration/mysql_php_client": { -# "name": "clickhouse/mysql-php-client", -# "dependent": [] -# }, -# "docker/test/integration/postgresql_java_client": { -# "name": "clickhouse/postgresql-java-client", -# "dependent": [] -# }, -# "docker/test/integration/kerberos_kdc": { -# "only_amd64": true, -# "name": "clickhouse/kerberos-kdc", -# "dependent": [] -# }, -# "docker/test/integration/kerberized_hadoop": { -# "only_amd64": true, -# "name": "clickhouse/kerberized-hadoop", -# "dependent": [] -# }, -# "docker/test/sqlancer": { -# "name": "clickhouse/sqlancer-test", -# "dependent": [] -# }, -# "docker/test/install/deb": { -# "name": "clickhouse/install-deb-test", -# "dependent": [] -# }, -# "docker/test/install/rpm": { -# "name": "clickhouse/install-rpm-test", -# "dependent": [] -# }, -# "docker/test/integration/nginx_dav": { -# "name": "clickhouse/nginx-dav", -# "dependent": [] -# } - - -class JobNames: - STYLE_CHECK = "Style Check" - FAST_TEST = "Fast test" - BUILD = "Build" - STATELESS = "Stateless tests" - STATEFUL = "Stateful tests" - STRESS = "Stress tests" diff --git a/ci/settings/settings.py b/ci/settings/settings.py index 0f3b1efcee0..2b274d266a2 100644 --- a/ci/settings/settings.py +++ b/ci/settings/settings.py @@ -1,14 +1,13 @@ -from ci.settings.definitions import ( - S3_BUCKET_HTTP_ENDPOINT, - S3_BUCKET_NAME, - RunnerLabels, -) +# aux settings: +S3_BUCKET_NAME = "clickhouse-builds" +S3_BUCKET_HTTP_ENDPOINT = "clickhouse-builds.s3.amazonaws.com" +# praktika settings: MAIN_BRANCH = "master" S3_ARTIFACT_PATH = f"{S3_BUCKET_NAME}/artifacts" -CI_CONFIG_RUNS_ON = [RunnerLabels.CI_SERVICES] -DOCKER_BUILD_RUNS_ON = [RunnerLabels.CI_SERVICES_EBS] +CI_CONFIG_RUNS_ON = ["ci_services"] +DOCKER_BUILD_RUNS_ON = ["ci_services_ebs"] CACHE_S3_PATH = f"{S3_BUCKET_NAME}/ci_ch_cache" HTML_S3_PATH = f"{S3_BUCKET_NAME}/reports" S3_BUCKET_TO_HTTP_ENDPOINT = {S3_BUCKET_NAME: S3_BUCKET_HTTP_ENDPOINT} diff --git a/ci/setup.py b/ci/setup.py new file mode 100644 index 00000000000..55e21f75227 --- /dev/null +++ b/ci/setup.py @@ -0,0 +1,17 @@ +from setuptools import find_packages, setup + +setup( + name="praktika", + version="0.1", + packages=find_packages(), + url="https://github.com/ClickHouse/praktika", + license="Apache 2.0", + author="Max Kainov", + author_email="max.kainov@clickhouse.com", + description="CI Infrastructure Toolbox", + entry_points={ + "console_scripts": [ + "praktika=praktika.__main__:main", + ] + }, +) diff --git a/ci/workflows/defs.py b/ci/workflows/defs.py new file mode 100644 index 00000000000..6aadc875512 --- /dev/null +++ b/ci/workflows/defs.py @@ -0,0 +1,571 @@ +from praktika import Artifact, Docker, Job, Secret +from praktika.settings import Settings + + +class RunnerLabels: + CI_SERVICES = "ci_services" + CI_SERVICES_EBS = "ci_services_ebs" + BUILDER_AMD = "builder" + BUILDER_ARM = "builder-aarch64" + FUNC_TESTER_AMD = "func-tester" + FUNC_TESTER_ARM = "func-tester-aarch64" + + +class CIFiles: + UNIT_TESTS_RESULTS = "/tmp/praktika/output/unit_tests_result.json" + UNIT_TESTS_BIN = "/tmp/praktika/build/src/unit_tests_dbms" + + +BASE_BRANCH = "master" + +azure_secret = Secret.Config( + name="azure_connection_string", + type=Secret.Type.AWS_SSM_VAR, +) + +SECRETS = [ + Secret.Config( + name="dockerhub_robot_password", + type=Secret.Type.AWS_SSM_VAR, + ), + azure_secret, + # Secret.Config( + # name="woolenwolf_gh_app.clickhouse-app-id", + # type=Secret.Type.AWS_SSM_SECRET, + # ), + # Secret.Config( + # name="woolenwolf_gh_app.clickhouse-app-key", + # type=Secret.Type.AWS_SSM_SECRET, + # ), +] + +DOCKERS = [ + Docker.Config( + name="clickhouse/binary-builder", + path="./ci/docker/binary-builder", + platforms=Docker.Platforms.arm_amd, + depends_on=["clickhouse/fasttest"], + ), + # Docker.Config( + # name="clickhouse/cctools", + # path="./ci/docker/packager/cctools", + # platforms=Docker.Platforms.arm_amd, + # depends_on=[], + # ), + # Docker.Config( + # name="clickhouse/test-old-centos", + # path="./ci/docker/test/compatibility/centos", + # platforms=Docker.Platforms.arm_amd, + # depends_on=[], + # ), + # Docker.Config( + # name="clickhouse/test-old-ubuntu", + # path="./ci/docker/test/compatibility/ubuntu", + # platforms=Docker.Platforms.arm_amd, + # depends_on=[], + # ), + # Docker.Config( + # name="clickhouse/test-util", + # path="./ci/docker/test/util", + # platforms=Docker.Platforms.arm_amd, + # depends_on=[], + # ), + # Docker.Config( + # name="clickhouse/integration-test", + # path="./ci/docker/test/integration/base", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/fuzzer", + # path="./ci/docker/test/fuzzer", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/performance-comparison", + # path="./ci/docker/test/performance-comparison", + # platforms=Docker.Platforms.arm_amd, + # depends_on=[], + # ), + Docker.Config( + name="clickhouse/fasttest", + path="./ci/docker/fasttest", + platforms=Docker.Platforms.arm_amd, + depends_on=[], + ), + # Docker.Config( + # name="clickhouse/test-base", + # path="./ci/docker/test/base", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-util"], + # ), + # Docker.Config( + # name="clickhouse/clickbench", + # path="./ci/docker/test/clickbench", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/keeper-jepsen-test", + # path="./ci/docker/test/keeper-jepsen", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/server-jepsen-test", + # path="./ci/docker/test/server-jepsen", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/sqllogic-test", + # path="./ci/docker/test/sqllogic", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/sqltest", + # path="./ci/docker/test/sqltest", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + Docker.Config( + name="clickhouse/stateless-test", + path="./ci/docker/stateless-test", + platforms=Docker.Platforms.arm_amd, + depends_on=[], + ), + Docker.Config( + name="clickhouse/stateful-test", + path="./ci/docker/stateful-test", + platforms=Docker.Platforms.arm_amd, + depends_on=["clickhouse/stateless-test"], + ), + # Docker.Config( + # name="clickhouse/stress-test", + # path="./ci/docker/test/stress", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/stateful-test"], + # ), + # Docker.Config( + # name="clickhouse/unit-test", + # path="./ci/docker/test/unit", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + # Docker.Config( + # name="clickhouse/integration-tests-runner", + # path="./ci/docker/test/integration/runner", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), + Docker.Config( + name="clickhouse/style-test", + path="./ci/docker/style-test", + platforms=Docker.Platforms.arm_amd, + depends_on=[], + ), + # Docker.Config( + # name="clickhouse/docs-builder", + # path="./ci/docker/docs/builder", + # platforms=Docker.Platforms.arm_amd, + # depends_on=["clickhouse/test-base"], + # ), +] + +# TODO: +# "docker/test/integration/s3_proxy": { +# "name": "clickhouse/s3-proxy", +# "dependent": [] +# }, +# "docker/test/integration/resolver": { +# "name": "clickhouse/python-bottle", +# "dependent": [] +# }, +# "docker/test/integration/helper_container": { +# "name": "clickhouse/integration-helper", +# "dependent": [] +# }, +# "docker/test/integration/mysql_golang_client": { +# "name": "clickhouse/mysql-golang-client", +# "dependent": [] +# }, +# "docker/test/integration/dotnet_client": { +# "name": "clickhouse/dotnet-client", +# "dependent": [] +# }, +# "docker/test/integration/mysql_java_client": { +# "name": "clickhouse/mysql-java-client", +# "dependent": [] +# }, +# "docker/test/integration/mysql_js_client": { +# "name": "clickhouse/mysql-js-client", +# "dependent": [] +# }, +# "docker/test/integration/mysql_php_client": { +# "name": "clickhouse/mysql-php-client", +# "dependent": [] +# }, +# "docker/test/integration/postgresql_java_client": { +# "name": "clickhouse/postgresql-java-client", +# "dependent": [] +# }, +# "docker/test/integration/kerberos_kdc": { +# "only_amd64": true, +# "name": "clickhouse/kerberos-kdc", +# "dependent": [] +# }, +# "docker/test/integration/kerberized_hadoop": { +# "only_amd64": true, +# "name": "clickhouse/kerberized-hadoop", +# "dependent": [] +# }, +# "docker/test/sqlancer": { +# "name": "clickhouse/sqlancer-test", +# "dependent": [] +# }, +# "docker/test/install/deb": { +# "name": "clickhouse/install-deb-test", +# "dependent": [] +# }, +# "docker/test/install/rpm": { +# "name": "clickhouse/install-rpm-test", +# "dependent": [] +# }, +# "docker/test/integration/nginx_dav": { +# "name": "clickhouse/nginx-dav", +# "dependent": [] +# } + + +class JobNames: + STYLE_CHECK = "Style Check" + FAST_TEST = "Fast test" + BUILD = "Build" + STATELESS = "Stateless tests" + STATEFUL = "Stateful tests" + STRESS = "Stress tests" + + +class ToolSet: + COMPILER_C = "clang-19" + COMPILER_CPP = "clang++-19" + + +class ArtifactNames: + CH_AMD_DEBUG = "CH_AMD_DEBUG" + CH_AMD_RELEASE = "CH_AMD_RELEASE" + CH_AMD_ASAN = "CH_AMD_ASAN" + CH_AMD_TSAN = "CH_AMD_TSAN" + CH_AMD_MSAN = "CH_AMD_MSAN" + CH_AMD_UBSAN = "CH_AMD_UBSAN" + CH_AMD_BINARY = "CH_AMD_BINARY" + CH_ARM_RELEASE = "CH_ARM_RELEASE" + CH_ARM_ASAN = "CH_ARM_ASAN" + + CH_ODBC_B_AMD_DEBUG = "CH_ODBC_B_AMD_DEBUG" + CH_ODBC_B_AMD_RELEASE = "CH_ODBC_B_AMD_RELEASE" + CH_ODBC_B_AMD_ASAN = "CH_ODBC_B_AMD_ASAN" + CH_ODBC_B_AMD_TSAN = "CH_ODBC_B_AMD_TSAN" + CH_ODBC_B_AMD_MSAN = "CH_ODBC_B_AMD_MSAN" + CH_ODBC_B_AMD_UBSAN = "CH_ODBC_B_AMD_UBSAN" + CH_ODBC_B_ARM_RELEASE = "CH_ODBC_B_ARM_RELEASE" + CH_ODBC_B_ARM_ASAN = "CH_ODBC_B_ARM_ASAN" + + UNITTEST_AMD_ASAN = "UNITTEST_AMD_ASAN" + UNITTEST_AMD_TSAN = "UNITTEST_AMD_TSAN" + UNITTEST_AMD_MSAN = "UNITTEST_AMD_MSAN" + UNITTEST_AMD_UBSAN = "UNITTEST_AMD_UBSAN" + UNITTEST_AMD_BINARY = "UNITTEST_AMD_BINARY" + + DEB_AMD_DEBUG = "DEB_AMD_DEBUG" + DEB_AMD_RELEASE = "DEB_AMD_RELEASE" + DEB_AMD_ASAN = "DEB_AMD_ASAN" + DEB_AMD_TSAN = "DEB_AMD_TSAN" + DEB_AMD_MSAM = "DEB_AMD_MSAM" + DEB_AMD_UBSAN = "DEB_AMD_UBSAN" + DEB_ARM_RELEASE = "DEB_ARM_RELEASE" + DEB_ARM_ASAN = "DEB_ARM_ASAN" + + +ARTIFACTS = [ + *Artifact.Config( + name="...", + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", + ).parametrize( + names=[ + ArtifactNames.CH_AMD_DEBUG, + ArtifactNames.CH_AMD_RELEASE, + ArtifactNames.CH_AMD_ASAN, + ArtifactNames.CH_AMD_TSAN, + ArtifactNames.CH_AMD_MSAN, + ArtifactNames.CH_AMD_UBSAN, + ArtifactNames.CH_AMD_BINARY, + ArtifactNames.CH_ARM_RELEASE, + ArtifactNames.CH_ARM_ASAN, + ] + ), + *Artifact.Config( + name="...", + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/build/programs/clickhouse-odbc-bridge", + ).parametrize( + names=[ + ArtifactNames.CH_ODBC_B_AMD_DEBUG, + ArtifactNames.CH_ODBC_B_AMD_ASAN, + ArtifactNames.CH_ODBC_B_AMD_TSAN, + ArtifactNames.CH_ODBC_B_AMD_MSAN, + ArtifactNames.CH_ODBC_B_AMD_UBSAN, + ArtifactNames.CH_ODBC_B_AMD_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_ASAN, + ] + ), + # *Artifact.Config( + # name="...", + # type=Artifact.Type.S3, + # path=f"{Settings.TEMP_DIR}/build/src/unit_tests_dbms", + # ).parametrize( + # names=[ + # ArtifactNames.UNITTEST_AMD_BINARY, + # ArtifactNames.UNITTEST_AMD_ASAN, + # ArtifactNames.UNITTEST_AMD_TSAN, + # ArtifactNames.UNITTEST_AMD_MSAN, + # ArtifactNames.UNITTEST_AMD_UBSAN, + # ] + # ), + *Artifact.Config( + name="*", + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/output/*.deb", + ).parametrize( + names=[ + ArtifactNames.DEB_AMD_DEBUG, + ArtifactNames.DEB_AMD_ASAN, + ArtifactNames.DEB_AMD_TSAN, + ArtifactNames.DEB_AMD_MSAM, + ArtifactNames.DEB_AMD_UBSAN, + ] + ), + Artifact.Config( + name=ArtifactNames.DEB_AMD_RELEASE, + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/output/*.deb", + ), + Artifact.Config( + name=ArtifactNames.DEB_ARM_RELEASE, + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/output/*.deb", + ), + Artifact.Config( + name=ArtifactNames.DEB_ARM_ASAN, + type=Artifact.Type.S3, + path=f"{Settings.TEMP_DIR}/output/*.deb", + ), +] + + +class Jobs: + style_check_job = Job.Config( + name=JobNames.STYLE_CHECK, + runs_on=[RunnerLabels.CI_SERVICES], + command="python3 ./ci/jobs/check_style.py", + run_in_docker="clickhouse/style-test", + ) + + fast_test_job = Job.Config( + name=JobNames.FAST_TEST, + runs_on=[RunnerLabels.BUILDER_AMD], + command="python3 ./ci/jobs/fast_test.py", + run_in_docker="clickhouse/fasttest", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/fast_test.py", + "./tests/queries/0_stateless/", + "./src", + ], + ), + ) + + build_jobs = Job.Config( + name=JobNames.BUILD, + runs_on=["...from params..."], + requires=[], + command="python3 ./ci/jobs/build_clickhouse.py --build-type {PARAMETER}", + run_in_docker="clickhouse/binary-builder", + timeout=3600 * 2, + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./src", + "./contrib/", + "./CMakeLists.txt", + "./PreLoad.cmake", + "./cmake", + "./base", + "./programs", + "./docker/packager/packager", + "./rust", + "./tests/ci/version_helper.py", + "./ci/jobs/build_clickhouse.py", + ], + ), + ).parametrize( + parameter=[ + "amd_debug", + "amd_release", + "amd_asan", + "amd_tsan", + "amd_msan", + "amd_ubsan", + "amd_binary", + "arm_release", + "arm_asan", + ], + provides=[ + [ + ArtifactNames.CH_AMD_DEBUG, + ArtifactNames.DEB_AMD_DEBUG, + ArtifactNames.CH_ODBC_B_AMD_DEBUG, + ], + [ + ArtifactNames.CH_AMD_RELEASE, + ArtifactNames.DEB_AMD_RELEASE, + ArtifactNames.CH_ODBC_B_AMD_RELEASE, + ], + [ + ArtifactNames.CH_AMD_ASAN, + ArtifactNames.DEB_AMD_ASAN, + ArtifactNames.CH_ODBC_B_AMD_ASAN, + # ArtifactNames.UNITTEST_AMD_ASAN, + ], + [ + ArtifactNames.CH_AMD_TSAN, + ArtifactNames.DEB_AMD_TSAN, + ArtifactNames.CH_ODBC_B_AMD_TSAN, + # ArtifactNames.UNITTEST_AMD_TSAN, + ], + [ + ArtifactNames.CH_AMD_MSAN, + ArtifactNames.DEB_AMD_MSAM, + ArtifactNames.CH_ODBC_B_AMD_MSAN, + # ArtifactNames.UNITTEST_AMD_MSAN, + ], + [ + ArtifactNames.CH_AMD_UBSAN, + ArtifactNames.DEB_AMD_UBSAN, + ArtifactNames.CH_ODBC_B_AMD_UBSAN, + # ArtifactNames.UNITTEST_AMD_UBSAN, + ], + [ + ArtifactNames.CH_AMD_BINARY, + # ArtifactNames.UNITTEST_AMD_BINARY, + ], + [ + ArtifactNames.CH_ARM_RELEASE, + ArtifactNames.DEB_ARM_RELEASE, + ArtifactNames.CH_ODBC_B_ARM_RELEASE, + ], + [ + ArtifactNames.CH_ARM_ASAN, + ArtifactNames.DEB_ARM_ASAN, + ArtifactNames.CH_ODBC_B_ARM_ASAN, + ], + ], + runs_on=[ + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.BUILDER_ARM], + [RunnerLabels.BUILDER_ARM], + ], + ) + + stateless_tests_jobs = Job.Config( + name=JobNames.STATELESS, + runs_on=[RunnerLabels.BUILDER_AMD], + command="python3 ./ci/jobs/functional_stateless_tests.py --test-options {PARAMETER}", + # many tests expect to see "/var/lib/clickhouse" in various output lines - add mount for now, consider creating this dir in docker file + run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/functional_stateless_tests.py", + ], + ), + ).parametrize( + parameter=[ + "amd_debug,parallel", + "amd_debug,non-parallel", + "amd_release,parallel", + "amd_release,non-parallel", + "arm_asan,parallel", + "arm_asan,non-parallel", + ], + runs_on=[ + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.FUNC_TESTER_AMD], + [RunnerLabels.BUILDER_AMD], + [RunnerLabels.FUNC_TESTER_AMD], + [RunnerLabels.BUILDER_ARM], + [RunnerLabels.FUNC_TESTER_ARM], + ], + requires=[ + [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], + [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], + [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], + [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], + [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], + [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], + ], + ) + + stateful_tests_jobs = Job.Config( + name=JobNames.STATEFUL, + runs_on=[RunnerLabels.BUILDER_AMD], + command="python3 ./ci/jobs/functional_stateful_tests.py --test-options {PARAMETER}", + # many tests expect to see "/var/lib/clickhouse" + # some tests expect to see "/var/log/clickhouse" + run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/functional_stateful_tests.py", + ], + ), + ).parametrize( + parameter=[ + "amd_release,parallel", + ], + runs_on=[ + [RunnerLabels.BUILDER_AMD], + ], + requires=[ + [ArtifactNames.CH_AMD_DEBUG], + ], + ) + + # TODO: refactor job to be aligned with praktika style (remove wrappers, run in docker) + stress_test_jobs = Job.Config( + name=JobNames.STRESS, + runs_on=[RunnerLabels.BUILDER_ARM], + command="python3 ./tests/ci/stress_check.py {PARAMETER}", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./ci/jobs/functional_stateful_tests.py", + ], + ), + ).parametrize( + parameter=[ + "arm_release", + ], + runs_on=[ + [RunnerLabels.FUNC_TESTER_ARM], + ], + requires=[ + [ArtifactNames.DEB_ARM_RELEASE], + ], + ) diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 1d1bcca9fa7..3bcab79470b 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -1,250 +1,20 @@ -from praktika import Artifact, Job, Workflow -from praktika.settings import Settings - -from ci.settings.definitions import ( - BASE_BRANCH, - DOCKERS, - SECRETS, - JobNames, - RunnerLabels, -) - - -class ArtifactNames: - CH_AMD_DEBUG = "CH_AMD_DEBUG" - CH_AMD_RELEASE = "CH_AMD_RELEASE" - CH_ARM_RELEASE = "CH_ARM_RELEASE" - CH_ARM_ASAN = "CH_ARM_ASAN" - - CH_ODBC_B_AMD_DEBUG = "CH_ODBC_B_AMD_DEBUG" - CH_ODBC_B_AMD_RELEASE = "CH_ODBC_B_AMD_RELEASE" - CH_ODBC_B_ARM_RELEASE = "CH_ODBC_B_ARM_RELEASE" - CH_ODBC_B_ARM_ASAN = "CH_ODBC_B_ARM_ASAN" - - DEB_AMD_DEBUG = "DEB_AMD_DEBUG" - DEB_AMD_RELEASE = "DEB_AMD_RELEASE" - DEB_ARM_RELEASE = "DEB_ARM_RELEASE" - DEB_ARM_ASAN = "DEB_ARM_ASAN" - - -style_check_job = Job.Config( - name=JobNames.STYLE_CHECK, - runs_on=[RunnerLabels.CI_SERVICES], - command="python3 ./ci/jobs/check_style.py", - run_in_docker="clickhouse/style-test", -) - -fast_test_job = Job.Config( - name=JobNames.FAST_TEST, - runs_on=[RunnerLabels.BUILDER_AMD], - command="python3 ./ci/jobs/fast_test.py", - run_in_docker="clickhouse/fasttest", - digest_config=Job.CacheDigestConfig( - include_paths=[ - "./ci/jobs/fast_test.py", - "./tests/queries/0_stateless/", - "./src", - ], - ), -) - -build_jobs = Job.Config( - name=JobNames.BUILD, - runs_on=["...from params..."], - requires=[], - command="python3 ./ci/jobs/build_clickhouse.py --build-type {PARAMETER}", - run_in_docker="clickhouse/fasttest", - timeout=3600 * 2, - digest_config=Job.CacheDigestConfig( - include_paths=[ - "./src", - "./contrib/", - "./CMakeLists.txt", - "./PreLoad.cmake", - "./cmake", - "./base", - "./programs", - "./docker/packager/packager", - "./rust", - "./tests/ci/version_helper.py", - "./ci/jobs/build_clickhouse.py", - ], - ), -).parametrize( - parameter=["amd_debug", "amd_release", "arm_release", "arm_asan"], - provides=[ - [ - ArtifactNames.CH_AMD_DEBUG, - ArtifactNames.DEB_AMD_DEBUG, - ArtifactNames.CH_ODBC_B_AMD_DEBUG, - ], - [ - ArtifactNames.CH_AMD_RELEASE, - ArtifactNames.DEB_AMD_RELEASE, - ArtifactNames.CH_ODBC_B_AMD_RELEASE, - ], - [ - ArtifactNames.CH_ARM_RELEASE, - ArtifactNames.DEB_ARM_RELEASE, - ArtifactNames.CH_ODBC_B_ARM_RELEASE, - ], - [ - ArtifactNames.CH_ARM_ASAN, - ArtifactNames.DEB_ARM_ASAN, - ArtifactNames.CH_ODBC_B_ARM_ASAN, - ], - ], - runs_on=[ - [RunnerLabels.BUILDER_AMD], - [RunnerLabels.BUILDER_AMD], - [RunnerLabels.BUILDER_ARM], - [RunnerLabels.BUILDER_ARM], - ], -) - -stateless_tests_jobs = Job.Config( - name=JobNames.STATELESS, - runs_on=[RunnerLabels.BUILDER_AMD], - command="python3 ./ci/jobs/functional_stateless_tests.py --test-options {PARAMETER}", - # many tests expect to see "/var/lib/clickhouse" in various output lines - add mount for now, consider creating this dir in docker file - run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined", - digest_config=Job.CacheDigestConfig( - include_paths=[ - "./ci/jobs/functional_stateless_tests.py", - ], - ), -).parametrize( - parameter=[ - "amd_debug,parallel", - "amd_debug,non-parallel", - "amd_release,parallel", - "amd_release,non-parallel", - "arm_asan,parallel", - "arm_asan,non-parallel", - ], - runs_on=[ - [RunnerLabels.BUILDER_AMD], - [RunnerLabels.FUNC_TESTER_AMD], - [RunnerLabels.BUILDER_AMD], - [RunnerLabels.FUNC_TESTER_AMD], - [RunnerLabels.BUILDER_ARM], - [RunnerLabels.FUNC_TESTER_ARM], - ], - requires=[ - [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], - [ArtifactNames.CH_AMD_DEBUG, ArtifactNames.CH_ODBC_B_AMD_DEBUG], - [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], - [ArtifactNames.CH_AMD_RELEASE, ArtifactNames.CH_ODBC_B_AMD_RELEASE], - [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], - [ArtifactNames.CH_ARM_ASAN, ArtifactNames.CH_ODBC_B_ARM_ASAN], - ], -) - -stateful_tests_jobs = Job.Config( - name=JobNames.STATEFUL, - runs_on=[RunnerLabels.BUILDER_AMD], - command="python3 ./ci/jobs/functional_stateful_tests.py --test-options {PARAMETER}", - # many tests expect to see "/var/lib/clickhouse" - # some tests expect to see "/var/log/clickhouse" - run_in_docker="clickhouse/stateless-test+--security-opt seccomp=unconfined", - digest_config=Job.CacheDigestConfig( - include_paths=[ - "./ci/jobs/functional_stateful_tests.py", - ], - ), -).parametrize( - parameter=[ - "amd_release,parallel", - ], - runs_on=[ - [RunnerLabels.BUILDER_AMD], - ], - requires=[ - [ArtifactNames.CH_AMD_DEBUG], - ], -) - -# TODO: refactor job to be aligned with praktika style (remove wrappers, run in docker) -stress_test_jobs = Job.Config( - name=JobNames.STRESS, - runs_on=[RunnerLabels.BUILDER_ARM], - command="python3 ./tests/ci/stress_check.py {PARAMETER}", - digest_config=Job.CacheDigestConfig( - include_paths=[ - "./ci/jobs/functional_stateful_tests.py", - ], - ), -).parametrize( - parameter=[ - "arm_release", - ], - runs_on=[ - [RunnerLabels.FUNC_TESTER_ARM], - ], - requires=[ - [ArtifactNames.DEB_ARM_RELEASE], - ], -) +from praktika import Workflow +from ci.workflows.defs import ARTIFACTS, BASE_BRANCH, DOCKERS, SECRETS, Jobs workflow = Workflow.Config( name="PR", event=Workflow.Event.PULL_REQUEST, base_branches=[BASE_BRANCH], jobs=[ - style_check_job, - fast_test_job, - *build_jobs, - *stateless_tests_jobs, - *stateful_tests_jobs, - *stress_test_jobs, - ], - artifacts=[ - *Artifact.Config( - name="...", - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/build/programs/clickhouse", - ).parametrize( - names=[ - ArtifactNames.CH_AMD_DEBUG, - ArtifactNames.CH_AMD_RELEASE, - ArtifactNames.CH_ARM_RELEASE, - ArtifactNames.CH_ARM_ASAN, - ] - ), - *Artifact.Config( - name="...", - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/build/programs/clickhouse-odbc-bridge", - ).parametrize( - names=[ - ArtifactNames.CH_ODBC_B_AMD_DEBUG, - ArtifactNames.CH_ODBC_B_AMD_RELEASE, - ArtifactNames.CH_ODBC_B_ARM_RELEASE, - ArtifactNames.CH_ODBC_B_ARM_ASAN, - ] - ), - Artifact.Config( - name=ArtifactNames.DEB_AMD_DEBUG, - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/output/*.deb", - ), - Artifact.Config( - name=ArtifactNames.DEB_AMD_RELEASE, - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/output/*.deb", - ), - Artifact.Config( - name=ArtifactNames.DEB_ARM_RELEASE, - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/output/*.deb", - ), - Artifact.Config( - name=ArtifactNames.DEB_ARM_ASAN, - type=Artifact.Type.S3, - path=f"{Settings.TEMP_DIR}/output/*.deb", - ), + Jobs.style_check_job, + Jobs.fast_test_job, + *Jobs.build_jobs, + *Jobs.stateless_tests_jobs, + *Jobs.stateful_tests_jobs, + *Jobs.stress_test_jobs, ], + artifacts=ARTIFACTS, dockers=DOCKERS, secrets=SECRETS, enable_cache=True, @@ -255,13 +25,3 @@ workflow = Workflow.Config( WORKFLOWS = [ workflow, ] - - -# if __name__ == "__main__": -# # local job test inside praktika environment -# from praktika.runner import Runner -# from praktika.digest import Digest -# -# print(Digest().calc_job_digest(amd_debug_build_job)) -# -# Runner().run(workflow, fast_test_job, docker="fasttest", local_run=True) diff --git a/tests/ci/report.py b/tests/ci/report.py index 842d4d44423..393f131e5ff 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -519,7 +519,7 @@ class JobReport: json.dump(asdict(self), json_file, default=path_converter, indent=2) # temporary WA to ease integration with praktika - check_name = os.getenv("CHECK_NAME", "") + check_name = os.getenv("JOB_NAME", "") if check_name: self.to_praktika_result(job_name=check_name).dump() From 3bb6b579be40dcac4ddac8da075861b954aed445 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Nov 2024 12:43:39 +0100 Subject: [PATCH 263/433] add perf test --- ci/jobs/functional_stateless_tests.py | 1 - ci/jobs/scripts/performance_compare.sh | 1444 ++++++++++++++++++++++++ ci/jobs/scripts/performance_test.sh | 200 ++++ ci/workflows/defs.py | 16 + ci/workflows/pull_request.py | 1 + 5 files changed, 1661 insertions(+), 1 deletion(-) create mode 100755 ci/jobs/scripts/performance_compare.sh create mode 100755 ci/jobs/scripts/performance_test.sh diff --git a/ci/jobs/functional_stateless_tests.py b/ci/jobs/functional_stateless_tests.py index ed717f0dff9..27a9591f3a8 100644 --- a/ci/jobs/functional_stateless_tests.py +++ b/ci/jobs/functional_stateless_tests.py @@ -1,5 +1,4 @@ import argparse -import os import time from pathlib import Path diff --git a/ci/jobs/scripts/performance_compare.sh b/ci/jobs/scripts/performance_compare.sh new file mode 100755 index 00000000000..10587b9eef5 --- /dev/null +++ b/ci/jobs/scripts/performance_compare.sh @@ -0,0 +1,1444 @@ +#!/bin/bash + +set -exu +set -o pipefail +trap "exit" INT TERM +# The watchdog is in the separate process group, so we have to kill it separately +# if the script terminates earlier. +trap 'kill $(jobs -pr) ${watchdog_pid:-} ||:' EXIT + +stage=${stage:-} +script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +# upstream/master +LEFT_SERVER_PORT=9001 +LEFT_SERVER_KEEPER_PORT=9181 +LEFT_SERVER_KEEPER_RAFT_PORT=9234 +LEFT_SERVER_INTERSERVER_PORT=9009 +# patched version +RIGHT_SERVER_PORT=19001 +RIGHT_SERVER_KEEPER_PORT=19181 +RIGHT_SERVER_KEEPER_RAFT_PORT=19234 +RIGHT_SERVER_INTERSERVER_PORT=19009 + +# abort_conf -- abort if some options is not recognized +# abort -- abort if something is not right in the env (i.e. per-cpu arenas does not work) +# narenas -- set them explicitly to avoid disabling per-cpu arena in env +# that returns different number of CPUs for some of the following +# _SC_NPROCESSORS_ONLN/_SC_NPROCESSORS_CONF/sched_getaffinity +export MALLOC_CONF="abort_conf:true,abort:true,narenas:$(nproc --all)" + +function wait_for_server # port, pid +{ + for _ in {1..60} + do + if clickhouse-client --port "$1" --query "select 1" || ! kill -0 "$2" + then + break + fi + sleep 1 + done + + if ! clickhouse-client --port "$1" --query "select 1" + then + echo "Cannot connect to ClickHouse server at $1" + return 1 + fi + + if ! kill -0 "$2" + then + echo "Server pid '$2' is not running" + return 1 + fi +} + +function left_or_right() +{ + local from=$1 && shift + local basename=$1 && shift + + if [ -e "$from/$basename" ]; then + echo "$from/$basename" + return + fi + + case "$from" in + left) echo "right/$basename" ;; + right) echo "left/$basename" ;; + esac +} + +function configure +{ + cd /tmp/praktika/ + # Use the new config for both servers, so that we can change it in a PR. + rm right/config/config.d/text_log.xml ||: + # backups disk uses absolute path, and this overlaps between servers, that could lead to errors + rm right/config/config.d/backups.xml ||: + rm left/config/config.d/backups.xml ||: + cp -rv right/config left ||: + + # Start a temporary server to rename the tables + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done + echo all killed + + set -m # Spawn temporary in its own process groups + + local setup_left_server_opts=( + # server options + --config-file=left/config/config.xml + -- + # server *config* directives overrides + --path db0 + --user_files_path db0/user_files + --top_level_domains_path "$(left_or_right right top_level_domains)" + --keeper_server.storage_path coordination0 + --tcp_port $LEFT_SERVER_PORT + ) + ls -l left/clickhouse-server + left/clickhouse-server "${setup_left_server_opts[@]}" &> setup-server-log.log & + left_pid=$! + kill -0 $left_pid + disown $left_pid + set +m + + wait_for_server $LEFT_SERVER_PORT $left_pid + echo "Server for setup started" + + clickhouse-client --port $LEFT_SERVER_PORT --query "create database test" ||: + clickhouse-client --port $LEFT_SERVER_PORT --query "rename table datasets.hits_v1 to test.hits" ||: + + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done + echo all killed + + # Make copies of the original db for both servers. Use hardlinks instead + # of copying to save space. Before that, remove preprocessed configs and + # system tables, because sharing them between servers with hardlinks may + # lead to weird effects. + rm -r left/db ||: + rm -r right/db ||: + rm -r db0/preprocessed_configs ||: + rm -r db0/{data,metadata}/system ||: + rm db0/status ||: + + cp -al db0/ left/db/ + cp -R coordination0 left/coordination + + cp -al db0/ right/db/ + cp -R coordination0 right/coordination +} + +function restart +{ + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done + echo all killed + + set -m # Spawn servers in their own process groups + + local left_server_opts=( + # server options + --config-file=left/config/config.xml + -- + # server *config* directives overrides + --path left/db + --user_files_path left/db/user_files + --top_level_domains_path "$(left_or_right left top_level_domains)" + --tcp_port $LEFT_SERVER_PORT + --keeper_server.tcp_port $LEFT_SERVER_KEEPER_PORT + --keeper_server.raft_configuration.server.port $LEFT_SERVER_KEEPER_RAFT_PORT + --keeper_server.storage_path left/coordination + --zookeeper.node.port $LEFT_SERVER_KEEPER_PORT + --interserver_http_port $LEFT_SERVER_INTERSERVER_PORT + ) + left/clickhouse-server "${left_server_opts[@]}" &>> left-server-log.log & + left_pid=$! + kill -0 $left_pid + disown $left_pid + + local right_server_opts=( + # server options + --config-file=right/config/config.xml + -- + # server *config* directives overrides + --path right/db + --user_files_path right/db/user_files + --top_level_domains_path "$(left_or_right right top_level_domains)" + --tcp_port $RIGHT_SERVER_PORT + --keeper_server.tcp_port $RIGHT_SERVER_KEEPER_PORT + --keeper_server.raft_configuration.server.port $RIGHT_SERVER_KEEPER_RAFT_PORT + --keeper_server.storage_path right/coordination + --zookeeper.node.port $RIGHT_SERVER_KEEPER_PORT + --interserver_http_port $RIGHT_SERVER_INTERSERVER_PORT + ) + right/clickhouse-server "${right_server_opts[@]}" &>> right-server-log.log & + right_pid=$! + kill -0 $right_pid + disown $right_pid + + set +m + + wait_for_server $LEFT_SERVER_PORT $left_pid + echo left ok + + wait_for_server $RIGHT_SERVER_PORT $right_pid + echo right ok + + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.tables where database NOT IN ('system', 'INFORMATION_SCHEMA', 'information_schema')" + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.build_options" + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.tables where database NOT IN ('system', 'INFORMATION_SCHEMA', 'information_schema')" + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.build_options" + + # Check again that both servers we started are running -- this is important + # for running locally, when there might be some other servers started and we + # will connect to them instead. + kill -0 $left_pid + kill -0 $right_pid +} + +function run_tests +{ + # Just check that the script runs at all + "$script_dir/perf.py" --help > /dev/null + + # Find the directory with test files. + if [ -v CHPC_TEST_PATH ] + then + # Use the explicitly set path to directory with test files. + test_prefix="$CHPC_TEST_PATH" + elif [ "$PR_TO_TEST" == "0" ] + then + # When testing commits from master, use the older test files. This + # allows the tests to pass even when we add new functions and tests for + # them, that are not supported in the old revision. + test_prefix=left/performance + else + # For PRs, use newer test files so we can test these changes. + test_prefix=right/performance + fi + + run_only_changed_tests=0 + + # Determine which tests to run. + if [ -v CHPC_TEST_GREP ] + then + # Run only explicitly specified tests, if any. + # shellcheck disable=SC2010 + test_files=($(ls "$test_prefix" | rg "$CHPC_TEST_GREP" | xargs -I{} -n1 readlink -f "$test_prefix/{}")) + elif [ "$PR_TO_TEST" -ne 0 ] \ + && [ "$(wc -l < changed-test-definitions.txt)" -gt 0 ] \ + && [ "$(wc -l < other-changed-files.txt)" -eq 0 ] + then + # If only the perf tests were changed in the PR, we will run only these + # tests. The lists of changed files are prepared in entrypoint.sh because + # it has the repository. + test_files=($(sed "s/tests\/performance/${test_prefix//\//\\/}/" changed-test-definitions.txt)) + run_only_changed_tests=1 + else + # The default -- run all tests found in the test dir. + test_files=($(ls "$test_prefix"/*.xml)) + fi + + # We can filter out certain tests + if [ -v CHPC_TEST_GREP_EXCLUDE ]; then + # filter tests array in bash https://stackoverflow.com/a/40375567 + filtered_test_files=( $( for i in ${test_files[@]} ; do echo $i ; done | rg -v ${CHPC_TEST_GREP_EXCLUDE} ) ) + test_files=("${filtered_test_files[@]}") + fi + + # We split perf tests into multiple checks to make them faster + if [ -v CHPC_TEST_RUN_BY_HASH_TOTAL ]; then + # filter tests array in bash https://stackoverflow.com/a/40375567 + for index in "${!test_files[@]}"; do + [ $(( index % CHPC_TEST_RUN_BY_HASH_TOTAL )) != "$CHPC_TEST_RUN_BY_HASH_NUM" ] && \ + unset -v 'test_files[$index]' + done + # to have sequential indexes... + test_files=("${test_files[@]}") + fi + + if [ "$run_only_changed_tests" -ne 0 ]; then + if [ ${#test_files[@]} -eq 0 ]; then + time "$script_dir/report.py" --no-tests-run > report.html + exit 0 + fi + fi + + # For PRs w/o changes in test definitions, test only a subset of queries, + # and run them less times. If the corresponding environment variables are + # already set, keep those values. + # + # NOTE: too high CHPC_RUNS/CHPC_MAX_QUERIES may hit internal CI timeout. + # NOTE: Currently we disabled complete run even for master branch + #if [ "$PR_TO_TEST" -ne 0 ] && [ "$(wc -l < changed-test-definitions.txt)" -eq 0 ] + #then + # CHPC_RUNS=${CHPC_RUNS:-7} + # CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-10} + #else + # CHPC_RUNS=${CHPC_RUNS:-13} + # CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-0} + #fi + + CHPC_RUNS=${CHPC_RUNS:-7} + CHPC_MAX_QUERIES=${CHPC_MAX_QUERIES:-10} + + export CHPC_RUNS + export CHPC_MAX_QUERIES + + # Determine which concurrent benchmarks to run. For now, the only test + # we run as a concurrent benchmark is 'website'. Run it as benchmark if we + # are also going to run it as a normal test. + for test in ${test_files[@]}; do echo "$test"; done | sed -n '/website/p' > benchmarks-to-run.txt + + # Delete old report files. + for x in {test-times,wall-clock-times}.tsv + do + rm -v "$x" ||: + touch "$x" + done + + # Randomize test order. BTW, it's not an array no more. + test_files=$(for f in ${test_files[@]}; do echo "$f"; done | sort -R) + + # Limit profiling time to 10 minutes, not to run for too long. + profile_seconds_left=600 + + # Run the tests. + total_tests=$(echo "$test_files" | wc -w) + current_test=0 + test_name="" + for test in $test_files + do + echo "$current_test of $total_tests tests complete" > status.txt + # Check that both servers are alive, and restart them if they die. + clickhouse-client --port $LEFT_SERVER_PORT --query "select 1 format Null" \ + || { echo $test_name >> left-server-died.log ; restart ; } + clickhouse-client --port $RIGHT_SERVER_PORT --query "select 1 format Null" \ + || { echo $test_name >> right-server-died.log ; restart ; } + + test_name=$(basename "$test" ".xml") + echo test "$test_name" + + # Don't profile if we're past the time limit. + # Use awk because bash doesn't support floating point arithmetic. + profile_seconds=$(awk "BEGIN { print ($profile_seconds_left > 0 ? 10 : 0) }") + + if rg --quiet "$(basename $test)" changed-test-definitions.txt + then + # Run all queries from changed test files to ensure that all new queries will be tested. + max_queries=0 + else + max_queries=$CHPC_MAX_QUERIES + fi + + ( + set +x + argv=( + --host localhost localhost + --port "$LEFT_SERVER_PORT" "$RIGHT_SERVER_PORT" + --runs "$CHPC_RUNS" + --max-queries "$max_queries" + --profile-seconds "$profile_seconds" + + "$test" + ) + TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") + # one more subshell to suppress trace output for "set +x" + ( + time "$script_dir/perf.py" "${argv[@]}" > "$test_name-raw.tsv" 2> "$test_name-err.log" + ) 2>>wall-clock-times.tsv >/dev/null \ + || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" + ) 2>/dev/null + + profile_seconds_left=$(awk -F' ' \ + 'BEGIN { s = '$profile_seconds_left'; } /^profile-total/ { s -= $2 } END { print s }' \ + "$test_name-raw.tsv") + current_test=$((current_test + 1)) + done + + wait +} + +function get_profiles_watchdog +{ + sleep 600 + + echo "The trace collection did not finish in time." >> profile-errors.log + + for pid in $(pgrep -f clickhouse) + do + sudo gdb -p "$pid" --batch --ex "info proc all" --ex "thread apply all bt" --ex quit &> "$pid.gdb.log" & + done + wait + + for _ in {1..10} + do + if ! pkill -f clickhouse + then + break + fi + sleep 1 + done +} + +function get_profiles +{ + # Collect the profiles + clickhouse-client --port $LEFT_SERVER_PORT --query "system flush logs" & + clickhouse-client --port $RIGHT_SERVER_PORT --query "system flush logs" & + + wait + + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.query_log where type in ('QueryFinish', 'ExceptionWhileProcessing') format TSVWithNamesAndTypes" > left-query-log.tsv ||: & + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.trace_log format TSVWithNamesAndTypes" > left-trace-log.tsv ||: & + clickhouse-client --port $LEFT_SERVER_PORT --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > left-addresses.tsv ||: & + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.metric_log format TSVWithNamesAndTypes" > left-metric-log.tsv ||: & + clickhouse-client --port $LEFT_SERVER_PORT --query "select * from system.asynchronous_metric_log format TSVWithNamesAndTypes" > left-async-metric-log.tsv ||: & + + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.query_log where type in ('QueryFinish', 'ExceptionWhileProcessing') format TSVWithNamesAndTypes" > right-query-log.tsv ||: & + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.trace_log format TSVWithNamesAndTypes" > right-trace-log.tsv ||: & + clickhouse-client --port $RIGHT_SERVER_PORT --query "select arrayJoin(trace) addr, concat(splitByChar('/', addressToLine(addr))[-1], '#', demangle(addressToSymbol(addr)) ) name from system.trace_log group by addr format TSVWithNamesAndTypes" > right-addresses.tsv ||: & + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.metric_log format TSVWithNamesAndTypes" > right-metric-log.tsv ||: & + clickhouse-client --port $RIGHT_SERVER_PORT --query "select * from system.asynchronous_metric_log format TSVWithNamesAndTypes" > right-async-metric-log.tsv ||: & + + wait + + # Just check that the servers are alive so that we return a proper exit code. + # We don't consistently check the return codes of the above background jobs. + clickhouse-client --port $LEFT_SERVER_PORT --query "select 1" + clickhouse-client --port $RIGHT_SERVER_PORT --query "select 1" +} + +# Build and analyze randomization distribution for all queries. +function analyze_queries +{ +rm -v analyze-commands.txt analyze-errors.log all-queries.tsv unstable-queries.tsv ./*-report.tsv raw-queries.tsv ||: +rm -rf analyze ||: +mkdir analyze analyze/tmp ||: + +# Split the raw test output into files suitable for analysis. +# To debug calculations only for a particular test, substitute a suitable +# wildcard here, e.g. `for test_file in modulo-raw.tsv`. +for test_file in *-raw.tsv +do + test_name=$(basename "$test_file" "-raw.tsv") + sed -n "s/^query\t/$test_name\t/p" < "$test_file" >> "analyze/query-runs.tsv" + sed -n "s/^profile\t/$test_name\t/p" < "$test_file" >> "analyze/query-profiles.tsv" + sed -n "s/^client-time\t/$test_name\t/p" < "$test_file" >> "analyze/client-times.tsv" + sed -n "s/^report-threshold\t/$test_name\t/p" < "$test_file" >> "analyze/report-thresholds.tsv" + sed -n "s/^skipped\t/$test_name\t/p" < "$test_file" >> "analyze/skipped-tests.tsv" + sed -n "s/^display-name\t/$test_name\t/p" < "$test_file" >> "analyze/query-display-names.tsv" + sed -n "s/^partial\t/$test_name\t/p" < "$test_file" >> "analyze/partial-queries.tsv" +done + +# for each query run, prepare array of metrics from query log +clickhouse-local --query " +create view query_runs as select * from file('analyze/query-runs.tsv', TSV, + 'test text, query_index int, query_id text, version UInt8, time float'); + +-- Separately process backward-incompatible ('partial') queries which we could only run on the new server +-- because they use new functions. We can't make normal stats for them, but still +-- have to show some stats so that the PR author can tweak them. +create view partial_queries as select test, query_index + from file('analyze/partial-queries.tsv', TSV, + 'test text, query_index int, servers Array(int)'); + +create table partial_query_times engine File(TSVWithNamesAndTypes, + 'analyze/partial-query-times.tsv') + as select test, query_index, stddevPop(time) time_stddev, median(time) time_median + from query_runs + where (test, query_index) in partial_queries + group by test, query_index + ; + +-- Process queries that were run normally, on both servers. +create view left_query_log as select * + from file('left-query-log.tsv', TSVWithNamesAndTypes); + +create view right_query_log as select * + from file('right-query-log.tsv', TSVWithNamesAndTypes); + +create view query_logs as + select 0 version, query_id, ProfileEvents, + query_duration_ms, memory_usage from left_query_log + union all + select 1 version, query_id, ProfileEvents, + query_duration_ms, memory_usage from right_query_log + ; + +-- This is a single source of truth on all metrics we have for query runs. The +-- metrics include ProfileEvents from system.query_log, and query run times +-- reported by the perf.py test runner. +create table query_run_metric_arrays engine File(TSV, 'analyze/query-run-metric-arrays.tsv') + as + with ( + -- sumMapState with the list of all keys with nullable '0' values because sumMap removes keys with default values + -- and 0::Nullable != NULL + with (select groupUniqArrayArray(mapKeys(ProfileEvents)) from query_logs) as all_names + select arrayReduce('sumMapState', [(all_names, arrayMap(x->0::Nullable(Float64), all_names))]) + ) as all_metrics + select test, query_index, version, query_id, + (finalizeAggregation( + arrayReduce('sumMapMergeState', + [ + all_metrics, + arrayReduce('sumMapState', + [(mapKeys(ProfileEvents), + arrayMap(x->toNullable(toFloat64(x)), mapValues(ProfileEvents)))] + ), + arrayReduce('sumMapState', [( + ['client_time', 'server_time', 'memory_usage'], + [toNullable(toFloat64(query_runs.time)), toNullable(toFloat64(query_duration_ms / 1000.)), toNullable(toFloat64(memory_usage))] + )]) + ] + )) as metrics_tuple).1 metric_names, + arrayMap(x->if(isNaN(x),0,x), metrics_tuple.2) metric_values + from query_logs + right join query_runs + on query_logs.query_id = query_runs.query_id + and query_logs.version = query_runs.version + where (test, query_index) not in partial_queries + ; + +-- This is just for convenience -- human-readable + easy to make plots. +create table query_run_metrics_denorm engine File(TSV, 'analyze/query-run-metrics-denorm.tsv') + as select test, query_index, metric_names, version, query_id, metric_values + from query_run_metric_arrays + array join metric_names, metric_values + order by test, query_index, metric_names, version, query_id + ; + +-- Filter out tests that don't have an even number of runs, to avoid breaking +-- the further calculations. This may happen if there was an error during the +-- test runs, e.g. the server died. It will be reported in test errors, so we +-- don't have to report it again. +create view broken_queries as + select test, query_index + from query_runs + group by test, query_index + having count(*) % 2 != 0 + ; + +-- This is for statistical processing with eqmed.sql +create table query_run_metrics_for_stats engine File( + TSV, -- do not add header -- will parse with grep + 'analyze/query-run-metrics-for-stats.tsv') + as select test, query_index, 0 run, version, + -- For debugging, add a filter for a particular metric like this: + -- arrayFilter(m, n -> n = 'client_time', metric_values, metric_names) + -- metric_values + -- Note that further reporting may break, because the metric names are + -- not filtered. + metric_values + from query_run_metric_arrays + where (test, query_index) not in broken_queries + order by test, query_index, run, version + ; + +-- This is the list of metric names, so that we can join them back after +-- statistical processing. +create table query_run_metric_names engine File(TSV, 'analyze/query-run-metric-names.tsv') + as select metric_names from query_run_metric_arrays limit 1 + ; +" 2> >(tee -a analyze/errors.log 1>&2) + +# This is a lateral join in bash... please forgive me. +# We don't have arrayPermute(), so I have to make random permutations with +# `order by rand`, and it becomes really slow if I do it for more than one +# query. We also don't have lateral joins. So I just put all runs of each +# query into a separate file, and then compute randomization distribution +# for each file. I do this in parallel using GNU parallel. +( set +x # do not bloat the log +IFS=$'\n' +for prefix in $(cut -f1,2 "analyze/query-run-metrics-for-stats.tsv" | sort | uniq) +do + file="analyze/tmp/${prefix// /_}.tsv" + rg "^$prefix " "analyze/query-run-metrics-for-stats.tsv" > "$file" & + printf "%s\0\n" \ + "clickhouse-local \ + --file \"$file\" \ + --structure 'test text, query text, run int, version UInt8, metrics Array(float)' \ + --query \"$(cat "$script_dir/eqmed.sql")\" \ + >> \"analyze/query-metric-stats.tsv\"" \ + 2>> analyze/errors.log \ + >> analyze/commands.txt +done +wait +unset IFS +) + +# The comparison script might be bound to one NUMA node for better test +# stability, and the calculation runs out of memory because of this. Use +# all nodes. +numactl --show +numactl --cpunodebind=all --membind=all numactl --show + +# Notes for parallel: +# +# Some queries can consume 8+ GB of memory, so it worth to limit amount of jobs +# that can be run in parallel. +# +# --memfree: +# +# will kill jobs, which is not good (and retried until --retries exceeded) +# +# --memsuspend: +# +# If the available memory falls below 2 * size, GNU parallel will suspend some of the running jobs. +numactl --cpunodebind=all --membind=all parallel -v --joblog analyze/parallel-log.txt --memsuspend 15G --null < analyze/commands.txt 2>> analyze/errors.log + +clickhouse-local --query " +-- Join the metric names back to the metric statistics we've calculated, and make +-- a denormalized table of them -- statistics for all metrics for all queries. +-- The WITH, ARRAY JOIN and CROSS JOIN do not like each other: +-- https://github.com/ClickHouse/ClickHouse/issues/11868 +-- https://github.com/ClickHouse/ClickHouse/issues/11757 +-- Because of this, we make a view with arrays first, and then apply all the +-- array joins. +create view query_metric_stat_arrays as + with (select * from file('analyze/query-run-metric-names.tsv', + TSV, 'n Array(String)')) as metric_name + select test, query_index, metric_name, left, right, diff, stat_threshold + from file('analyze/query-metric-stats.tsv', TSV, 'left Array(float), + right Array(float), diff Array(float), stat_threshold Array(float), + test text, query_index int') reports + order by test, query_index, metric_name + ; + +create table query_metric_stats_denorm engine File(TSVWithNamesAndTypes, + 'analyze/query-metric-stats-denorm.tsv') + as select test, query_index, metric_name, left, right, diff, stat_threshold + from query_metric_stat_arrays + left array join metric_name, left, right, diff, stat_threshold + order by test, query_index, metric_name + ; +" 2> >(tee -a analyze/errors.log 1>&2) + +# Fetch historical query variability thresholds from the CI database +if [ -v CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL ] +then + set +x # Don't show password in the log + client=(clickhouse-client + # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 + # so I have to extract host and port with clickhouse-local. I tried to use + # Poco URI parser to support this in the client, but it's broken and can't + # parse host:port. + $(clickhouse-local --query "with '${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") + --secure + --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" + --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" + --config "right/config/client_config.xml" + --date_time_input_format=best_effort) + + +# Precision is going to be 1.5 times worse for PRs, because we run the queries +# less times. How do I know it? I ran this: +# SELECT quantilesExact(0., 0.1, 0.5, 0.75, 0.95, 1.)(p / m) +# FROM +# ( +# SELECT +# quantileIf(0.95)(stat_threshold, pr_number = 0) AS m, +# quantileIf(0.95)(stat_threshold, (pr_number != 0) AND (abs(diff) < stat_threshold)) AS p +# FROM query_metrics_v2 +# WHERE (event_date > (today() - toIntervalMonth(1))) AND (metric = 'client_time') +# GROUP BY +# test, +# query_index, +# query_display_name +# HAVING count(*) > 100 +# ) +# +# The file can be empty if the server is inaccessible, so we can't use +# TSVWithNamesAndTypes. +# + "${client[@]}" --query " + select test, query_index, + quantileExact(0.99)(abs(diff)) * 1.5 AS max_diff, + quantileExactIf(0.99)(stat_threshold, abs(diff) < stat_threshold) * 1.5 AS max_stat_threshold, + query_display_name + from query_metrics_v2 + -- We use results at least one week in the past, so that the current + -- changes do not immediately influence the statistics, and we have + -- some time to notice that something is wrong. + where event_date between now() - interval 1 month - interval 1 week + and now() - interval 1 week + and metric = 'client_time' + and pr_number = 0 + group by test, query_index, query_display_name + having count(*) > 100 + " > analyze/historical-thresholds.tsv + set -x +else + touch analyze/historical-thresholds.tsv +fi + +} + +# Analyze results +function report +{ +rm -r report ||: +mkdir report report/tmp ||: + +rm ./*.{rep,svg} test-times.tsv test-dump.tsv unstable.tsv unstable-query-ids.tsv unstable-query-metrics.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv all-queries.tsv run-errors.tsv ||: + +cat analyze/errors.log >> report/errors.log ||: +cat profile-errors.log >> report/errors.log ||: + +clickhouse-local --query " +create view query_display_names as select * from + file('analyze/query-display-names.tsv', TSV, + 'test text, query_index int, query_display_name text') + ; + +create view partial_query_times as select * from + file('analyze/partial-query-times.tsv', TSVWithNamesAndTypes, + 'test text, query_index int, time_stddev float, time_median double') + ; + +-- Report for backward-incompatible ('partial') queries that we could only run on the new server (e.g. +-- queries with new functions added in the tested PR). +create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') + as select round(time_median, 3) time, + round(time_stddev / time_median, 3) relative_time_stddev, + test, query_index, query_display_name + from partial_query_times + join query_display_names using (test, query_index) + order by test, query_index + ; + +create view query_metric_stats as + select * from file('analyze/query-metric-stats-denorm.tsv', + TSVWithNamesAndTypes, + 'test text, query_index int, metric_name text, left float, right float, + diff float, stat_threshold float') + ; + +create table report_thresholds engine File(TSVWithNamesAndTypes, 'report/thresholds.tsv') + as select + query_display_names.test test, query_display_names.query_index query_index, + ceil(greatest(0.1, historical_thresholds.max_diff, + test_thresholds.report_threshold), 2) changed_threshold, + ceil(greatest(0.2, historical_thresholds.max_stat_threshold, + test_thresholds.report_threshold + 0.1), 2) unstable_threshold, + query_display_names.query_display_name query_display_name + from query_display_names + left join file('analyze/historical-thresholds.tsv', TSV, + 'test text, query_index int, max_diff float, max_stat_threshold float, + query_display_name text') historical_thresholds + on query_display_names.test = historical_thresholds.test + and query_display_names.query_index = historical_thresholds.query_index + and query_display_names.query_display_name = historical_thresholds.query_display_name + left join file('analyze/report-thresholds.tsv', TSV, + 'test text, report_threshold float') test_thresholds + on query_display_names.test = test_thresholds.test + ; + +-- Main statistics for queries -- query time as reported in query log. +create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') + as select + -- It is important to have a non-strict inequality with stat_threshold + -- here. The randomization distribution is actually discrete, and when + -- the number of runs is small, the quantile we need (e.g. 0.99) turns + -- out to be the maximum value of the distribution. We can also hit this + -- maximum possible value with our test run, and this obviously means + -- that we have observed the difference to the best precision possible + -- for the given number of runs. If we use a strict equality here, we + -- will miss such cases. This happened in the wild and lead to some + -- uncaught regressions, because for the default 7 runs we do for PRs, + -- the randomization distribution has only 16 values, so the max quantile + -- is actually 0.9375. + abs(diff) > changed_threshold and abs(diff) >= stat_threshold as changed_fail, + abs(diff) > changed_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show, + + not changed_fail and stat_threshold > unstable_threshold as unstable_fail, + not changed_show and stat_threshold > unstable_threshold - 0.05 as unstable_show, + + left, right, diff, stat_threshold, + query_metric_stats.test test, query_metric_stats.query_index query_index, + query_display_names.query_display_name query_display_name + from query_metric_stats + left join query_display_names + on query_metric_stats.test = query_display_names.test + and query_metric_stats.query_index = query_display_names.query_index + left join report_thresholds + on query_display_names.test = report_thresholds.test + and query_display_names.query_index = report_thresholds.query_index + and query_display_names.query_display_name = report_thresholds.query_display_name + -- 'server_time' is rounded down to ms, which might be bad for very short queries. + -- Use 'client_time' instead. + where metric_name = 'client_time' + order by test, query_index, metric_name + ; + +create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') + as with + -- server_time is sometimes reported as zero (if it's less than 1 ms), + -- so we have to work around this to not get an error about conversion + -- of NaN to decimal. + (left > right ? left / right : right / left) as times_change_float, + isFinite(times_change_float) as times_change_finite, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + times_change_finite + ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' + : '--' as times_change_str + select + round(left, 3), round(right, 3), times_change_str, + round(diff, 3), round(stat_threshold, 3), + changed_fail, test, query_index, query_display_name + from queries where changed_show order by abs(diff) desc; + +create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') + as select + round(left, 3), round(right, 3), round(diff, 3), + round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name + from queries where unstable_show order by stat_threshold desc; + + +create view test_speedup as + select + test, + exp2(avg(log2(left / right))) times_speedup, + count(*) queries, + unstable + changed bad, + sum(changed_show) changed, + sum(unstable_show) unstable + from queries + group by test + order by times_speedup desc + ; + +create view total_speedup as + select + 'Total' test, + exp2(avg(log2(times_speedup))) times_speedup, + sum(queries) queries, + unstable + changed bad, + sum(changed) changed, + sum(unstable) unstable + from test_speedup + ; + +create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') + as with + (times_speedup >= 1 + ? '-' || toString(round(times_speedup, 3)) || 'x' + : '+' || toString(round(1 / times_speedup, 3)) || 'x') + as times_speedup_str + select test, times_speedup_str, queries, bad, changed, unstable + -- Not sure what's the precedence of UNION ALL vs WHERE & ORDER BY, hence all + -- the braces. + from ( + ( + select * from total_speedup + ) union all ( + select * from test_speedup + where + (times_speedup >= 1 ? times_speedup : (1 / times_speedup)) >= 1.005 + or bad + ) + ) + order by test = 'Total' desc, times_speedup desc + ; + + +create view total_client_time_per_query as select * + from file('analyze/client-times.tsv', TSV, + 'test text, query_index int, client float, server float'); + +create table wall_clock_time_per_test engine Memory as select * + from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float'); + +create table test_time engine Memory as + select test, sum(client) total_client_time, + max(client) query_max, + min(client) query_min, + count(*) queries + from total_client_time_per_query full join queries using (test, query_index) + group by test; + +create view query_runs as select * from file('analyze/query-runs.tsv', TSV, + 'test text, query_index int, query_id text, version UInt8, time float'); + +-- +-- Guess the number of query runs used for this test. The number is required to +-- calculate and check the average query run time in the report. +-- We have to be careful, because we will encounter: +-- 1) backward-incompatible ('partial') queries which run only on one server +-- 3) some errors that make query run for a different number of times on a +-- particular server. +-- +create view test_runs as + select test, + -- Default to 7 runs if there are only 'short' queries in the test, and + -- we can't determine the number of runs. + if((ceil(median(t.runs), 0) as r) != 0, r, 7) runs + from ( + select + -- The query id is the same for both servers, so no need to divide here. + uniqExact(query_id) runs, + test, query_index + from query_runs + group by test, query_index + ) t + group by test + ; + +create view test_times_view as + select + wall_clock_time_per_test.test test, + real, + total_client_time, + queries, + query_max, + real / if(queries > 0, queries, 1) avg_real_per_query, + query_min, + runs + from test_time + -- wall clock times are also measured for skipped tests, so don't + -- do full join + left join wall_clock_time_per_test + on wall_clock_time_per_test.test = test_time.test + full join test_runs + on test_runs.test = test_time.test + ; + +-- WITH TOTALS doesn't work with INSERT SELECT, so we have to jump through these +-- hoops: https://github.com/ClickHouse/ClickHouse/issues/15227 +create view test_times_view_total as + select + 'Total' test, + sum(real), + sum(total_client_time), + sum(queries), + max(query_max), + sum(real) / if(sum(queries) > 0, sum(queries), 1) avg_real_per_query, + min(query_min), + -- Totaling the number of runs doesn't make sense, but use the max so + -- that the reporting script doesn't complain about queries being too + -- long. + max(runs) + from test_times_view + ; + +create table test_times_report engine File(TSV, 'report/test-times.tsv') + as select + test, + round(real, 3), + round(total_client_time, 3), + queries, + round(query_max, 3), + round(avg_real_per_query, 3), + round(query_min, 3), + runs + from ( + select * from test_times_view + union all + select * from test_times_view_total + ) + order by test = 'Total' desc, avg_real_per_query desc + ; + +-- report for all queries page, only main metric +create table all_tests_report engine File(TSV, 'report/all-queries.tsv') + as with + -- server_time is sometimes reported as zero (if it's less than 1 ms), + -- so we have to work around this to not get an error about conversion + -- of NaN to decimal. + (left > right ? left / right : right / left) as times_change_float, + isFinite(times_change_float) as times_change_finite, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + times_change_finite + ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' + : '--' as times_change_str + select changed_fail, unstable_fail, + round(left, 3), round(right, 3), times_change_str, + round(isFinite(diff) ? diff : 0, 3), + round(isFinite(stat_threshold) ? stat_threshold : 0, 3), + test, query_index, query_display_name + from queries order by test, query_index; + + +-------------------------------------------------------------------------------- +-- various compatibility data formats follow, not related to the main report + +-- keep the table in old format so that we can analyze new and old data together +create table queries_old_format engine File(TSVWithNamesAndTypes, 'queries.rep') + as select 0 short, changed_fail, unstable_fail, left, right, diff, + stat_threshold, test, query_display_name query + from queries + ; + +-- new report for all queries with all metrics (no page yet) +create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.tsv') as + select metric_name, left, right, diff, + floor(left > right ? left / right : right / left, 3), + stat_threshold, test, query_index, query_display_name + from query_metric_stats + left join query_display_names + on query_metric_stats.test = query_display_names.test + and query_metric_stats.query_index = query_display_names.query_index + order by test, query_index; +" 2> >(tee -a report/errors.log 1>&2) + +# Prepare source data for metrics and flamegraphs for queries that were profiled +# by perf.py. +for version in {right,left} +do + rm -rf data + clickhouse-local --query " +create view query_profiles as + with 0 as left, 1 as right + select * from file('analyze/query-profiles.tsv', TSV, + 'test text, query_index int, query_id text, version UInt8, time float') + where version = $version + ; + +create view query_display_names as select * from + file('analyze/query-display-names.tsv', TSV, + 'test text, query_index int, query_display_name text') + ; + +create table unstable_query_runs engine File(TSVWithNamesAndTypes, + 'unstable-query-runs.$version.rep') as + select query_profiles.test test, query_profiles.query_index query_index, + query_display_name, query_id + from query_profiles + left join query_display_names on + query_profiles.test = query_display_names.test + and query_profiles.query_index = query_display_names.query_index + ; + +create view query_log as select * + from file('$version-query-log.tsv', TSVWithNamesAndTypes); + +create table unstable_run_metrics engine File(TSVWithNamesAndTypes, + 'unstable-run-metrics.$version.rep') as + select test, query_index, query_id, value, metric + from query_log + array join + mapValues(ProfileEvents) as value, + mapKeys(ProfileEvents) as metric + join unstable_query_runs using (query_id) + ; + +create table unstable_run_metrics_2 engine File(TSVWithNamesAndTypes, + 'unstable-run-metrics-2.$version.rep') as + select + test, query_index, query_id, + v, n + from ( + select + test, query_index, query_id, + ['memory_usage', 'read_bytes', 'written_bytes', 'query_duration_ms'] n, + [memory_usage, read_bytes, written_bytes, query_duration_ms] v + from query_log + join unstable_query_runs using (query_id) + ) + array join v, n; + +create view trace_log as select * + from file('$version-trace-log.tsv', TSVWithNamesAndTypes); + +create view addresses_src as select addr, + -- Some functions change name between builds, e.g. '__clone' or 'clone' or + -- even '__GI__clone@@GLIBC_2.32'. This breaks differential flame graphs, so + -- filter them out here. + [name, 'clone.S (filtered by script)', 'pthread_cond_timedwait (filtered by script)'] + -- this line is a subscript operator of the above array + [1 + multiSearchFirstIndex(name, ['clone.S', 'pthread_cond_timedwait'])] name + from file('$version-addresses.tsv', TSVWithNamesAndTypes); + +create table addresses_join_$version engine Join(any, left, address) as + select addr address, name from addresses_src; + +create table unstable_run_traces engine File(TSVWithNamesAndTypes, + 'unstable-run-traces.$version.rep') as + select + test, query_index, query_id, + count() value, + joinGet(addresses_join_$version, 'name', arrayJoin(trace)) + || '(' || toString(trace_type) || ')' metric + from trace_log + join unstable_query_runs using query_id + group by test, query_index, query_id, metric + order by count() desc + ; + +create table stacks engine File(TSV, 'report/stacks.$version.tsv') as + select + -- first goes the key used to split the file with grep + test, query_index, trace_type, any(query_display_name), + -- next go the stacks in flamegraph format: 'func1;...;funcN count' + arrayStringConcat( + arrayMap( + addr -> joinGet(addresses_join_$version, 'name', addr), + arrayReverse(trace) + ), + ';' + ) readable_trace, + count() c + from trace_log + join unstable_query_runs using query_id + group by test, query_index, trace_type, trace + order by test, query_index, trace_type, trace + ; +" 2> >(tee -a report/errors.log 1>&2) & +done +wait + +# Create per-query flamegraphs +touch report/query-files.txt +IFS=$'\n' +for version in {right,left} +do + for query in $(cut -d' ' -f1-4 "report/stacks.$version.tsv" | sort | uniq) + do + query_file=$(echo "$query" | cut -c-120 | sed 's/[/ ]/_/g') + echo "$query_file" >> report/query-files.txt + + # Build separate .svg flamegraph for each query. + # -F is somewhat unsafe because it might match not the beginning of the + # string, but this is unlikely and escaping the query for grep is a pain. + rg -F "$query " "report/stacks.$version.tsv" \ + | cut -f 5- \ + | sed 's/\t/ /g' \ + | tee "report/tmp/$query_file.stacks.$version.tsv" \ + | ~/fg/flamegraph.pl --hash > "$query_file.$version.svg" & + done +done +wait +unset IFS + +# Create differential flamegraphs. +while IFS= read -r query_file +do + ~/fg/difffolded.pl "report/tmp/$query_file.stacks.left.tsv" \ + "report/tmp/$query_file.stacks.right.tsv" \ + | tee "report/tmp/$query_file.stacks.diff.tsv" \ + | ~/fg/flamegraph.pl > "$query_file.diff.svg" & +done < report/query-files.txt +wait + +# Create per-query files with metrics. Note that the key is different from flamegraphs. +IFS=$'\n' +for version in {right,left} +do + for query in $(cut -d' ' -f1-3 "report/metric-deviation.$version.tsv" | sort | uniq) + do + query_file=$(echo "$query" | cut -c-120 | sed 's/[/ ]/_/g') + + # Ditto the above comment about -F. + rg -F "$query " "report/metric-deviation.$version.tsv" \ + | cut -f4- > "$query_file.$version.metrics.rep" & + done +done +wait +unset IFS + +# Prefer to grep for clickhouse_driver exception messages, but if there are none, +# just show a couple of lines from the log. +for log in *-err.log +do + test=$(basename "$log" "-err.log") + { + # The second grep is a heuristic for error messages like + # "socket.timeout: timed out". + rg --no-filename --max-count=2 -i '\(Exception\|Error\):[^:]' "$log" \ + || rg --no-filename --max-count=2 -i '^[^ ]\+: ' "$log" \ + || head -2 "$log" + } | sed "s/^/$test\t/" >> run-errors.tsv ||: +done +} + +function report_metrics +{ +rm -rf metrics ||: +mkdir metrics + +clickhouse-local --query " +create view right_async_metric_log as + select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes) + ; + +-- Use the right log as time reference because it may have higher precision. +create table metrics engine File(TSV, 'metrics/metrics.tsv') as + with (select min(event_time) from right_async_metric_log) as min_time + select metric, r.event_time - min_time event_time, l.value as left, r.value as right + from right_async_metric_log r + asof join file('left-async-metric-log.tsv', TSVWithNamesAndTypes) l + on l.metric = r.metric and r.event_time <= l.event_time + order by metric, event_time + ; + +-- Show metrics that have changed +create table changes engine File(TSV, 'metrics/changes.tsv') + as select metric, left, right, + round(diff, 3), round(times_diff, 3) + from ( + select metric, median(left) as left, median(right) as right, + (right - left) / left diff, + if(left > right, left / right, right / left) times_diff + from metrics + group by metric + having abs(diff) > 0.05 and isFinite(diff) and isFinite(times_diff) + ) + order by diff desc + ; +" 2> >(tee -a metrics/errors.log 1>&2) + +IFS=$'\n' +for prefix in $(cut -f1 "metrics/metrics.tsv" | sort | uniq) +do + file="metrics/$prefix.tsv" + rg "^$prefix " "metrics/metrics.tsv" | cut -f2- > "$file" + + gnuplot -e " + set datafile separator '\t'; + set terminal png size 960,540; + set xtics time format '%tH:%tM'; + set title '$prefix' noenhanced offset 0,-3; + set key left top; + plot + '$file' using 1:2 with lines title 'Left' + , '$file' using 1:3 with lines title 'Right' + ; + " \ + | convert - -filter point -resize "200%" "metrics/$prefix.png" & + +done +wait +unset IFS +} + +function upload_results +{ + # Prepare info for the CI checks table. + rm -f ci-checks.tsv + + clickhouse-local --query " +create view queries as select * from file('report/queries.tsv', TSVWithNamesAndTypes); + +create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') + as select + $PR_TO_TEST :: UInt32 AS pull_request_number, + '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, + '${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name, + '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, + (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, + fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time, + test_name :: LowCardinality(String) AS test_name , + test_status :: LowCardinality(String) AS test_status, + test_duration_ms :: UInt64 AS test_duration_ms, + report_url, + $PR_TO_TEST = 0 + ? 'https://github.com/ClickHouse/ClickHouse/commit/$SHA_TO_TEST' + : 'https://github.com/ClickHouse/ClickHouse/pull/$PR_TO_TEST' pull_request_url, + '' commit_url, + '' task_url, + '' base_ref, + '' base_repo, + '' head_ref, + '' head_repo + from ( + select '' test_name, + '$(sed -n 's/.*/\1/p' report.html)' test_status, + 0 test_duration_ms, + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#fail1' report_url + union all + select + test || ' #' || toString(query_index) || '::' || test_desc_.1 test_name, + 'slower' test_status, + test_desc_.2*1e3 test_duration_ms, + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#changes-in-performance.' || test || '.' || toString(query_index) report_url + from queries + array join map('old', left, 'new', right) as test_desc_ + where changed_fail != 0 and diff > 0 + union all + select + test || ' #' || toString(query_index) || '::' || test_desc_.1 test_name, + 'unstable' test_status, + test_desc_.2*1e3 test_duration_ms, + 'https://s3.amazonaws.com/clickhouse-test-reports/$PR_TO_TEST/$SHA_TO_TEST/${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME_PREFIX}/report.html#unstable-queries.' || test || '.' || toString(query_index) report_url + from queries + array join map('old', left, 'new', right) as test_desc_ + where unstable_fail != 0 + ) +; + " + + if ! [ -v CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL ] + then + echo Database for test results is not specified, will not upload them. + return 0 + fi + + set +x # Don't show password in the log + client=(clickhouse-client + # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 + # so I have to extract host and port with clickhouse-local. I tried to use + # Poco URI parser to support this in the client, but it's broken and can't + # parse host:port. + $(clickhouse-local --query "with '${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") + --secure + --user "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER}" + --password "${CLICKHOUSE_PERFORMANCE_COMPARISON_DATABASE_USER_PASSWORD}" + --config "right/config/client_config.xml" + --date_time_input_format=best_effort) + + # CREATE TABLE IF NOT EXISTS query_metrics_v2 ( + # `event_date` Date, + # `event_time` DateTime, + # `pr_number` UInt32, + # `old_sha` String, + # `new_sha` String, + # `test` LowCardinality(String), + # `query_index` UInt32, + # `query_display_name` String, + # `metric` LowCardinality(String), + # `old_value` Float64, + # `new_value` Float64, + # `diff` Float64, + # `stat_threshold` Float64 + # ) ENGINE = ReplicatedMergeTree + # ORDER BY event_date + + # CREATE TABLE IF NOT EXISTS run_attributes_v1 ( + # `old_sha` String, + # `new_sha` String, + # `metric` LowCardinality(String), + # `metric_value` String + # ) ENGINE = ReplicatedMergeTree + # ORDER BY (old_sha, new_sha) + + "${client[@]}" --query " + insert into query_metrics_v2 + select + toDate(event_time) event_date, + toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + $PR_TO_TEST pr_number, + '$REF_SHA' old_sha, + '$SHA_TO_TEST' new_sha, + test, + query_index, + query_display_name, + metric_name as metric, + old_value, + new_value, + diff, + stat_threshold + from input('metric_name text, old_value float, new_value float, diff float, + ratio_display_text text, stat_threshold float, + test text, query_index int, query_display_name text') + format TSV +" < report/all-query-metrics.tsv # Don't leave whitespace after INSERT: https://github.com/ClickHouse/ClickHouse/issues/16652 + + # Upload some run attributes. I use this weird form because it is the same + # form that can be used for historical data when you only have compare.log. + cat compare.log \ + | sed -n ' + s/.*Model name:[[:space:]]\+\(.*\)$/metric lscpu-model-name \1/p; + s/.*L1d cache:[[:space:]]\+\(.*\)$/metric lscpu-l1d-cache \1/p; + s/.*L1i cache:[[:space:]]\+\(.*\)$/metric lscpu-l1i-cache \1/p; + s/.*L2 cache:[[:space:]]\+\(.*\)$/metric lscpu-l2-cache \1/p; + s/.*L3 cache:[[:space:]]\+\(.*\)$/metric lscpu-l3-cache \1/p; + s/.*left_sha=\(.*\)$/old-sha \1/p; + s/.*right_sha=\(.*\)/new-sha \1/p' \ + | awk ' + BEGIN { FS = "\t"; OFS = "\t" } + /^old-sha/ { old_sha=$2 } + /^new-sha/ { new_sha=$2 } + /^metric/ { print old_sha, new_sha, $2, $3 }' \ + | "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" + + # Grepping numactl results from log is too crazy, I'll just call it again. + "${client[@]}" --query "INSERT INTO run_attributes_v1 FORMAT TSV" < /dev/null +clickhouse-client --version > /dev/null + +case "$stage" in +"") + ;& +"configure") + time configure + ;& +"restart") + numactl --show ||: + numactl --hardware ||: + lscpu ||: + dmidecode -t 4 ||: + time restart + ;& +"run_tests") + # Ignore the errors to collect the log and build at least some report, anyway + time run_tests ||: + ;& +"get_profiles") + # Check for huge pages. + cat /sys/kernel/mm/transparent_hugepage/enabled > thp-enabled.txt ||: + cat /proc/meminfo > meminfo.txt ||: + for pid in $(pgrep -f clickhouse-server) + do + cat "/proc/$pid/smaps" > "$pid-smaps.txt" ||: + done + + # We had a bug where getting profiles froze sometimes, so try to save some + # logs if this happens again. Give the servers some time to collect all info, + # then trace and kill. Start in a subshell, so that both function don't + # interfere with each other's jobs through `wait`. Also make the subshell + # have its own process group, so that we can then kill it with all its child + # processes. Somehow it doesn't kill the children by itself when dying. + set -m + ( get_profiles_watchdog ) & + watchdog_pid=$! + set +m + # Check that the watchdog started OK. + kill -0 $watchdog_pid + + # If the tests fail with OOM or something, still try to restart the servers + # to collect the logs. Prefer not to restart, because addresses might change + # and we won't be able to process trace_log data. Start in a subshell, so that + # it doesn't interfere with the watchdog through `wait`. + ( get_profiles || { restart && get_profiles ; } ) ||: + + # Kill the whole process group, because somehow when the subshell is killed, + # the sleep inside remains alive and orphaned. + while env kill -- -$watchdog_pid ; do sleep 1; done + + # Stop the servers to free memory for the subsequent query analysis. + while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done + echo Servers stopped. + ;& +"analyze_queries") + time analyze_queries ||: + ;& +"report") + time report ||: + ;& +"report_metrics") + time report_metrics ||: + cat metrics/errors.log >> report/errors.log ||: + ;& +"report_html") + time "$script_dir/report.py" --report=all-queries > all-queries.html 2> >(tee -a report/errors.log 1>&2) ||: + time "$script_dir/report.py" > report.html + ;& +"upload_results") + time upload_results ||: + ;& +esac + +# Print some final debug info to help debug Weirdness, of which there is plenty. +jobs +pstree -apgT diff --git a/ci/jobs/scripts/performance_test.sh b/ci/jobs/scripts/performance_test.sh new file mode 100755 index 00000000000..9e0f20a9a9e --- /dev/null +++ b/ci/jobs/scripts/performance_test.sh @@ -0,0 +1,200 @@ +#!/bin/bash + +set -e +x + +CHPC_CHECK_START_TIMESTAMP="$(date +%s)" +export CHPC_CHECK_START_TIMESTAMP + +S3_URL=${S3_URL:="https://clickhouse-builds.s3.amazonaws.com"} +BUILD_NAME=${BUILD_NAME:-package_release} +export S3_URL BUILD_NAME +SCRIPT_DIR="$(dirname "$(readlink -f "$0")")" + +# Sometimes AWS responds with DNS error and it's impossible to retry it with +# current curl version options. +function curl_with_retry +{ + for _ in 1 2 3 4 5 6 7 8 9 10; do + if curl --fail --head "$1" + then + return 0 + else + sleep 1 + fi + done + return 1 +} + +# Use the packaged repository to find the revision we will compare to. +function find_reference_sha +{ + git -C right/ch log -1 origin/master + git -C right/ch log -1 pr + # Go back from the revision to be tested, trying to find the closest published + # testing release. The PR branch may be either pull/*/head which is the + # author's branch, or pull/*/merge, which is head merged with some master + # automatically by Github. We will use a merge base with master as a reference + # for tesing (or some older commit). A caveat is that if we're testing the + # master, the merge base is the tested commit itself, so we have to step back + # once. + start_ref=$(git -C right/ch merge-base origin/master pr) + if [ "$PR_TO_TEST" == "0" ] + then + start_ref=$start_ref~ + fi + + # Loop back to find a commit that actually has a published perf test package. + while : + do + # FIXME the original idea was to compare to a closest testing tag, which + # is a version that is verified to work correctly. However, we're having + # some test stability issues now, and the testing release can't roll out + # for more that a weak already because of that. Temporarily switch to + # using just closest master, so that we can go on. + #ref_tag=$(git -C ch describe --match='v*-testing' --abbrev=0 --first-parent "$start_ref") + ref_tag="$start_ref" + + echo Reference tag is "$ref_tag" + # We use annotated tags which have their own shas, so we have to further + # dereference the tag to get the commit it points to, hence the '~0' thing. + REF_SHA=$(git -C right/ch rev-parse "$ref_tag~0") + + # FIXME sometimes we have testing tags on commits without published builds. + # Normally these are documentation commits. Loop to skip them. + # Historically there were various path for the performance test package, + # test all of them. + unset found + declare -a urls_to_try=( + "$S3_URL/PRs/0/$REF_SHA/$BUILD_NAME/performance.tar.zst" + "$S3_URL/0/$REF_SHA/$BUILD_NAME/performance.tar.zst" + "$S3_URL/0/$REF_SHA/$BUILD_NAME/performance.tgz" + ) + for path in "${urls_to_try[@]}" + do + if curl_with_retry "$path" + then + found="$path" + break + fi + done + if [ -n "$found" ] ; then break; fi + + start_ref="$REF_SHA~" + done + + REF_PR=0 +} + +#chown nobody workspace output +#chgrp nogroup workspace output +#chmod 777 workspace output + +#[ ! -e "/artifacts/performance.tar.zst" ] && echo "ERROR: performance.tar.zst not found" && exit 1 +#mkdir -p right +#tar -xf "/artifacts/performance.tar.zst" -C right --no-same-owner --strip-components=1 --zstd --extract --verbose + +## Find reference revision if not specified explicitly +#if [ "$REF_SHA" == "" ]; then find_reference_sha; fi +#if [ "$REF_SHA" == "" ]; then echo Reference SHA is not specified ; exit 1 ; fi +#if [ "$REF_PR" == "" ]; then echo Reference PR is not specified ; exit 1 ; fi + +# Show what we're testing +#( +# git -C right/ch log -1 --decorate "$REF_SHA" ||: +#) | tee left-commit.txt +# +#( +# git -C right/ch log -1 --decorate "$SHA_TO_TEST" ||: +# echo +# echo Real tested commit is: +# git -C right/ch log -1 --decorate "pr" +#) | tee right-commit.txt + +#if [ "$PR_TO_TEST" != "0" ] +#then +# # If the PR only changes the tests and nothing else, prepare a list of these +# # tests for use by compare.sh. Compare to merge base, because master might be +# # far in the future and have unrelated test changes. +# base=$(git -C right/ch merge-base pr origin/master) +# git -C right/ch diff --name-only "$base" pr -- . | tee all-changed-files.txt +# git -C right/ch diff --name-only --diff-filter=d "$base" pr -- tests/performance/*.xml | tee changed-test-definitions.txt +# git -C right/ch diff --name-only "$base" pr -- :!tests/performance/*.xml :!docker/test/performance-comparison | tee other-changed-files.txt +#fi + +# prepare config for the right server +export PATH="/tmp/praktika/input:$PATH" +rm -rf /tmp/praktika/right/config && mkdir -p /tmp/praktika/right/config +cp -r ./tests/config /tmp/praktika/right/config +cp ./programs/server/config.xml /tmp/praktika/right/config/ +cd /tmp/praktika/input +chmod +x clickhouse +ln -sf clickhouse clickhouse-local +ln -sf clickhouse clickhouse-client +#for file in /tmp/praktika/right/config/config.d/*.xml; do [ -f $file ] && echo Change config $file && sed -i 's|>/var/log|>/tmp/praktika/right/var/log|g; s|>/etc/|>/tmp/praktika/right/etc/|g' $(readlink -f $file); done +cd - + + +# prepare config for the left server +left_sha=$(sed -n 's/SET(VERSION_GITHASH \(.*\))/\1/p' cmake/autogenerated_versions.txt) +version_major=$(sed -n 's/SET(VERSION_MAJOR \(.*\))/\1/p' cmake/autogenerated_versions.txt) +version_minor=$(sed -n 's/SET(VERSION_MINOR \(.*\))/\1/p' cmake/autogenerated_versions.txt) +rm -rf /tmp/praktika/left/config && mkdir -p /tmp/praktika/left/config +#git checkout left_sha +#rm -rf /tmp/praktika/left && mkdir -p /tmp/praktika/left +#cp -r ./tests/config /tmp/praktika/left/config +#git checkout - +cd /tmp/praktika/left +[ ! -f clickhouse ] && wget -nv https://clickhouse-builds.s3.us-east-1.amazonaws.com/$version_major.$version_minor/020d843058ae211c43285852e5f4f0e0e9cc1eb6/package_aarch64/clickhouse +chmod +x clickhouse +ln -sf clickhouse clickhouse-local +ln -sf clickhouse clickhouse-client +ln -sf clickhouse clickhouse-server +cd - + + +# Set python output encoding so that we can print queries with non-ASCII letters. +export PYTHONIOENCODING=utf-8 + +script_path="tests/performance/scripts/" + +## Even if we have some errors, try our best to save the logs. +#set +e + +# Use clickhouse-client and clickhouse-local from the right server. + + +export REF_PR +export REF_SHA + +# Try to collect some core dumps. +# At least we remove the ulimit and then try to pack some common file names into output. +ulimit -c unlimited +cat /proc/sys/kernel/core_pattern + +# Start the main comparison script. +{ +# time $SCRIPT_DIR/download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ + time stage=configure ./ci/jobs/scripts/performance_compare.sh ; \ +} 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee -a compare.log + +# Stop the servers to free memory. Normally they are restarted before getting +# the profile info, so they shouldn't use much, but if the comparison script +# fails in the middle, this might not be the case. +for _ in {1..30} +do + killall clickhouse || break + sleep 1 +done + +dmesg -T > dmesg.log + +ls -lath + +7z a '-x!*/tmp' /output/output.7z ./*.{log,tsv,html,txt,rep,svg,columns} \ + {right,left}/{performance,scripts} {{right,left}/db,db0}/preprocessed_configs \ + report analyze benchmark metrics \ + ./*.core.dmp ./*.core + +# If the files aren't same, copy it +cmp --silent compare.log /output/compare.log || \ + cp compare.log /output diff --git a/ci/workflows/defs.py b/ci/workflows/defs.py index 6aadc875512..30a481aa778 100644 --- a/ci/workflows/defs.py +++ b/ci/workflows/defs.py @@ -246,6 +246,7 @@ class JobNames: STATELESS = "Stateless tests" STATEFUL = "Stateful tests" STRESS = "Stress tests" + PERFORMANCE = "Performance tests" class ToolSet: @@ -569,3 +570,18 @@ class Jobs: [ArtifactNames.DEB_ARM_RELEASE], ], ) + + performance_test_job = Job.Config( + name=JobNames.PERFORMANCE, + runs_on=[RunnerLabels.FUNC_TESTER_ARM], + command="./ci/jobs/scripts/performance_test.sh", + run_in_docker="clickhouse/stateless-test", + requires=[ArtifactNames.CH_ARM_RELEASE] + # digest_config=Job.CacheDigestConfig( + # include_paths=[ + # "./ci/jobs/fast_test.py", + # "./tests/queries/0_stateless/", + # "./src", + # ], + # ), + ) diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 3bcab79470b..1d3e67dd461 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -13,6 +13,7 @@ workflow = Workflow.Config( *Jobs.stateless_tests_jobs, *Jobs.stateful_tests_jobs, *Jobs.stress_test_jobs, + Jobs.performance_test_job, ], artifacts=ARTIFACTS, dockers=DOCKERS, From 115df8705dcb8a6e7bfc09e1565f1c13f9eb4af8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 14:14:32 +0100 Subject: [PATCH 264/433] Move dictionary sorting out of template method --- src/Columns/ColumnLowCardinality.cpp | 26 +++++++++++++------------- src/Columns/ColumnLowCardinality.h | 4 ++-- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 2317cca01fa..284b00d621a 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -460,19 +460,12 @@ struct LowCardinalityComparator template void ColumnLowCardinality::updatePermutationWithIndexType( - IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const + IColumn::PermutationSortStability stability, size_t limit, const PaddedPODArray & position_by_index, + IColumn::Permutation & res, EqualRanges & equal_ranges) const { /// Cast indexes column to the real type so that compareAt and getUInt methods can be inlined. const IndexColumn * real_indexes = assert_cast(&getIndexes()); - IColumn::Permutation dict_perm; - getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); - - PaddedPODArray position_by_index(dict_perm.size()); - for (size_t i = 0; i < dict_perm.size(); ++i) - position_by_index[dict_perm[i]] = i; - auto equal_comparator = [real_indexes](size_t lhs, size_t rhs) { return real_indexes->getUInt(lhs) == real_indexes->getUInt(rhs); @@ -488,20 +481,27 @@ void ColumnLowCardinality::updatePermutationWithIndexType( void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const { + IColumn::Permutation dict_perm; + getDictionary().getNestedColumn()->getPermutation(direction, stability, 0, nan_direction_hint, dict_perm); + + PaddedPODArray position_by_index(dict_perm.size()); + for (size_t i = 0; i < dict_perm.size(); ++i) + position_by_index[dict_perm[i]] = i; + /// Dispatch by index column type. switch (idx.getSizeOfIndexType()) { case sizeof(UInt8): - updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(stability, limit, position_by_index, res, equal_ranges); return; case sizeof(UInt16): - updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(stability, limit, position_by_index, res, equal_ranges); return; case sizeof(UInt32): - updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(stability, limit, position_by_index, res, equal_ranges); return; case sizeof(UInt64): - updatePermutationWithIndexType(direction, stability, limit, nan_direction_hint, res, equal_ranges); + updatePermutationWithIndexType(stability, limit, position_by_index, res, equal_ranges); return; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of index type for low cardinality column."); } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 7380b5eff07..e09c51e2054 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -392,8 +392,8 @@ private: template void updatePermutationWithIndexType( - IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, - size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const; + IColumn::PermutationSortStability stability, size_t limit, const PaddedPODArray & position_by_index, + IColumn::Permutation & res, EqualRanges & equal_ranges) const; }; bool isColumnLowCardinalityNullable(const IColumn & column); From cc0f8271e21a30c5aec495060533cac7ef44e3d6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 25 Nov 2024 13:14:55 +0000 Subject: [PATCH 265/433] add a warning to lagInFrame/leadInFrame --- .../window-functions/lagInFrame.md | 18 +++++++--- .../window-functions/leadInFrame.md | 36 ++++++++++++++----- 2 files changed, 40 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/window-functions/lagInFrame.md b/docs/en/sql-reference/window-functions/lagInFrame.md index 01bf809e76e..c4b7b377761 100644 --- a/docs/en/sql-reference/window-functions/lagInFrame.md +++ b/docs/en/sql-reference/window-functions/lagInFrame.md @@ -8,11 +8,17 @@ sidebar_position: 9 Returns a value evaluated at the row that is at a specified physical offset row before the current row within the ordered frame. +:::warning +`lagInFrame` behavior differs from the standard SQL `lag` window function. +Clickhouse window function `lagInFrame` respects the window frame. +To get behavior identical to the `lag`, use `rows between unbounded preceding and unbounded following`. +::: + **Syntax** ```sql lagInFrame(x[, offset[, default]]) - OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) @@ -21,7 +27,7 @@ WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column] For more detail on window function syntax see: [Window Functions - Syntax](./index.md/#syntax). **Parameters** -- `x` — Column name. +- `x` — Column name. - `offset` — Offset to apply. [(U)Int*](../data-types/int-uint.md). (Optional - `1` by default). - `default` — Value to return if calculated row exceeds the boundaries of the window frame. (Optional - default value of column type when omitted). @@ -59,11 +65,13 @@ INSERT INTO stock_prices FORMAT Values SELECT date, close, - lagInFrame(close, 1, close) OVER (ORDER BY date ASC) AS previous_day_close, + lagInFrame(close, 1, close) OVER (ORDER BY date ASC + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) AS previous_day_close, COALESCE(ROUND(close - previous_day_close, 2)) AS delta, COALESCE(ROUND((delta / previous_day_close) * 100, 2)) AS percent_change FROM stock_prices -ORDER BY date DESC; +ORDER BY date DESC ``` Result: @@ -76,4 +84,4 @@ Result: 4. │ 2024-06-04 │ 116.44 │ 115 │ 1.44 │ 1.25 │ 5. │ 2024-06-03 │ 115 │ 115 │ 0 │ 0 │ └────────────┴────────┴────────────────────┴───────┴────────────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/window-functions/leadInFrame.md b/docs/en/sql-reference/window-functions/leadInFrame.md index dae4353b582..16c7aefd81a 100644 --- a/docs/en/sql-reference/window-functions/leadInFrame.md +++ b/docs/en/sql-reference/window-functions/leadInFrame.md @@ -8,11 +8,17 @@ sidebar_position: 10 Returns a value evaluated at the row that is offset rows after the current row within the ordered frame. +:::warning +`leadInFrame` behavior differs from the standard SQL `lead` window function. +Clickhouse window function `leadInFrame` respects the window frame. +To get behavior identical to the `lead`, use `rows between unbounded preceding and unbounded following`. +::: + **Syntax** ```sql leadInFrame(x[, offset[, default]]) - OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] + OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] [ROWS or RANGE expression_to_bound_rows_withing_the_group]] | [window_name]) FROM table_name WINDOW window_name as ([[PARTITION BY grouping_column] [ORDER BY sorting_column]) @@ -36,25 +42,37 @@ This example looks at [historical data](https://www.kaggle.com/datasets/sazidthe Query: ```sql -CREATE OR REPLACE VIEW nobel_prize_laureates AS FROM file('nobel_laureates_data.csv') SELECT *; +CREATE OR REPLACE VIEW nobel_prize_laureates +AS SELECT * +FROM file('nobel_laureates_data.csv'); ``` ```sql -FROM nobel_prize_laureates SELECT fullName, leadInFrame(year, 1, year) OVER (PARTITION BY category ORDER BY year) AS year, category, motivation WHERE category == 'physics' ORDER BY year DESC LIMIT 9; +SELECT + fullName, + leadInFrame(year, 1, year) OVER (PARTITION BY category ORDER BY year ASC + ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) AS year, + category, + motivation +FROM nobel_prize_laureates +WHERE category = 'physics' +ORDER BY year DESC +LIMIT 9 ``` Result: ```response ┌─fullName─────────┬─year─┬─category─┬─motivation─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -1. │ Pierre Agostini │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ -2. │ Ferenc Krausz │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ -3. │ Anne L Huillier │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +1. │ Anne L Huillier │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +2. │ Pierre Agostini │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ +3. │ Ferenc Krausz │ 2023 │ physics │ for experimental methods that generate attosecond pulses of light for the study of electron dynamics in matter │ 4. │ Alain Aspect │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ 5. │ Anton Zeilinger │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ 6. │ John Clauser │ 2022 │ physics │ for experiments with entangled photons establishing the violation of Bell inequalities and pioneering quantum information science │ -7. │ Syukuro Manabe │ 2021 │ physics │ for the physical modelling of Earths climate quantifying variability and reliably predicting global warming │ +7. │ Giorgio Parisi │ 2021 │ physics │ for the discovery of the interplay of disorder and fluctuations in physical systems from atomic to planetary scales │ 8. │ Klaus Hasselmann │ 2021 │ physics │ for the physical modelling of Earths climate quantifying variability and reliably predicting global warming │ -9. │ Giorgio Parisi │ 2021 │ physics │ for the discovery of the interplay of disorder and fluctuations in physical systems from atomic to planetary scales │ +9. │ Syukuro Manabe │ 2021 │ physics │ for the physical modelling of Earths climate quantifying variability and reliably predicting global warming │ └──────────────────┴──────┴──────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` \ No newline at end of file +``` From fc61f33bea945099c27759f49cfadcbc19a51cde Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 14:29:29 +0100 Subject: [PATCH 266/433] Move all debug code under #ifndef NDEBUG --- src/Interpreters/sortBlock.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 3ebfde8b7a3..1c4754b1ef2 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -16,7 +16,9 @@ namespace DB namespace ErrorCodes { extern const int BAD_COLLATION; +#ifndef NDEBUG extern const int LOGICAL_ERROR; +#endif } /// Column with description for sort @@ -273,6 +275,7 @@ bool isAlreadySortedImpl(size_t rows, Comparator compare) return true; } +#ifndef NDEBUG template void checkSortedWithPermutationImpl(size_t rows, Comparator compare, UInt64 limit, const IColumn::Permutation & permutation) { @@ -322,6 +325,7 @@ void checkSortedWithPermutation(const Block & block, const SortDescription & des return; } } +#endif } From 8f5a939d65e5ff6e318c24eee5497624db01fe8d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 25 Nov 2024 13:33:33 +0000 Subject: [PATCH 267/433] Try to fix link --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index c5b015cddfb..fcecce2f973 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -23,8 +23,8 @@ LIMIT N where - `DistanceFunction` computes a distance between two vectors (e.g. the - [L2Distance](../../../sql-referenc/functions/distance-functions.md#L2Distance) or - [cosineDistance](../../../sql-referenc/functions/distance-functions.md#cosineDistance)), + [L2Distance](../../../sql-reference/functions/distance-functions.md#L2Distance) or + [cosineDistance](../../../sql-reference/functions/distance-functions.md#cosineDistance)), - `vectors` is a column of type [Array(Float64)](../../../sql-reference/data-types/array.md) or [Array(Float32)](../../../sql-reference/data-types/array.md), or [Array(BFloat16)](../../../sql-reference/data-types/array.md), typically storing embeddings, From 6c4c8155d840f0187ebcaaeca727009adaaca2f8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Nov 2024 13:39:28 +0000 Subject: [PATCH 268/433] PR: ANTI JOIN fix --- src/Planner/PlannerJoinTree.cpp | 4 +- ...275_pr_simple_any_semi_anti_join.reference | 38 +++++++++++++++ .../03275_pr_simple_any_semi_anti_join.sql | 47 +++++++++++++++++++ 3 files changed, 88 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference create mode 100644 tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0107df61c22..a3517981481 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -963,7 +963,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres return true; const auto join_kind = join_node->getKind(); - if (join_kind == JoinKind::Left || join_kind == JoinKind::Right || join_kind == JoinKind::Inner) + const auto join_strictness = join_node->getStrictness(); + if (join_kind == JoinKind::Left || join_kind == JoinKind::Right + || (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All)) return true; return false; diff --git a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference b/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference new file mode 100644 index 00000000000..b48b20942c6 --- /dev/null +++ b/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference @@ -0,0 +1,38 @@ +any left +0 a1 0 +1 a2 0 +2 a3 2 b1 +3 a4 0 +4 a5 4 b2 +any left (rev) +0 5 b4 +2 a3 2 b1 +4 a5 4 b2 +any inner +2 a3 2 b1 +4 a5 4 b2 +any inner (rev) +2 a3 2 b1 +4 a5 4 b2 +any right +0 5 b4 +2 a3 2 b1 +4 a5 4 b2 +any right (rev) +0 a1 0 +1 a2 0 +2 a3 2 b1 +3 a4 0 +4 a5 4 b2 +semi left +2 a3 2 b1 +4 a5 4 b2 +semi right +2 a3 2 b1 +4 a5 4 b2 +anti left +0 a1 0 +1 a2 1 +3 a4 3 +anti right +0 5 b4 diff --git a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql b/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql new file mode 100644 index 00000000000..6e92f70c151 --- /dev/null +++ b/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS t1 SYNC; +DROP TABLE IF EXISTS t2 SYNC; + +CREATE TABLE t1 (x UInt32, s String) engine = ReplicatedMergeTree('/clickhouse/{database}/t1', 'r1') ORDER BY tuple(); +CREATE TABLE t2 (x UInt32, s String) engine = ReplicatedMergeTree('/clickhouse/{database}/t2', 'r1') ORDER BY tuple(); + +INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); +INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b2'), (5, 'b4'); + +SET join_algorithm = 'prefer_partial_merge'; +SET join_use_nulls = 0; +SET any_join_distinct_right_table_keys = 0; + +set enable_analyzer = 1, enable_parallel_replicas = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; + +SELECT 'any left'; +SELECT t1.*, t2.* FROM t1 ANY LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any left (rev)'; +SELECT t1.*, t2.* FROM t2 ANY LEFT JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any inner'; +SELECT t1.*, t2.* FROM t1 ANY INNER JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any inner (rev)'; +SELECT t1.*, t2.* FROM t2 ANY INNER JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any right'; +SELECT t1.*, t2.* FROM t1 ANY RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any right (rev)'; +SELECT t1.*, t2.* FROM t2 ANY RIGHT JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'semi left'; +SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'semi right'; +SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'anti left'; +SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'anti right'; +SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +-- DROP TABLE t1 SYNC; +-- DROP TABLE t2 SYNC; From bf23cc865e16cf07ed81186042c8b7b9c1ac2756 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 25 Nov 2024 13:47:06 +0000 Subject: [PATCH 269/433] Fix data race in Squashing with LowCardinality --- src/Columns/ColumnLowCardinality.h | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 3cc1c8919c0..1e9d576b423 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -190,6 +190,26 @@ public: callback(dictionary.getColumnUniquePtr()); } + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override + { + /** It is important to have both const and non-const versions here. + * The behavior of ColumnUnique::forEachSubcolumnRecursively differs between const and non-const versions. + * The non-const version will update a field in ColumnUnique. + * In the meantime, the default implementation IColumn::forEachSubcolumnRecursively uses const_cast, + * so when the const version is called, the field will still be mutated. + * This can lead to a data race if constness is expected. + */ + callback(*idx.getPositionsPtr()); + idx.getPositionsPtr()->forEachSubcolumnRecursively(callback); + + /// Column doesn't own dictionary if it's shared. + if (!dictionary.isShared()) + { + callback(*dictionary.getColumnUniquePtr()); + dictionary.getColumnUniquePtr()->forEachSubcolumnRecursively(callback); + } + } + void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) override { callback(*idx.getPositionsPtr()); From 517d8093b5b798056f5db4df92e21c4e265f27c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Nov 2024 13:52:24 +0000 Subject: [PATCH 270/433] Only ANY test --- ....reference => 03275_pr_any_join.reference} | 12 ----------- ...mi_anti_join.sql => 03275_pr_any_join.sql} | 20 ++----------------- 2 files changed, 2 insertions(+), 30 deletions(-) rename tests/queries/0_stateless/{03275_pr_simple_any_semi_anti_join.reference => 03275_pr_any_join.reference} (69%) rename tests/queries/0_stateless/{03275_pr_simple_any_semi_anti_join.sql => 03275_pr_any_join.sql} (68%) diff --git a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference b/tests/queries/0_stateless/03275_pr_any_join.reference similarity index 69% rename from tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference rename to tests/queries/0_stateless/03275_pr_any_join.reference index b48b20942c6..f9821398047 100644 --- a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.reference +++ b/tests/queries/0_stateless/03275_pr_any_join.reference @@ -24,15 +24,3 @@ any right (rev) 2 a3 2 b1 3 a4 0 4 a5 4 b2 -semi left -2 a3 2 b1 -4 a5 4 b2 -semi right -2 a3 2 b1 -4 a5 4 b2 -anti left -0 a1 0 -1 a2 1 -3 a4 3 -anti right -0 5 b4 diff --git a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql b/tests/queries/0_stateless/03275_pr_any_join.sql similarity index 68% rename from tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql rename to tests/queries/0_stateless/03275_pr_any_join.sql index 6e92f70c151..6a3daff95b1 100644 --- a/tests/queries/0_stateless/03275_pr_simple_any_semi_anti_join.sql +++ b/tests/queries/0_stateless/03275_pr_any_join.sql @@ -7,10 +7,6 @@ CREATE TABLE t2 (x UInt32, s String) engine = ReplicatedMergeTree('/clickhouse/{ INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b2'), (5, 'b4'); -SET join_algorithm = 'prefer_partial_merge'; -SET join_use_nulls = 0; -SET any_join_distinct_right_table_keys = 0; - set enable_analyzer = 1, enable_parallel_replicas = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; SELECT 'any left'; @@ -31,17 +27,5 @@ SELECT t1.*, t2.* FROM t1 ANY RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; SELECT 'any right (rev)'; SELECT t1.*, t2.* FROM t2 ANY RIGHT JOIN t1 USING(x) ORDER BY t1.x, t2.x; -SELECT 'semi left'; -SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; - -SELECT 'semi right'; -SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; - -SELECT 'anti left'; -SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; - -SELECT 'anti right'; -SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; - --- DROP TABLE t1 SYNC; --- DROP TABLE t2 SYNC; +DROP TABLE t1 SYNC; +DROP TABLE t2 SYNC; From 0073a74881d6392ca4a263dc79ca0a1d6a65333e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 25 Nov 2024 14:09:11 +0000 Subject: [PATCH 271/433] fix mismatched types after optimize_function_to_subcolumns --- .../Passes/FunctionToSubcolumnsPass.cpp | 25 +++++++++++++------ ...03276_functions_to_subcolumns_lc.reference | 1 + .../03276_functions_to_subcolumns_lc.sql | 14 +++++++++++ 3 files changed, 33 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03276_functions_to_subcolumns_lc.reference create mode 100644 tests/queries/0_stateless/03276_functions_to_subcolumns_lc.sql diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index abd0a95c6f2..92dc0301fd6 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -46,7 +46,7 @@ using NodeToSubcolumnTransformer = std::function()}; @@ -56,8 +56,8 @@ void optimizeFunctionLength(QueryTreeNodePtr & node, FunctionNode &, ColumnConte template void optimizeFunctionEmpty(QueryTreeNodePtr &, FunctionNode & function_node, ColumnContext & ctx) { - /// Replace `empty(argument)` with `equals(argument.size0, 0)` if positive - /// Replace `notEmpty(argument)` with `notEquals(argument.size0, 0)` if not positive + /// Replace `empty(argument)` with `equals(argument.size0, 0)` if positive. + /// Replace `notEmpty(argument)` with `notEquals(argument.size0, 0)` if not positive. /// `argument` may be Array or Map. NameAndTypePair column{ctx.column.name + ".size0", std::make_shared()}; @@ -136,19 +136,25 @@ std::map, NodeToSubcolumnTransformer> node_transfor }, { {TypeIndex::Map, "mapKeys"}, - [](QueryTreeNodePtr & node, FunctionNode & function_node, ColumnContext & ctx) + [](QueryTreeNodePtr & node, FunctionNode &, ColumnContext & ctx) { /// Replace `mapKeys(map_argument)` with `map_argument.keys` - NameAndTypePair column{ctx.column.name + ".keys", function_node.getResultType()}; + const auto & data_type_map = assert_cast(*ctx.column.type); + auto key_type = std::make_shared(data_type_map.getKeyType()); + + NameAndTypePair column{ctx.column.name + ".keys", key_type}; node = std::make_shared(column, ctx.column_source); }, }, { {TypeIndex::Map, "mapValues"}, - [](QueryTreeNodePtr & node, FunctionNode & function_node, ColumnContext & ctx) + [](QueryTreeNodePtr & node, FunctionNode &, ColumnContext & ctx) { /// Replace `mapValues(map_argument)` with `map_argument.values` - NameAndTypePair column{ctx.column.name + ".values", function_node.getResultType()}; + const auto & data_type_map = assert_cast(*ctx.column.type); + auto value_type = std::make_shared(data_type_map.getValueType()); + + NameAndTypePair column{ctx.column.name + ".values", value_type}; node = std::make_shared(column, ctx.column_source); }, }, @@ -439,11 +445,16 @@ public: if (!identifiers_to_optimize.contains(qualified_name)) return; + auto result_type = function_node->getResultType(); auto transformer_it = node_transformers.find({column.type->getTypeId(), function_node->getFunctionName()}); + if (transformer_it != node_transformers.end()) { ColumnContext ctx{std::move(column), first_argument_column_node->getColumnSource(), getContext()}; transformer_it->second(node, *function_node, ctx); + + if (!result_type->equals(*node->getResultType())) + node = buildCastFunction(node, result_type, getContext()); } } }; diff --git a/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.reference b/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.reference new file mode 100644 index 00000000000..3bc835eaeac --- /dev/null +++ b/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.reference @@ -0,0 +1 @@ +['foo'] ['bar'] diff --git a/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.sql b/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.sql new file mode 100644 index 00000000000..b3b8c1a79f8 --- /dev/null +++ b/tests/queries/0_stateless/03276_functions_to_subcolumns_lc.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS t_map_lc; + +CREATE TABLE t_map_lc +( + kv Map(LowCardinality(String), LowCardinality(String)), + k Array(LowCardinality(String)) ALIAS mapKeys(kv), + v Array(LowCardinality(String)) ALIAS mapValues(kv) +) ENGINE = Memory; + +INSERT INTO t_map_lc VALUES (map('foo', 'bar')); + +SELECT k, v FROM t_map_lc SETTINGS optimize_functions_to_subcolumns=1; + +DROP TABLE t_map_lc; From 207cc837f5103e27a45296db89aee9eddb69175a Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Nov 2024 15:29:03 +0100 Subject: [PATCH 272/433] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 4f468ac600e..eb5b939469c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -91,6 +91,9 @@ static std::initializer_list Date: Mon, 25 Nov 2024 15:29:34 +0100 Subject: [PATCH 273/433] make d-tor Finalizer more obvious --- src/Storages/MergeTree/MergeTreeSink.cpp | 2 ++ .../MergeTree/MergedBlockOutputStream.cpp | 17 +++++------------ .../MergeTree/MergedBlockOutputStream.h | 2 +- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 99852309c77..a4fd8a5ed33 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -44,6 +44,8 @@ MergeTreeSink::~MergeTreeSink() if (!delayed_chunk) return; + chassert(isCancelled()); + for (auto & partition : delayed_chunk->partitions) { partition.temp_part.cancel(); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 979b4698738..d1eb488371e 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -113,7 +113,7 @@ struct MergedBlockOutputStream::Finalizer::Impl } void finish(); - void cancel(); + void cancel() noexcept; }; void MergedBlockOutputStream::Finalizer::finish() @@ -124,7 +124,7 @@ void MergedBlockOutputStream::Finalizer::finish() to_finish->finish(); } -void MergedBlockOutputStream::Finalizer::cancel() +void MergedBlockOutputStream::Finalizer::cancel() noexcept { std::unique_ptr to_cancel = std::move(impl); impl.reset(); @@ -161,7 +161,7 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() part->getDataPartStorage().removeFile(file_name); } -void MergedBlockOutputStream::Finalizer::Impl::cancel() +void MergedBlockOutputStream::Finalizer::Impl::cancel() noexcept { writer.cancel(); @@ -177,15 +177,8 @@ MergedBlockOutputStream::Finalizer::Finalizer(std::unique_ptr impl_) : imp MergedBlockOutputStream::Finalizer::~Finalizer() { - try - { - if (impl) - finish(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + if (impl) + cancel(); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index afa2eaf18ec..f3a60d29a6e 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -55,7 +55,7 @@ public: ~Finalizer(); void finish(); - void cancel(); + void cancel() noexcept; }; /// Finalize writing part and fill inner structures From 7eda31550f00cc5280de7e6ef0eaf6d9952e8206 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 25 Nov 2024 14:41:03 +0000 Subject: [PATCH 274/433] Fix bug with two errors --- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 4eadb158ffa..c16f11043d2 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -346,6 +346,8 @@ std::pair parseTableSchema( } catch (const Exception & first_error) { + if (first_error.code() != ErrorCodes::BAD_ARGUMENTS) + throw; try { std::tie(schema, current_schema_id) = parseTableSchemaV2Method(metadata_object, ignore_schema_evolution); @@ -357,6 +359,8 @@ std::pair parseTableSchema( } catch (const Exception & second_error) { + if (first_error.code() != ErrorCodes::BAD_ARGUMENTS) + throw; throw Exception( ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema both with v1 and v2 methods. Old method error: {}. New method error: {}", From 69b4743b13e575f936f90c1c0912c98eabd04868 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Mon, 25 Nov 2024 16:07:45 +0100 Subject: [PATCH 275/433] Fix env var name --- src/IO/S3/Credentials.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/IO/S3/Credentials.cpp b/src/IO/S3/Credentials.cpp index cde9a7a3662..03068258e1a 100644 --- a/src/IO/S3/Credentials.cpp +++ b/src/IO/S3/Credentials.cpp @@ -695,7 +695,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN"; - static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN_PATH"; + static const char AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_FILE[] = "AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE"; static const char AWS_EC2_METADATA_DISABLED[] = "AWS_EC2_METADATA_DISABLED"; /// The only difference from DefaultAWSCredentialsProviderChain::DefaultAWSCredentialsProviderChain() @@ -754,11 +754,11 @@ S3CredentialsProviderChain::S3CredentialsProviderChain( else if (!absolute_uri.empty()) { auto token = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN); - const auto token_path = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH); + const auto token_path = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_FILE); if (!token_path.empty()) { - LOG_INFO(logger, "The environment variable value {} is {}", AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_PATH, token_path); + LOG_INFO(logger, "The environment variable value {} is {}", AWS_ECS_CONTAINER_AUTHORIZATION_TOKEN_FILE, token_path); String token_from_file; From 0b22faa97366f46667355399e4b42fe0a42b6c69 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Nov 2024 16:13:02 +0100 Subject: [PATCH 276/433] Rename setting in error message --- src/DataTypes/DataTypeObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index eaae7ffebca..9a60d1e55b8 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -550,7 +550,7 @@ static DataTypePtr createJSON(const ASTPtr & arguments) if (context->getSettingsRef()[Setting::allow_experimental_object_type] && context->getSettingsRef()[Setting::use_json_alias_for_old_object_type]) { if (arguments && !arguments->children.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set settings allow_experimental_json_type = 1 and use_json_alias_for_old_object_type = 0"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set settings enable_json_type = 1 and use_json_alias_for_old_object_type = 0"); return std::make_shared("JSON", false); } From 84fc0fa6fdb96306b9db44dc6e73e9da1dfd760a Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Nov 2024 16:14:20 +0100 Subject: [PATCH 277/433] Update parseColumnsListForTableFunction.h --- src/Interpreters/parseColumnsListForTableFunction.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.h b/src/Interpreters/parseColumnsListForTableFunction.h index 39b9f092d89..f44e8d0c398 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.h +++ b/src/Interpreters/parseColumnsListForTableFunction.h @@ -19,12 +19,12 @@ struct DataTypeValidationSettings bool allow_suspicious_low_cardinality_types = true; bool allow_experimental_object_type = true; bool allow_suspicious_fixed_string_types = true; - bool allow_experimental_variant_type = true; + bool enable_variant_type = true; bool allow_experimental_bfloat16_type = true; bool allow_suspicious_variant_types = true; bool validate_nested_types = true; - bool allow_experimental_dynamic_type = true; - bool allow_experimental_json_type = true; + bool enable_dynamic_type = true; + bool enable_json_type = true; }; void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings & settings); From c97cdedd9bf38decda521026364b837f068e91a0 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 25 Nov 2024 16:16:12 +0100 Subject: [PATCH 278/433] Rename settings in error messages --- .../parseColumnsListForTableFunction.cpp | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/parseColumnsListForTableFunction.cpp b/src/Interpreters/parseColumnsListForTableFunction.cpp index d3bf6f860f3..a375cb18297 100644 --- a/src/Interpreters/parseColumnsListForTableFunction.cpp +++ b/src/Interpreters/parseColumnsListForTableFunction.cpp @@ -16,10 +16,10 @@ namespace DB { namespace Setting { - extern const SettingsBool allow_experimental_dynamic_type; - extern const SettingsBool allow_experimental_json_type; + extern const SettingsBool enable_dynamic_type; + extern const SettingsBool enable_json_type; extern const SettingsBool allow_experimental_object_type; - extern const SettingsBool allow_experimental_variant_type; + extern const SettingsBool enable_variant_type; extern const SettingsBool allow_experimental_bfloat16_type; extern const SettingsBool allow_suspicious_fixed_string_types; extern const SettingsBool allow_suspicious_low_cardinality_types; @@ -42,12 +42,12 @@ DataTypeValidationSettings::DataTypeValidationSettings(const DB::Settings & sett : allow_suspicious_low_cardinality_types(settings[Setting::allow_suspicious_low_cardinality_types]) , allow_experimental_object_type(settings[Setting::allow_experimental_object_type]) , allow_suspicious_fixed_string_types(settings[Setting::allow_suspicious_fixed_string_types]) - , allow_experimental_variant_type(settings[Setting::allow_experimental_variant_type]) + , enable_variant_type(settings[Setting::enable_variant_type]) , allow_experimental_bfloat16_type(settings[Setting::allow_experimental_bfloat16_type]) , allow_suspicious_variant_types(settings[Setting::allow_suspicious_variant_types]) , validate_nested_types(settings[Setting::validate_experimental_and_suspicious_types_inside_nested_types]) - , allow_experimental_dynamic_type(settings[Setting::allow_experimental_dynamic_type]) - , allow_experimental_json_type(settings[Setting::allow_experimental_json_type]) + , enable_dynamic_type(settings[Setting::enable_dynamic_type]) + , enable_json_type(settings[Setting::enable_json_type]) { } @@ -95,14 +95,14 @@ void validateDataType(const DataTypePtr & type_to_check, const DataTypeValidatio } } - if (!settings.allow_experimental_variant_type) + if (!settings.enable_variant_type) { if (isVariant(data_type)) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because experimental Variant type is not allowed. " - "Set setting allow_experimental_variant_type = 1 in order to allow it", + "Cannot create column with type '{}' because Variant type is not allowed. " + "Set setting enable_variant_type = 1 in order to allow it", data_type.getName()); } } @@ -151,27 +151,27 @@ void validateDataType(const DataTypePtr & type_to_check, const DataTypeValidatio } } - if (!settings.allow_experimental_dynamic_type) + if (!settings.enable_dynamic_type) { if (isDynamic(data_type)) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because experimental Dynamic type is not allowed. " - "Set setting allow_experimental_dynamic_type = 1 in order to allow it", + "Cannot create column with type '{}' because Dynamic type is not allowed. " + "Set setting enable_dynamic_type = 1 in order to allow it", data_type.getName()); } } - if (!settings.allow_experimental_json_type) + if (!settings.enable_json_type) { const auto * object_type = typeid_cast(&data_type); if (object_type && object_type->getSchemaFormat() == DataTypeObject::SchemaFormat::JSON) { throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Cannot create column with type '{}' because experimental JSON type is not allowed. " - "Set setting allow_experimental_json_type = 1 in order to allow it", + "Cannot create column with type '{}' because JSON type is not allowed. " + "Set setting enable_json_type = 1 in order to allow it", data_type.getName()); } } From 32a6b823ef1c7bb57ed9370a52308d93a1698c56 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Nov 2024 16:35:59 +0100 Subject: [PATCH 279/433] Bump From 0b3185fa136eb3cc406e40173a97744905ee4727 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 25 Nov 2024 15:54:52 +0000 Subject: [PATCH 280/433] Docs: Slightly better caption --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index fcecce2f973..e1c7e74a119 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,4 +1,4 @@ -# Approximate Nearest Neighbor Search Indexes [experimental] +# Approximate Nearest Neighbor Search with Vector Similarity Indexes [experimental] Nearest neighborhood search is the problem of finding the M closest vectors to a given vector in an N-dimensional vector space. The most straightforward approach to solve this problem is an exhaustive (brute-force) search which computes the distance between the reference diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 2ca25a044b1..bb445479f97 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -421,7 +421,7 @@ Syntax: `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, ran #### Special-purpose -- Experimental indexes to support approximate nearest neighbor (ANN) search. See [here](annindexes.md) for details. +- An experimental index to support approximate nearest neighbor search. See [here](annindexes.md) for details. - An experimental full-text index to support full-text search. See [here](invertedindexes.md) for details. ### Functions Support {#functions-support} From 3000cef7b8ba09d7b6f4eb001aff269f84d82fd2 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 25 Nov 2024 16:57:25 +0100 Subject: [PATCH 281/433] make operations_to_execute as shared ptr --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 4 ++-- src/Disks/ObjectStorages/DiskObjectStorageTransaction.h | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 19de2bb78af..5314ed42cca 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -768,9 +768,9 @@ std::unique_ptr DiskObjectStorageTransaction::writeFile } else { - auto write_operation = std::make_unique(object_storage, metadata_storage, object); + auto write_operation = std::make_shared(object_storage, metadata_storage, object); - create_metadata_callback = [object_storage_tx = shared_from_this(), write_op = write_operation.get(), mode, path, key_ = std::move(object_key)](size_t count) + create_metadata_callback = [object_storage_tx = shared_from_this(), write_op = write_operation, mode, path, key_ = std::move(object_key)](size_t count) { /// This callback called in WriteBuffer finalize method -- only there we actually know /// how many bytes were written. We don't control when this finalize method will be called diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index 23f66990d54..d69378b8cba 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -34,7 +35,7 @@ public: virtual std::string getInfoForLog() const = 0; }; -using DiskObjectStorageOperation = std::unique_ptr; +using DiskObjectStorageOperation = std::shared_ptr; using DiskObjectStorageOperations = std::vector; From 978b423b5bf726fdcb5def38fb0d0f00dba44ec4 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Nov 2024 16:50:32 +0100 Subject: [PATCH 282/433] add compatibility check --- .github/workflows/pr.yaml | 901 ---------------------- ci/docker/compatibility/centos/Dockerfile | 5 + ci/docker/compatibility/ubuntu/Dockerfile | 5 + ci/jobs/build_clickhouse.py | 7 + ci/praktika/native_jobs.py | 12 +- ci/praktika/runner.py | 10 +- ci/workflows/defs.py | 49 +- ci/workflows/pull_request.py | 3 + tests/ci/compatibility_check.py | 14 +- tests/ci/stress_check.py | 2 +- 10 files changed, 84 insertions(+), 924 deletions(-) delete mode 100644 .github/workflows/pr.yaml create mode 100644 ci/docker/compatibility/centos/Dockerfile create mode 100644 ci/docker/compatibility/ubuntu/Dockerfile diff --git a/.github/workflows/pr.yaml b/.github/workflows/pr.yaml deleted file mode 100644 index a663b3f08ce..00000000000 --- a/.github/workflows/pr.yaml +++ /dev/null @@ -1,901 +0,0 @@ -# generated by praktika - -name: PR - -on: - pull_request: - branches: ['master'] - -# Cancel the previous wf run in PRs. -concurrency: - group: ${{ github.workflow }}-${{ github.ref }} - cancel-in-progress: true - -env: - # Force the stdout and stderr streams to be unbuffered - PYTHONUNBUFFERED: 1 - GH_TOKEN: ${{ github.token }} - -# Allow updating GH commit statuses and PR comments to post an actual job reports link -permissions: write-all - -jobs: - - config_workflow: - runs-on: [ci_services] - needs: [] - name: "Config Workflow" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Config Workflow''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Config Workflow''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - docker_builds: - runs-on: [ci_services_ebs] - needs: [config_workflow] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'RG9ja2VyIEJ1aWxkcw==') }} - name: "Docker Builds" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Docker Builds''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Docker Builds''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - style_check: - runs-on: [ci_services] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3R5bGUgQ2hlY2s=') }} - name: "Style Check" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Style Check''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Style Check''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - fast_test: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'RmFzdCB0ZXN0') }} - name: "Fast test" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Fast test''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Fast test''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_debug: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9kZWJ1Zyk=') }} - name: "Build (amd_debug)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_debug)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_debug)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_release: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9yZWxlYXNlKQ==') }} - name: "Build (amd_release)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_asan: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9hc2FuKQ==') }} - name: "Build (amd_asan)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_asan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_asan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_tsan: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF90c2FuKQ==') }} - name: "Build (amd_tsan)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_tsan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_tsan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_msan: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9tc2FuKQ==') }} - name: "Build (amd_msan)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_msan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_msan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_ubsan: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF91YnNhbik=') }} - name: "Build (amd_ubsan)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_ubsan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_ubsan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_amd_binary: - runs-on: [builder] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFtZF9iaW5hcnkp') }} - name: "Build (amd_binary)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (amd_binary)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (amd_binary)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_arm_release: - runs-on: [builder-aarch64] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFybV9yZWxlYXNlKQ==') }} - name: "Build (arm_release)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (arm_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (arm_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - build_arm_asan: - runs-on: [builder-aarch64] - needs: [config_workflow, docker_builds] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'QnVpbGQgKGFybV9hc2FuKQ==') }} - name: "Build (arm_asan)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Build (arm_asan)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Build (arm_asan)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_amd_debugparallel: - runs-on: [builder] - needs: [config_workflow, docker_builds, build_amd_debug] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfZGVidWcscGFyYWxsZWwp') }} - name: "Stateless tests (amd_debug,parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (amd_debug,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (amd_debug,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_amd_debugnon_parallel: - runs-on: [func-tester] - needs: [config_workflow, docker_builds, build_amd_debug] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfZGVidWcsbm9uLXBhcmFsbGVsKQ==') }} - name: "Stateless tests (amd_debug,non-parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (amd_debug,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (amd_debug,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_amd_releaseparallel: - runs-on: [builder] - needs: [config_workflow, docker_builds, build_amd_release] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfcmVsZWFzZSxwYXJhbGxlbCk=') }} - name: "Stateless tests (amd_release,parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (amd_release,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (amd_release,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_amd_releasenon_parallel: - runs-on: [func-tester] - needs: [config_workflow, docker_builds, build_amd_release] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhbWRfcmVsZWFzZSxub24tcGFyYWxsZWwp') }} - name: "Stateless tests (amd_release,non-parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (amd_release,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (amd_release,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_arm_asanparallel: - runs-on: [builder-aarch64] - needs: [config_workflow, docker_builds, build_arm_asan] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhcm1fYXNhbixwYXJhbGxlbCk=') }} - name: "Stateless tests (arm_asan,parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (arm_asan,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (arm_asan,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateless_tests_arm_asannon_parallel: - runs-on: [func-tester-aarch64] - needs: [config_workflow, docker_builds, build_arm_asan] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVsZXNzIHRlc3RzIChhcm1fYXNhbixub24tcGFyYWxsZWwp') }} - name: "Stateless tests (arm_asan,non-parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateless tests (arm_asan,non-parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateless tests (arm_asan,non-parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stateful_tests_amd_releaseparallel: - runs-on: [builder] - needs: [config_workflow, docker_builds, build_amd_debug] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RhdGVmdWwgdGVzdHMgKGFtZF9yZWxlYXNlLHBhcmFsbGVsKQ==') }} - name: "Stateful tests (amd_release,parallel)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stateful tests (amd_release,parallel)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stateful tests (amd_release,parallel)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - stress_tests_arm_release: - runs-on: [func-tester-aarch64] - needs: [config_workflow, docker_builds, build_arm_release] - if: ${{ !failure() && !cancelled() && !contains(fromJson(needs.config_workflow.outputs.data).cache_success_base64, 'U3RyZXNzIHRlc3RzIChhcm1fcmVsZWFzZSk=') }} - name: "Stress tests (arm_release)" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Stress tests (arm_release)''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Stress tests (arm_release)''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi - - finish_workflow: - runs-on: [ci_services] - needs: [config_workflow, docker_builds, style_check, fast_test, build_amd_debug, build_amd_release, build_amd_asan, build_amd_tsan, build_amd_msan, build_amd_ubsan, build_amd_binary, build_arm_release, build_arm_asan, stateless_tests_amd_debugparallel, stateless_tests_amd_debugnon_parallel, stateless_tests_amd_releaseparallel, stateless_tests_amd_releasenon_parallel, stateless_tests_arm_asanparallel, stateless_tests_arm_asannon_parallel, stateful_tests_amd_releaseparallel, stress_tests_arm_release] - if: ${{ !cancelled() }} - name: "Finish Workflow" - outputs: - data: ${{ steps.run.outputs.DATA }} - steps: - - name: Checkout code - uses: actions/checkout@v4 - with: - ref: ${{ github.head_ref }} - - - name: Prepare env script - run: | - cat > /tmp/praktika_setup_env.sh << 'ENV_SETUP_SCRIPT_EOF' - export PYTHONPATH=./ci:. - - cat > /tmp/praktika/workflow_config_pr.json << 'EOF' - ${{ needs.config_workflow.outputs.data }} - EOF - cat > /tmp/praktika/workflow_status.json << 'EOF' - ${{ toJson(needs) }} - EOF - ENV_SETUP_SCRIPT_EOF - - rm -rf /tmp/praktika/input /tmp/praktika/output /tmp/praktika - mkdir -p /tmp/praktika /tmp/praktika/input /tmp/praktika/output - - - name: Run - id: run - run: | - . /tmp/praktika_setup_env.sh - set -o pipefail - if command -v ts &> /dev/null; then - python3 -m praktika run '''Finish Workflow''' --workflow "PR" --ci |& ts '[%Y-%m-%d %H:%M:%S]' | tee /tmp/praktika/praktika_run.log - else - python3 -m praktika run '''Finish Workflow''' --workflow "PR" --ci |& tee /tmp/praktika/praktika_run.log - fi diff --git a/ci/docker/compatibility/centos/Dockerfile b/ci/docker/compatibility/centos/Dockerfile new file mode 100644 index 00000000000..628609e374f --- /dev/null +++ b/ci/docker/compatibility/centos/Dockerfile @@ -0,0 +1,5 @@ +# docker build -t clickhouse/test-old-centos . +FROM centos:5 + +CMD /bin/sh -c "/clickhouse server --config /config/config.xml > /var/log/clickhouse-server/stderr.log 2>&1 & \ + sleep 5 && /clickhouse client --query \"select 'OK'\" 2> /var/log/clickhouse-server/clientstderr.log || echo 'FAIL'" diff --git a/ci/docker/compatibility/ubuntu/Dockerfile b/ci/docker/compatibility/ubuntu/Dockerfile new file mode 100644 index 00000000000..ddd0a76bd44 --- /dev/null +++ b/ci/docker/compatibility/ubuntu/Dockerfile @@ -0,0 +1,5 @@ +# docker build -t clickhouse/test-old-ubuntu . +FROM ubuntu:12.04 + +CMD /bin/sh -c "/clickhouse server --config /config/config.xml > /var/log/clickhouse-server/stderr.log 2>&1 & \ + sleep 5 && /clickhouse client --query \"select 'OK'\" 2> /var/log/clickhouse-server/clientstderr.log || echo 'FAIL'" diff --git a/ci/jobs/build_clickhouse.py b/ci/jobs/build_clickhouse.py index 473027b5779..e130507e573 100644 --- a/ci/jobs/build_clickhouse.py +++ b/ci/jobs/build_clickhouse.py @@ -1,4 +1,5 @@ import argparse +import os from praktika.result import Result from praktika.settings import Settings @@ -6,6 +7,7 @@ from praktika.utils import MetaClasses, Shell, Utils from ci.jobs.scripts.clickhouse_version import CHVersion from ci.workflows.defs import CIFiles, ToolSet +from ci.workflows.pull_request import S3_BUILDS_BUCKET class JobStages(metaclass=MetaClasses.WithIter): @@ -49,6 +51,11 @@ CMAKE_CMD = """cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA \ def main(): args = parse_args() + # # for sccache + # os.environ["SCCACHE_BUCKET"] = S3_BUILDS_BUCKET + # os.environ["SCCACHE_S3_KEY_PREFIX"] = "ccache/sccache" + # TODO: check with SCCACHE_LOG=debug SCCACHE_NO_DAEMON=1 + stop_watch = Utils.Stopwatch() stages = list(JobStages) diff --git a/ci/praktika/native_jobs.py b/ci/praktika/native_jobs.py index 58cd440283a..3307ba30b43 100644 --- a/ci/praktika/native_jobs.py +++ b/ci/praktika/native_jobs.py @@ -144,7 +144,7 @@ def _config_workflow(workflow: Workflow.Config, job_name): f"git diff-index HEAD -- {Settings.WORKFLOW_PATH_PREFIX}" ) info = "" - status = Result.Status.SUCCESS + status = Result.Status.FAILED if exit_code != 0: info = f"workspace has uncommitted files unexpectedly [{output}]" status = Result.Status.ERROR @@ -154,10 +154,14 @@ def _config_workflow(workflow: Workflow.Config, job_name): exit_code, output, err = Shell.get_res_stdout_stderr( f"git diff-index HEAD -- {Settings.WORKFLOW_PATH_PREFIX}" ) - if exit_code != 0: - info = f"workspace has outdated workflows [{output}] - regenerate with [python -m praktika --generate]" - status = Result.Status.ERROR + if output: + info = f"workflows are outdated: [{output}]" + status = Result.Status.FAILED print("ERROR: ", info) + elif exit_code == 0 and not err: + status = Result.Status.SUCCESS + else: + print(f"ERROR: exit code [{exit_code}], err [{err}]") return ( Result( diff --git a/ci/praktika/runner.py b/ci/praktika/runner.py index 30c108d5aaa..fbd9b16e562 100644 --- a/ci/praktika/runner.py +++ b/ci/praktika/runner.py @@ -61,8 +61,6 @@ class Runner: docker, workflow.dockers ) - # work around for old clickhouse jobs - os.environ["DOCKER_TAG"] = json.dumps(workflow_config.digest_dockers) workflow_config.dump() Result.generate_pending(job.name).dump() @@ -149,6 +147,14 @@ class Runner: env.JOB_NAME = job.name env.dump() + # work around for old clickhouse jobs + try: + os.environ["DOCKER_TAG"] = json.dumps( + RunConfig.from_fs(workflow.name).digest_dockers + ) + except Exception as e: + print(f"WARNING: Failed to set DOCKER_TAG, ex [{e}]") + if param: if not isinstance(param, str): Utils.raise_with_error( diff --git a/ci/workflows/defs.py b/ci/workflows/defs.py index 30a481aa778..077921dc5da 100644 --- a/ci/workflows/defs.py +++ b/ci/workflows/defs.py @@ -9,6 +9,9 @@ class RunnerLabels: BUILDER_ARM = "builder-aarch64" FUNC_TESTER_AMD = "func-tester" FUNC_TESTER_ARM = "func-tester-aarch64" + STYLE_CHECK_AMD = "style-checker" + STYLE_CHECK_ARM = "style-checker-aarch64" + CI_SERVICES = "ci_services" class CIFiles: @@ -52,18 +55,18 @@ DOCKERS = [ # platforms=Docker.Platforms.arm_amd, # depends_on=[], # ), - # Docker.Config( - # name="clickhouse/test-old-centos", - # path="./ci/docker/test/compatibility/centos", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), - # Docker.Config( - # name="clickhouse/test-old-ubuntu", - # path="./ci/docker/test/compatibility/ubuntu", - # platforms=Docker.Platforms.arm_amd, - # depends_on=[], - # ), + Docker.Config( + name="clickhouse/test-old-centos", + path="./ci/docker/compatibility/centos", + platforms=Docker.Platforms.arm_amd, + depends_on=[], + ), + Docker.Config( + name="clickhouse/test-old-ubuntu", + path="./ci/docker/compatibility/ubuntu", + platforms=Docker.Platforms.arm_amd, + depends_on=[], + ), # Docker.Config( # name="clickhouse/test-util", # path="./ci/docker/test/util", @@ -247,6 +250,7 @@ class JobNames: STATEFUL = "Stateful tests" STRESS = "Stress tests" PERFORMANCE = "Performance tests" + COMPATIBILITY = "Compatibility check" class ToolSet: @@ -576,7 +580,7 @@ class Jobs: runs_on=[RunnerLabels.FUNC_TESTER_ARM], command="./ci/jobs/scripts/performance_test.sh", run_in_docker="clickhouse/stateless-test", - requires=[ArtifactNames.CH_ARM_RELEASE] + requires=[ArtifactNames.CH_ARM_RELEASE], # digest_config=Job.CacheDigestConfig( # include_paths=[ # "./ci/jobs/fast_test.py", @@ -585,3 +589,22 @@ class Jobs: # ], # ), ) + + compatibility_test_jobs = Job.Config( + name=JobNames.COMPATIBILITY, + runs_on=["#from param"], + command="python3 ./tests/ci/compatibility_check.py --check-name {PARAMETER}", + digest_config=Job.CacheDigestConfig( + include_paths=[ + "./tests/ci/compatibility_check.py", + "./docker/test/compatibility", + ], + ), + ).parametrize( + parameter=["amd_release", "arm_release"], + runs_on=[ + [RunnerLabels.STYLE_CHECK_AMD], + [RunnerLabels.STYLE_CHECK_ARM], + ], + requires=[[ArtifactNames.DEB_AMD_RELEASE], [ArtifactNames.DEB_ARM_RELEASE]], + ) diff --git a/ci/workflows/pull_request.py b/ci/workflows/pull_request.py index 1d3e67dd461..363fc98913b 100644 --- a/ci/workflows/pull_request.py +++ b/ci/workflows/pull_request.py @@ -2,6 +2,8 @@ from praktika import Workflow from ci.workflows.defs import ARTIFACTS, BASE_BRANCH, DOCKERS, SECRETS, Jobs +S3_BUILDS_BUCKET = "clickhouse-builds" + workflow = Workflow.Config( name="PR", event=Workflow.Event.PULL_REQUEST, @@ -14,6 +16,7 @@ workflow = Workflow.Config( *Jobs.stateful_tests_jobs, *Jobs.stress_test_jobs, Jobs.performance_test_job, + *Jobs.compatibility_test_jobs, ], artifacts=ARTIFACTS, dockers=DOCKERS, diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 38fb2eceb28..4677ae7f172 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -11,6 +11,7 @@ from typing import List, Tuple from pip._vendor.packaging.version import Version from build_download_helper import download_builds_filter +from ci_utils import Shell from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults @@ -151,7 +152,14 @@ def main(): "clickhouse-common-static_" in url or "clickhouse-server_" in url ) - download_builds_filter(check_name, reports_path, packages_path, url_filter) + if check_name in ("amd_release", "amd_debug", "arm_release"): + # this is praktika based CI + print("Copy input *.deb artifacts") + assert Shell.check( + f"cp /tmp/praktika/input/*.deb {packages_path}", verbose=True + ) + else: + download_builds_filter(check_name, reports_path, packages_path, url_filter) for package in packages_path.iterdir(): if package.suffix == ".deb": @@ -196,9 +204,9 @@ def main(): # See https://sourceware.org/glibc/wiki/Glibc%20Timeline max_glibc_version = "" - if "amd64" in check_name or "release" in check_name: + if "amd" in check_name or "(release)" in check_name: max_glibc_version = "2.4" - elif "aarch64" in check_name: + elif "aarch" in check_name or "arm" in check_name: max_glibc_version = "2.18" # because of build with newer sysroot? else: raise RuntimeError("Can't determine max glibc version") diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index cd1251871d5..4cfe2e0a99b 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -10,6 +10,7 @@ from pathlib import Path from typing import List, Tuple from build_download_helper import download_all_deb_packages +from ci_utils import Shell from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH @@ -18,7 +19,6 @@ from pr_info import PRInfo from report import ERROR, JobReport, TestResults, read_test_results from stopwatch import Stopwatch from tee_popen import TeePopen -from ci_utils import Shell class SensitiveFormatter(logging.Formatter): From c0a0fb1869775a264484fe95b3514838e82ec008 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 17:44:55 +0100 Subject: [PATCH 283/433] Better error message on bad keeper snapshots --- src/Coordination/KeeperStateMachine.cpp | 22 ++++++++++++++-------- src/Coordination/SnapshotableHashTable.h | 14 +++++++++++--- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 704d3365fa2..9f9de7ae2f1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -142,13 +142,14 @@ void KeeperStateMachine::init() } catch (...) { - tryLogCurrentException( + LOG_FATAL( log, - fmt::format( - "Aborting because of failure to load from latest snapshot with index {}. Problematic snapshot can be removed but it will " - "lead to data loss", - latest_log_index)); - std::abort(); + "Failure to load from latest snapshot with index {}: {}", + latest_log_index, + getCurrentExceptionMessage(true, true, false)); + LOG_FATAL( + log, "Manual intervention is necessary for recovery. Problematic snapshot can be removed but it will lead to data loss"); + abort(); } } @@ -427,8 +428,13 @@ bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestFor } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to preprocess stored log at index {}, aborting to avoid inconsistent state", request_for_session.log_idx)); - std::abort(); + LOG_FATAL( + log, + "Failed to preprocess stored log at index {}: {}", + request_for_session.log_idx, + getCurrentExceptionMessage(true, true, false)); + LOG_FATAL(log, "Aborting to avoid inconsistent state"); + abort(); } if (keeper_context->digestEnabled() && request_for_session.digest) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index de089913ea7..bfe987de1cb 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + template struct ListNode { @@ -292,7 +297,8 @@ public: { size_t hash_value = map.hash(key); auto it = map.find(key, hash_value); - chassert(it != map.end()); + if (it != map.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); auto list_itr = it->getMapped(); uint64_t old_value_size = list_itr->value.sizeInBytes(); @@ -348,7 +354,8 @@ public: const V & getValue(StringRef key) const { auto it = map.find(key); - chassert(it); + if (it != map.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); return it->getMapped()->value; } @@ -356,7 +363,8 @@ public: { for (auto & itr : snapshot_invalid_iters) { - chassert(!itr->isActiveInMap()); + if (itr->isActiveInMap()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} is not active in map", itr->key); updateDataSize(ERASE, itr->key.size, 0, itr->value.sizeInBytes(), /*remove_old=*/true); if (itr->getFreeKey()) arena.free(const_cast(itr->key.data), itr->key.size); From 2fb3871a62b94e532fa7d385b53bd7c6d11e26c0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 22 Nov 2024 23:02:46 +0100 Subject: [PATCH 284/433] fix uncaught exception --- src/Interpreters/HashJoin/ScatteredBlock.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/ScatteredBlock.h b/src/Interpreters/HashJoin/ScatteredBlock.h index 729377f6758..31ff773d04d 100644 --- a/src/Interpreters/HashJoin/ScatteredBlock.h +++ b/src/Interpreters/HashJoin/ScatteredBlock.h @@ -302,10 +302,11 @@ struct ScatteredBlock : private boost::noncopyable /// Cut first `num_rows` rows from `block` in place and returns block with remaining rows ScatteredBlock cut(size_t num_rows) { - SCOPE_EXIT(filterBySelector()); - if (num_rows >= rows()) + { + filterBySelector(); return ScatteredBlock{Block{}}; + } chassert(block); @@ -314,6 +315,7 @@ struct ScatteredBlock : private boost::noncopyable auto remaining = ScatteredBlock{block, std::move(remaining_selector)}; selector = std::move(first_num_rows); + filterBySelector(); return remaining; } From dc2bc3e68df2059065cfb79a6547849ef2a6bb6d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 25 Nov 2024 17:06:01 +0000 Subject: [PATCH 285/433] do not load index in mergeTreeIndex if not needed --- src/Storages/StorageMergeTreeIndex.cpp | 12 +++++----- ...03276_merge_tree_index_lazy_load.reference | 13 +++++++++++ .../03276_merge_tree_index_lazy_load.sql | 22 +++++++++++++++++++ 3 files changed, 42 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03276_merge_tree_index_lazy_load.reference create mode 100644 tests/queries/0_stateless/03276_merge_tree_index_lazy_load.sql diff --git a/src/Storages/StorageMergeTreeIndex.cpp b/src/Storages/StorageMergeTreeIndex.cpp index 2ced59751f0..c082db68549 100644 --- a/src/Storages/StorageMergeTreeIndex.cpp +++ b/src/Storages/StorageMergeTreeIndex.cpp @@ -68,7 +68,7 @@ protected: const auto & part_name_column = StorageMergeTreeIndex::part_name_column; const auto & mark_number_column = StorageMergeTreeIndex::mark_number_column; const auto & rows_in_granule_column = StorageMergeTreeIndex::rows_in_granule_column; - const auto & index = part->getIndex(); + IMergeTreeDataPart::Index index_ptr; Columns result_columns(num_columns); for (size_t pos = 0; pos < num_columns; ++pos) @@ -78,18 +78,20 @@ protected: if (index_header.has(column_name)) { + if (!index_ptr) + index_ptr = part->getIndex(); + size_t index_position = index_header.getPositionByName(column_name); /// Some of the columns from suffix of primary index may be not loaded /// according to setting 'primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns'. - if (index_position < index->size()) + if (index_position < index_ptr->size()) { - result_columns[pos] = index->at(index_position); + result_columns[pos] = index_ptr->at(index_position); } else { - const auto & index_type = index_header.getByPosition(index_position).type; - auto index_column = index_type->createColumnConstWithDefaultValue(num_rows); + auto index_column = column_type->createColumnConstWithDefaultValue(num_rows); result_columns[pos] = index_column->convertToFullColumnIfConst(); } } diff --git a/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.reference b/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.reference new file mode 100644 index 00000000000..bbaa43552c8 --- /dev/null +++ b/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.reference @@ -0,0 +1,13 @@ +all_1_1_0 0 +all_1_1_0 0 4 +all_1_1_0 1 4 +all_1_1_0 2 4 +all_1_1_0 3 3 +all_1_1_0 4 0 +all_1_1_0 0 +all_1_1_0 0 4 0 +all_1_1_0 1 4 4 +all_1_1_0 2 4 8 +all_1_1_0 3 3 12 +all_1_1_0 4 0 14 +all_1_1_0 40 diff --git a/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.sql b/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.sql new file mode 100644 index 00000000000..4e3137c5514 --- /dev/null +++ b/tests/queries/0_stateless/03276_merge_tree_index_lazy_load.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS t_index_lazy_load; + +CREATE TABLE t_index_lazy_load (a UInt64) +ENGINE = MergeTree ORDER BY a +SETTINGS index_granularity = 4, index_granularity_bytes = '10M', primary_key_lazy_load = 1; + +INSERT INTO t_index_lazy_load SELECT number FROM numbers(15); + +DETACH TABLE t_index_lazy_load; +ATTACH TABLE t_index_lazy_load; + +SELECT name, primary_key_bytes_in_memory FROM system.parts WHERE database = currentDatabase() AND table = 't_index_lazy_load'; + +-- Check that if index is not requested it is not loaded. +SELECT part_name, mark_number, rows_in_granule FROM mergeTreeIndex(currentDatabase(), t_index_lazy_load) ORDER BY part_name, mark_number; +SELECT name, primary_key_bytes_in_memory FROM system.parts WHERE database = currentDatabase() AND table = 't_index_lazy_load'; + +-- If index is requested we have to load it and keep in memory. +SELECT part_name, mark_number, rows_in_granule, a FROM mergeTreeIndex(currentDatabase(), t_index_lazy_load) ORDER BY part_name, mark_number; +SELECT name, primary_key_bytes_in_memory FROM system.parts WHERE database = currentDatabase() AND table = 't_index_lazy_load'; + +DROP TABLE t_index_lazy_load; From 631294c7e48a682182358f67b93a5db1d917de58 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Oct 2024 13:02:38 +0100 Subject: [PATCH 286/433] impl --- src/Interpreters/TableJoin.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index e1b12df6b25..0cddea476b1 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -997,8 +997,8 @@ void TableJoin::resetToCross() bool TableJoin::allowParallelHashJoin() const { - if (std::find(join_algorithm.begin(), join_algorithm.end(), JoinAlgorithm::PARALLEL_HASH) == join_algorithm.end()) - return false; + // if (std::find(join_algorithm.begin(), join_algorithm.end(), JoinAlgorithm::PARALLEL_HASH) == join_algorithm.end()) + // return false; if (!right_storage_name.empty()) return false; if (table_join.kind != JoinKind::Left && table_join.kind != JoinKind::Inner) From ba837900ca4189271e111fb3be209289dc2fb3f8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Oct 2024 22:23:24 +0100 Subject: [PATCH 287/433] fix test --- src/Interpreters/TableJoin.cpp | 5 +- .../0_stateless/00049_any_left_join.sql | 2 + .../0_stateless/00051_any_inner_join.sql | 3 +- .../0_stateless/00052_all_left_join.sql | 3 +- .../00163_shard_join_with_empty_table.sql | 1 + .../0_stateless/00830_join_overwrite.sql | 1 + .../0_stateless/00845_join_on_aliases.sql | 21 ++++--- .../02236_explain_pipeline_join.reference | 23 ++++--- .../0_stateless/02282_array_distance.sql | 2 + .../02346_additional_filters.reference | 2 +- .../0_stateless/02346_additional_filters.sql | 2 +- .../02374_analyzer_join_using.reference | 62 +++++++++++-------- .../02374_analyzer_join_using.sql.j2 | 8 ++- .../02835_join_step_explain.reference | 2 +- .../02962_max_joined_block_rows.sql | 2 + ...filter_push_down_equivalent_sets.reference | 10 +-- .../03037_recursive_cte_postgres_3.reference | 1 + .../03037_recursive_cte_postgres_3.sql | 1 + .../03038_recursive_cte_postgres_4.reference | 1 + .../03038_recursive_cte_postgres_4.sql | 1 + .../03046_column_in_block_array_join.sql | 3 +- ...61_analyzer_alias_as_right_key_in_join.sql | 3 +- .../03229_json_structure_comparison.sql | 3 +- 23 files changed, 100 insertions(+), 62 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 0cddea476b1..f03b8d44356 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -997,8 +997,9 @@ void TableJoin::resetToCross() bool TableJoin::allowParallelHashJoin() const { - // if (std::find(join_algorithm.begin(), join_algorithm.end(), JoinAlgorithm::PARALLEL_HASH) == join_algorithm.end()) - // return false; + if (std::ranges::find(join_algorithm, JoinAlgorithm::DEFAULT) == join_algorithm.end() + && std::ranges::find(join_algorithm, JoinAlgorithm::PARALLEL_HASH) == join_algorithm.end()) + return false; if (!right_storage_name.empty()) return false; if (table_join.kind != JoinKind::Left && table_join.kind != JoinKind::Inner) diff --git a/tests/queries/0_stateless/00049_any_left_join.sql b/tests/queries/0_stateless/00049_any_left_join.sql index ecd079a5085..de726a77bc7 100644 --- a/tests/queries/0_stateless/00049_any_left_join.sql +++ b/tests/queries/0_stateless/00049_any_left_join.sql @@ -1 +1,3 @@ +SELECT * FROM ( SELECT number, joined FROM system.numbers ANY LEFT JOIN (SELECT number * 2 AS number, number * 10 + 1 AS joined FROM system.numbers LIMIT 10) js2 USING number LIMIT 10 +) ORDER BY ALL; diff --git a/tests/queries/0_stateless/00051_any_inner_join.sql b/tests/queries/0_stateless/00051_any_inner_join.sql index 566b5ad526b..6de35ef46f4 100644 --- a/tests/queries/0_stateless/00051_any_inner_join.sql +++ b/tests/queries/0_stateless/00051_any_inner_join.sql @@ -8,4 +8,5 @@ ANY INNER JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 ) AS b -USING k; +USING k +ORDER BY ALL; diff --git a/tests/queries/0_stateless/00052_all_left_join.sql b/tests/queries/0_stateless/00052_all_left_join.sql index 6d5a1ba073c..0315b793460 100644 --- a/tests/queries/0_stateless/00052_all_left_join.sql +++ b/tests/queries/0_stateless/00052_all_left_join.sql @@ -6,4 +6,5 @@ ALL LEFT JOIN ( SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 ) js2 -USING k; +USING k +ORDER BY ALL; diff --git a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql index 07242f7a8d4..7cd653b66ec 100644 --- a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql +++ b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql @@ -2,6 +2,7 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; +SET join_algorithm = 'hash'; SELECT * FROM ( SELECT number, n, j1, j2 diff --git a/tests/queries/0_stateless/00830_join_overwrite.sql b/tests/queries/0_stateless/00830_join_overwrite.sql index f51a152ea46..4f19d5fc1c5 100644 --- a/tests/queries/0_stateless/00830_join_overwrite.sql +++ b/tests/queries/0_stateless/00830_join_overwrite.sql @@ -14,6 +14,7 @@ CREATE TABLE t2 (k UInt32, v UInt32) ENGINE = Memory; INSERT INTO t2 VALUES (1, 2), (1, 3); SET enable_analyzer = 1; +SET join_algorithm = 'hash'; SELECT v FROM (SELECT 1 as k) t1 ANY INNER JOIN t2 USING (k) SETTINGS join_any_take_last_row = 0; SELECT v FROM (SELECT 1 as k) t1 ANY INNER JOIN t2 USING (k) SETTINGS join_any_take_last_row = 1; diff --git a/tests/queries/0_stateless/00845_join_on_aliases.sql b/tests/queries/0_stateless/00845_join_on_aliases.sql index 0800d27caa6..769ed219064 100644 --- a/tests/queries/0_stateless/00845_join_on_aliases.sql +++ b/tests/queries/0_stateless/00845_join_on_aliases.sql @@ -9,33 +9,40 @@ INSERT INTO table2 SELECT number * 2, number * 20 FROM numbers(6); select t1.a t1_a, t2.a from table1 as t1 -join table2 as t2 on table1.a = table2.a and t1.a = table2.a and t1_a = table2.a; +join table2 as t2 on table1.a = table2.a and t1.a = table2.a and t1_a = table2.a +order by all; select t1.a t1_a, t2.a from table1 as t1 -join table2 as t2 on table1.a = t2.a and t1.a = t2.a and t1_a = t2.a; +join table2 as t2 on table1.a = t2.a and t1.a = t2.a and t1_a = t2.a +order by all; select t1.a as t1_a, t2.a t2_a from table1 as t1 -join table2 as t2 on table1.a = t2_a and t1.a = t2_a and t1_a = t2_a; +join table2 as t2 on table1.a = t2_a and t1.a = t2_a and t1_a = t2_a +order by all; select t1.a t1_a, t2.a from table1 as t1 -join table2 as t2 on table1.a = table2.a and t1.a = t2.a and t1_a = t2.a; +join table2 as t2 on table1.a = table2.a and t1.a = t2.a and t1_a = t2.a +order by all; select t1.a t1_a, t2.a as t2_a from table1 as t1 -join table2 as t2 on table1.a = table2.a and t1.a = t2.a and t1_a = t2_a; +join table2 as t2 on table1.a = table2.a and t1.a = t2.a and t1_a = t2_a +order by all; select * from table1 as t1 join table2 as t2 on t1_a = t2_a -where (table1.a as t1_a) > 4 and (table2.a as t2_a) > 2; +where (table1.a as t1_a) > 4 and (table2.a as t2_a) > 2 +order by all; select t1.*, t2.* from table1 as t1 join table2 as t2 on t1_a = t2_a -where (t1.a as t1_a) > 2 and (t2.a as t2_a) > 4; +where (t1.a as t1_a) > 2 and (t2.a as t2_a) > 4 +order by all; DROP TABLE table1; DROP TABLE table2; diff --git a/tests/queries/0_stateless/02236_explain_pipeline_join.reference b/tests/queries/0_stateless/02236_explain_pipeline_join.reference index 73d0ca8cb5e..f8306831330 100644 --- a/tests/queries/0_stateless/02236_explain_pipeline_join.reference +++ b/tests/queries/0_stateless/02236_explain_pipeline_join.reference @@ -1,17 +1,20 @@ (Expression) -ExpressionTransform +ExpressionTransform × 16 (Join) - JoiningTransform 2 → 1 - (Expression) - ExpressionTransform - (Limit) - Limit - (ReadFromSystemNumbers) - NumbersRange 0 → 1 - (Expression) - FillingRightJoinSide + JoiningTransform × 16 2 → 1 + Resize 1 → 16 + (Expression) ExpressionTransform (Limit) Limit (ReadFromSystemNumbers) NumbersRange 0 → 1 + (Expression) + Resize × 2 16 → 1 + FillingRightJoinSide × 16 + Resize 1 → 16 + ExpressionTransform + (Limit) + Limit + (ReadFromSystemNumbers) + NumbersRange 0 → 1 diff --git a/tests/queries/0_stateless/02282_array_distance.sql b/tests/queries/0_stateless/02282_array_distance.sql index 85abc8fa381..076ee93c341 100644 --- a/tests/queries/0_stateless/02282_array_distance.sql +++ b/tests/queries/0_stateless/02282_array_distance.sql @@ -1,3 +1,5 @@ +SET join_algorithm = 'hash'; + SELECT L1Distance([0, 0, 0], [1, 2, 3]); SELECT L2Distance([1, 2, 3], [0, 0, 0]); SELECT L2SquaredDistance([1, 2, 3], [0, 0, 0]); diff --git a/tests/queries/0_stateless/02346_additional_filters.reference b/tests/queries/0_stateless/02346_additional_filters.reference index 0a08995223d..0bdd406bae0 100644 --- a/tests/queries/0_stateless/02346_additional_filters.reference +++ b/tests/queries/0_stateless/02346_additional_filters.reference @@ -101,7 +101,7 @@ select * from (select number from system.numbers limit 5 union all select x from 4 4 5 -select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x order by all settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; 0 0 1 1 a 2 0 diff --git a/tests/queries/0_stateless/02346_additional_filters.sql b/tests/queries/0_stateless/02346_additional_filters.sql index 5a799e1c8c1..6dafe20eaee 100644 --- a/tests/queries/0_stateless/02346_additional_filters.sql +++ b/tests/queries/0_stateless/02346_additional_filters.sql @@ -42,7 +42,7 @@ select * from system.numbers as t limit 5 settings additional_table_filters={'t' select * from system.numbers limit 5 settings additional_table_filters={'system.numbers' : 'number != 3'}; select * from system.numbers limit 5 settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; select * from (select number from system.numbers limit 5 union all select x from table_1) order by number settings additional_table_filters={'system.numbers':'number != 3','table_1':'x!=2'}; -select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; +select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x order by all settings additional_table_filters={'system.numbers' : 'number != 3', 'table_1' : 'x != 2'}; select b + 1 as c from (select a + 1 as b from (select x + 1 as a from table_1)) settings additional_table_filters={'table_1' : 'x != 2 and x != 3'}; -- { echoOff } diff --git a/tests/queries/0_stateless/02374_analyzer_join_using.reference b/tests/queries/0_stateless/02374_analyzer_join_using.reference index 626bad032b2..dfc1bf275dc 100644 --- a/tests/queries/0_stateless/02374_analyzer_join_using.reference +++ b/tests/queries/0_stateless/02374_analyzer_join_using.reference @@ -4,13 +4,15 @@ SELECT 'JOIN INNER'; JOIN INNER SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; 0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String 1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String SELECT '--'; -- SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; Join_1_Value_0 String Join_2_Value_0 String Join_1_Value_1 String Join_2_Value_1 String SELECT '--'; @@ -24,14 +26,16 @@ SELECT 'JOIN LEFT'; JOIN LEFT SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; 0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String 1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String 2 UInt16 2 UInt16 Join_1_Value_2 String 0 UInt16 String SELECT '--'; -- SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; Join_1_Value_0 String Join_2_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_1_Value_2 String String @@ -47,17 +51,19 @@ SELECT 'JOIN RIGHT'; JOIN RIGHT SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; 0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String 1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String 3 UInt16 0 UInt16 String 3 UInt16 Join_2_Value_3 String SELECT '--'; -- SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; + String Join_2_Value_3 String Join_1_Value_0 String Join_2_Value_0 String Join_1_Value_1 String Join_2_Value_1 String - String Join_2_Value_3 String SELECT '--'; -- SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id); @@ -70,19 +76,21 @@ SELECT 'JOIN FULL'; JOIN FULL SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; +0 UInt16 0 UInt16 String 3 UInt16 Join_2_Value_3 String 0 UInt16 0 UInt16 Join_1_Value_0 String 0 UInt16 Join_2_Value_0 String 1 UInt16 1 UInt16 Join_1_Value_1 String 1 UInt16 Join_2_Value_1 String 2 UInt16 2 UInt16 Join_1_Value_2 String 0 UInt16 String -0 UInt16 0 UInt16 String 3 UInt16 Join_2_Value_3 String SELECT '--'; -- SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; + String Join_2_Value_3 String Join_1_Value_0 String Join_2_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_1_Value_2 String String - String Join_2_Value_3 String SELECT '--'; -- SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id); @@ -109,7 +117,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } @@ -130,7 +138,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 SELECT id FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 ON t1.id = t2.id LEFT JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } @@ -153,7 +161,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -177,7 +185,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 INNER JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -199,7 +207,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 SELECT id FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } @@ -222,7 +230,7 @@ Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String Join_1_Value_2 String String String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -246,7 +254,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -272,7 +280,7 @@ Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String Join_1_Value_2 String String String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 LEFT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -295,7 +303,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 SELECT id FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } @@ -318,7 +326,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -342,7 +350,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -368,7 +376,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 RIGHT JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -392,7 +400,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) INNER JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 SELECT id FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 ON t1.id = t2.id INNER JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } @@ -417,7 +425,7 @@ Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String Join_1_Value_2 String String String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) LEFT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -443,7 +451,7 @@ Join_1_Value_0 String Join_2_Value_0 String Join_3_Value_0 String Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) RIGHT JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 @@ -471,7 +479,7 @@ Join_1_Value_1 String Join_2_Value_1 String Join_3_Value_1 String Join_1_Value_2 String String String SELECT '--'; -- -SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 FULL JOIN test_table_join_2 AS t2 USING (id) FULL JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; 1 1 1 diff --git a/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 index 4c5097ca577..ba56c200ff3 100644 --- a/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 +++ b/tests/queries/0_stateless/02374_analyzer_join_using.sql.j2 @@ -41,12 +41,14 @@ SELECT 'JOIN {{ join_type }}'; SELECT id AS using_id, toTypeName(using_id), t1.id AS t1_id, toTypeName(t1_id), t1.value AS t1_value, toTypeName(t1_value), t2.id AS t2_id, toTypeName(t2_id), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; SELECT '--'; SELECT t1.value AS t1_value, toTypeName(t1_value), t2.value AS t2_value, toTypeName(t2_value) -FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id); +FROM test_table_join_1 AS t1 {{ join_type }} JOIN test_table_join_2 AS t2 USING (id) +ORDER BY ALL; SELECT '--'; @@ -75,7 +77,7 @@ ORDER BY ALL; SELECT '--'; -SELECT 1 FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id); +SELECT 1 FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 USING (id) {{ second_join_type }} JOIN test_table_join_3 AS t3 USING(id) ORDER BY ALL; SELECT id FROM test_table_join_1 AS t1 {{ first_join_type }} JOIN test_table_join_2 AS t2 ON t1.id = t2.id {{ second_join_type }} JOIN test_table_join_3 AS t3 USING (id); -- { serverError AMBIGUOUS_IDENTIFIER } diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index bdbc019d4f8..39a61adc853 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -19,7 +19,7 @@ Positions: 4 0 1 2 __table2.value_1 String Type: INNER Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.id UInt64 diff --git a/tests/queries/0_stateless/02962_max_joined_block_rows.sql b/tests/queries/0_stateless/02962_max_joined_block_rows.sql index 9edf757c0f7..75659fbbf48 100644 --- a/tests/queries/0_stateless/02962_max_joined_block_rows.sql +++ b/tests/queries/0_stateless/02962_max_joined_block_rows.sql @@ -13,6 +13,8 @@ SET min_joined_block_size_bytes = 0; -- block size is always multiple of 5 because we have 5 rows for each key in right table -- we do not split rows corresponding to the same key +SET join_algorithm = 'hash'; + SELECT max(bs) <= 5, b FROM ( SELECT blockSize() as bs, * FROM t1 JOIN t2 ON t1.a = t2.a ) GROUP BY b diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference index 1bea145c50a..aeba7408aa4 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.reference @@ -26,7 +26,7 @@ Positions: 4 1 0 2 __table2.value String Type: INNER Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Filter (( + (JOIN actions + Change column names to column identifiers))) Header: __table1.id UInt64 @@ -95,7 +95,7 @@ Positions: 4 1 0 2 __table2.value String Type: INNER Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Filter (( + (JOIN actions + Change column names to column identifiers))) Header: __table1.id UInt64 @@ -164,7 +164,7 @@ Positions: 4 1 0 2 __table2.value String Type: INNER Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Filter (( + (JOIN actions + Change column names to column identifiers))) Header: __table1.id UInt64 @@ -244,7 +244,7 @@ Positions: 4 1 0 2 __table2.value String Type: LEFT Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Filter (( + (JOIN actions + Change column names to column identifiers))) Header: __table1.id UInt64 @@ -326,7 +326,7 @@ Positions: 4 1 0 2 __table2.value String Type: LEFT Strictness: ALL - Algorithm: HashJoin + Algorithm: ConcurrentHashJoin Clauses: [(__table1.id) = (__table2.id)] Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.id UInt64 diff --git a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference index f0ff70a7c60..3a714e77e74 100644 --- a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference +++ b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.reference @@ -33,6 +33,7 @@ -- { echoOn } SET enable_analyzer = 1; +SET join_algorithm = 'hash'; -- -- different tree example -- diff --git a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql index 213e8bc995d..3413185fd6a 100644 --- a/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql +++ b/tests/queries/0_stateless/03037_recursive_cte_postgres_3.sql @@ -33,6 +33,7 @@ -- { echoOn } SET enable_analyzer = 1; +SET join_algorithm = 'hash'; -- -- different tree example diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference index 64d4d0d0eb1..bb0ba0428f1 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.reference @@ -33,6 +33,7 @@ -- { echoOn } SET enable_analyzer = 1; +SET join_algorithm = 'hash'; -- -- test cycle detection -- diff --git a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql index 65e4439c1fd..1fb34010223 100644 --- a/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql +++ b/tests/queries/0_stateless/03038_recursive_cte_postgres_4.sql @@ -33,6 +33,7 @@ -- { echoOn } SET enable_analyzer = 1; +SET join_algorithm = 'hash'; -- -- test cycle detection 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 f91a18da831..d7e932edddf 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 @@ -32,7 +32,8 @@ 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; +INNER JOIN join_test AS t2 ON nest.y = t2.y +ORDER BY ALL; DROP TABLE IF EXISTS nested_test; DROP TABLE IF EXISTS join_test; 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 a1b50967a46..c9d1e8e95b8 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 @@ -5,4 +5,5 @@ 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; +INNER JOIN xxxx_yyy AS b ON a.key = b.key_b +ORDER BY ALL; diff --git a/tests/queries/0_stateless/03229_json_structure_comparison.sql b/tests/queries/0_stateless/03229_json_structure_comparison.sql index 16db469325d..39029cee28b 100644 --- a/tests/queries/0_stateless/03229_json_structure_comparison.sql +++ b/tests/queries/0_stateless/03229_json_structure_comparison.sql @@ -16,7 +16,8 @@ SELECT FROM test_new_json_type a JOIN test_new_json_type b - ON a.id = b.id; + ON a.id = b.id +ORDER BY ALL; DROP TABLE test_new_json_type; From 12040bbf249f13e6a57ddfd90a6f92801ff5be9e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 25 Nov 2024 18:30:54 +0100 Subject: [PATCH 288/433] Fix for empty permutation --- src/Interpreters/sortBlock.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 1c4754b1ef2..064632d4c7b 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -282,10 +282,12 @@ void checkSortedWithPermutationImpl(size_t rows, Comparator compare, UInt64 limi if (limit && limit < rows) rows = limit; + const bool no_permutaiton = permutation.empty(); + for (size_t i = 1; i < rows; ++i) { - const size_t current_row = permutation[i]; - const size_t previous_row = permutation[i - 1]; + const size_t current_row = no_permutaiton ? i : permutation[i]; + const size_t previous_row = no_permutaiton ? (i - 1) : permutation[i - 1]; if (compare(current_row, previous_row)) throw Exception(ErrorCodes::LOGICAL_ERROR, From 7773873e726708a96857a4c3f9e5d91c84840826 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 25 Nov 2024 17:34:17 +0000 Subject: [PATCH 289/433] Restore deleted server setting --- src/Core/ServerSettings.cpp | 1 + src/Core/Settings.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index b920c905a54..d622d60221c 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -197,6 +197,7 @@ namespace DB DECLARE(UInt64, parts_kill_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to kill_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, parts_killer_pool_size, 128, "Threads for cleanup of shared merge tree outdated threads. Only available in ClickHouse Cloud", 0) \ DECLARE(UInt64, keeper_multiread_batch_size, 10'000, "Maximum size of batch for MultiRead request to [Zoo]Keeper that support batching. If set to 0, batching is disabled. Available only in ClickHouse Cloud.", 0) \ + DECLARE(Bool, use_legacy_mongodb_integration, true, "Obsolete, has no effect", 0) \ \ DECLARE(UInt64, prefetch_threadpool_pool_size, 100, "Size of background pool for prefetches for remote object storages", 0) \ DECLARE(UInt64, prefetch_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 698ceab0f76..178a9167d69 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5731,7 +5731,7 @@ Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting If enabled, only allow identifiers containing alphanumeric characters and underscores. )", 0) \ DECLARE(Bool, mongodb_throw_on_unsupported_query, true, R"( -If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option is not applied when 'allow_experimental_analyzer=0'. +If enabled, MongoDB tables will return an error when a MongoDB query cannot be built. Otherwise, ClickHouse reads the full table and processes it locally. This option is not applied when 'enable_analyzer=0'. )", 0) \ DECLARE(Bool, implicit_select, false, R"( Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. From 0bcc933592bbbc98350ff61d098765872216a33b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 22 Nov 2024 12:56:23 +0100 Subject: [PATCH 290/433] Remove workaround cleaner from postinst of clickhouse-server --- packages/clickhouse-server.postinstall | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/packages/clickhouse-server.postinstall b/packages/clickhouse-server.postinstall index d3b49db758f..c98e22c7e28 100644 --- a/packages/clickhouse-server.postinstall +++ b/packages/clickhouse-server.postinstall @@ -48,18 +48,6 @@ if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then fi fi - # /etc/systemd/system/clickhouse-server.service shouldn't be distributed by the package, but it was - # here we delete the service file if it was from our package - if [ -f /etc/systemd/system/clickhouse-server.service ]; then - SHA256=$(sha256sum /etc/systemd/system/clickhouse-server.service | cut -d' ' -f1) - for ref_sum in 7769a14773e811a56f67fd70f7960147217f5e68f746010aec96722e24d289bb 22890012047ea84fbfcebd6e291fe2ef2185cbfdd94a0294e13c8bf9959f58f8 b7790ae57156663c723f92e75ac2508453bf0a7b7e8313bb8081da99e5e88cd3 d1dcc1dbe92dab3ae17baa395f36abf1876b4513df272bf021484923e0111eef ac29ddd32a02eb31670bf5f0018c5d8a3cc006ca7ea572dcf717cb42310dcad7 c62d23052532a70115414833b500b266647d3924eb006a6f3eb673ff0d55f8fa b6b200ffb517afc2b9cf9e25ad8a4afdc0dad5a045bddbfb0174f84cc5a959ed; do - if [ "$SHA256" = "$ref_sum" ]; then - rm /etc/systemd/system/clickhouse-server.service - break - fi - done - fi - # Setup clickhouse-keeper directories chown -R "${CLICKHOUSE_USER}:${CLICKHOUSE_GROUP}" "${KEEPER_CONFDIR}" chmod 0755 "${KEEPER_CONFDIR}" From 4585a6225c552b5eff6ffc4fb0c31aac7e3b54bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 19:20:14 +0100 Subject: [PATCH 291/433] Revert "Resubmit #68682" --- src/Core/Settings.cpp | 7 - src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.cpp | 1 - src/Core/SettingsEnums.h | 2 + src/Core/SettingsFields.h | 4 +- src/Interpreters/ConcurrentHashJoin.h | 11 - src/Interpreters/FullSortingMergeJoin.h | 2 +- src/Interpreters/HashJoin/HashJoin.cpp | 30 +-- src/Interpreters/HashJoin/HashJoin.h | 5 +- .../HashJoin/HashJoinMethodsImpl.h | 16 -- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Interpreters/TableJoin.cpp | 56 +---- src/Interpreters/TableJoin.h | 19 +- src/Interpreters/TreeRewriter.cpp | 5 +- src/Parsers/CreateQueryUUIDs.cpp | 2 +- src/Planner/CollectColumnIdentifiers.cpp | 1 - src/Planner/PlannerJoinTree.cpp | 165 +++++--------- src/Processors/QueryPlan/JoinStep.cpp | 122 ++--------- src/Processors/QueryPlan/JoinStep.h | 18 +- .../QueryPlan/Optimizations/Optimizations.h | 1 - .../QueryPlan/Optimizations/optimizeJoin.cpp | 103 --------- .../QueryPlan/Optimizations/optimizeTree.cpp | 3 - .../QueryPlan/ReadFromMemoryStorageStep.h | 2 - .../Transforms/ColumnPermuteTransform.cpp | 49 ----- .../Transforms/ColumnPermuteTransform.h | 30 --- .../Transforms/JoiningTransform.cpp | 1 - .../Transforms/PasteJoinTransform.cpp | 2 - tests/clickhouse-test | 2 - tests/integration/helpers/cluster.py | 13 +- tests/integration/helpers/random_settings.py | 2 - .../test_peak_memory_usage/test.py | 2 +- .../0_stateless/00826_cross_to_inner_join.sql | 13 +- .../00847_multiple_join_same_column.sql | 14 +- .../01015_empty_in_inner_right_join.sql.j2 | 2 - .../01107_join_right_table_totals.reference | 7 - .../01107_join_right_table_totals.sql | 10 +- .../01763_filter_push_down_bugs.reference | 2 +- .../01881_join_on_conditions_hash.sql.j2 | 10 +- .../0_stateless/02000_join_on_const.reference | 18 +- .../0_stateless/02000_join_on_const.sql | 16 +- .../02001_join_on_const_bs_long.sql.j2 | 4 +- ...oin_with_nullable_lowcardinality_crash.sql | 5 +- .../0_stateless/02282_array_distance.sql | 12 +- .../02381_join_dup_columns_in_plan.reference | 1 + .../0_stateless/02461_join_lc_issue_42380.sql | 3 +- ...emove_redundant_sorting_analyzer.reference | 4 +- ...move_redundant_distinct_analyzer.reference | 18 +- .../02514_analyzer_drop_join_on.reference | 55 +++-- .../02514_analyzer_drop_join_on.sql | 1 - ...oin_with_totals_and_subquery_bug.reference | 2 +- .../02835_join_step_explain.reference | 32 +-- .../0_stateless/02835_join_step_explain.sql | 2 - .../02962_join_using_bug_57894.reference | 1 - .../02962_join_using_bug_57894.sql | 2 - ...filter_push_down_equivalent_sets.reference | 206 ++++++++---------- ..._join_filter_push_down_equivalent_sets.sql | 40 +--- .../03038_recursive_cte_postgres_4.reference | 4 +- .../03038_recursive_cte_postgres_4.sql | 4 +- .../0_stateless/03094_one_thousand_joins.sql | 1 - ...convert_outer_join_to_inner_join.reference | 36 +-- ...03130_convert_outer_join_to_inner_join.sql | 13 +- ...ter_push_down_equivalent_columns.reference | 3 +- .../03236_squashing_high_memory.sql | 1 - .../0_stateless/03267_join_swap_bug.reference | 4 - .../0_stateless/03267_join_swap_bug.sql | 33 --- 65 files changed, 336 insertions(+), 924 deletions(-) delete mode 100644 src/Processors/QueryPlan/Optimizations/optimizeJoin.cpp delete mode 100644 src/Processors/Transforms/ColumnPermuteTransform.cpp delete mode 100644 src/Processors/Transforms/ColumnPermuteTransform.h delete mode 100644 tests/queries/0_stateless/03267_join_swap_bug.reference delete mode 100644 tests/queries/0_stateless/03267_join_swap_bug.sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f5fde1b572b..2526334d290 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -1924,13 +1924,6 @@ See also: For single JOIN in case of identifier ambiguity prefer left table )", IMPORTANT) \ \ -DECLARE(BoolAuto, query_plan_join_swap_table, Field("auto"), R"( - Determine which side of the join should be the build table (also called inner, the one inserted into the hash table for a hash join) in the query plan. This setting is supported only for `ALL` join strictness with the `JOIN ON` clause. Possible values are: - - 'auto': Let the planner decide which table to use as the build table. - - 'false': Never swap tables (the right table is the build table). - - 'true': Always swap tables (the left table is the build table). -)", 0) \ - \ DECLARE(UInt64, preferred_block_size_bytes, 1000000, R"( This setting adjusts the data block size for query processing and represents additional fine-tuning to the more rough 'max_block_size' setting. If the columns are large and with 'max_block_size' rows the block size is likely to be larger than the specified amount of bytes, its size will be lowered for better CPU cache locality. )", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 464b626f2c7..b66f4403ddf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -45,7 +45,6 @@ class WriteBuffer; #define COMMON_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ M(CLASS_NAME, ArrowCompression) \ M(CLASS_NAME, Bool) \ - M(CLASS_NAME, BoolAuto) \ M(CLASS_NAME, CapnProtoEnumComparingMode) \ M(CLASS_NAME, Char) \ M(CLASS_NAME, DateTimeInputFormat) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c50e4137d30..768779c37db 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,7 +60,6 @@ static std::initializer_list #include -#include #include #include #include #include #include + namespace DB { namespace ErrorCodes @@ -125,10 +125,8 @@ struct SettingAutoWrapper void readBinary(ReadBuffer & in) { changed = true; is_auto = false; base.readBinary(in); } Type valueOr(Type default_value) const { return is_auto ? default_value : base.value; } - std::optional get() const { return is_auto ? std::nullopt : std::make_optional(base.value); } }; -using SettingFieldBoolAuto = SettingAutoWrapper; using SettingFieldUInt64Auto = SettingAutoWrapper; using SettingFieldInt64Auto = SettingAutoWrapper; using SettingFieldFloatAuto = SettingAutoWrapper; diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 2337ccaf714..c1a421f713b 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -63,17 +63,6 @@ public: IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; - - bool isCloneSupported() const override - { - return !getTotals() && getTotalRowCount() == 0; - } - - std::shared_ptr clone(const std::shared_ptr & table_join_, const Block &, const Block & right_sample_block_) const override - { - return std::make_shared(context, table_join_, slots, right_sample_block_, stats_collecting_params); - } - private: struct InternalHashJoin { diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index faa9114c618..3f1e0d59287 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -36,7 +36,7 @@ public: bool isCloneSupported() const override { - return !getTotals(); + return true; } std::shared_ptr clone(const std::shared_ptr & table_join_, diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 2984dda8a45..21885d4fab6 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -431,16 +431,6 @@ size_t HashJoin::getTotalByteCount() const return res; } -bool HashJoin::isUsedByAnotherAlgorithm() const -{ - return table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH); -} - -bool HashJoin::canRemoveColumnsFromLeftBlock() const -{ - return table_join->enableEnalyzer() && !table_join->hasUsing() && !isUsedByAnotherAlgorithm() && strictness != JoinStrictness::RightAny; -} - void HashJoin::initRightBlockStructure(Block & saved_block_sample) { if (isCrossOrComma(kind)) @@ -452,10 +442,8 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = isUsedByAnotherAlgorithm() || - isRightOrFull(kind) || - multiple_disjuncts || - table_join->getMixedJoinExpression(); + bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) + || isRightOrFull(kind) || multiple_disjuncts || table_join->getMixedJoinExpression(); if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -1368,10 +1356,7 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu { if (!JoinCommon::hasNonJoinedBlocks(*table_join)) return {}; - size_t left_columns_count = left_sample_block.columns(); - if (canRemoveColumnsFromLeftBlock()) - left_columns_count = table_join->getOutputColumns(JoinTableSide::Left).size(); bool flag_per_row = needUsedFlagsForPerRightTableRow(table_join); if (!flag_per_row) @@ -1380,9 +1365,14 @@ HashJoin::getNonJoinedBlocks(const Block & left_sample_block, const Block & resu size_t expected_columns_count = left_columns_count + required_right_keys.columns() + sample_block_with_columns_to_add.columns(); if (expected_columns_count != result_sample_block.columns()) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of columns in result sample block: {} expected {} ([{}] + [{}] + [{}])", - result_sample_block.columns(), expected_columns_count, - left_sample_block.dumpNames(), required_right_keys.dumpNames(), sample_block_with_columns_to_add.dumpNames()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unexpected number of columns in result sample block: {} instead of {} ({} + {} + {})", + result_sample_block.columns(), + expected_columns_count, + left_columns_count, + required_right_keys.columns(), + sample_block_with_columns_to_add.columns()); } } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 85cfb0869e7..e478bc66b3c 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -126,7 +126,7 @@ public: bool isCloneSupported() const override { - return !getTotals() && getTotalRowCount() == 0; + return true; } std::shared_ptr clone(const std::shared_ptr & table_join_, @@ -484,9 +484,6 @@ private: bool empty() const; - bool isUsedByAnotherAlgorithm() const; - bool canRemoveColumnsFromLeftBlock() const; - void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index b0fc7936278..47146485df5 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -80,7 +80,6 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]); } - auto & source_block = block.getSourceBlock(); size_t existing_columns = source_block.columns(); @@ -122,20 +121,6 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( block.filterBySelector(); - const auto & table_join = join.table_join; - std::set block_columns_to_erase; - if (join.canRemoveColumnsFromLeftBlock()) - { - std::unordered_set left_output_columns; - for (const auto & out_column : table_join->getOutputColumns(JoinTableSide::Left)) - left_output_columns.insert(out_column.name); - for (size_t i = 0; i < source_block.columns(); ++i) - { - if (!left_output_columns.contains(source_block.getByPosition(i).name)) - block_columns_to_erase.insert(i); - } - } - for (size_t i = 0; i < added_columns.size(); ++i) source_block.insert(added_columns.moveColumn(i)); @@ -191,7 +176,6 @@ ScatteredBlock HashJoinMethods::joinBlockImpl( columns[pos] = columns[pos]->replicate(offsets); block.getSourceBlock().setColumns(columns); - block.getSourceBlock().erase(block_columns_to_erase); block = ScatteredBlock(std::move(block).getSourceBlock()); } return remaining_block; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7b354aa6888..7e65e20724e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1889,9 +1889,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional