From 27150fbe48be0b0283d8ebc82e2bdfddba9a07ca Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 3 Sep 2025 22:21:47 +0200 Subject: [PATCH 01/20] Add TieredDistributedMerge engine stub registration and some basic test --- src/Storages/StorageDistributed.cpp | 51 +++++++++++++++++++ .../03372_tiered_distributed_merge.reference | 13 +++++ .../03372_tiered_distributed_merge.sql | 8 +++ 3 files changed, 72 insertions(+) create mode 100644 tests/queries/0_stateless/03372_tiered_distributed_merge.reference create mode 100644 tests/queries/0_stateless/03372_tiered_distributed_merge.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c5a89813f79c..f4e7db7a51e9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2159,6 +2159,57 @@ void registerStorageDistributed(StorageFactory & factory) .source_access_type = AccessType::REMOTE, .has_builtin_setting_fn = DistributedSettings::hasBuiltin, }); + + // Register TieredDistributedMerge engine - Step 3: Add first argument validation + factory.registerStorage("TieredDistributedMerge", [](const StorageFactory::Arguments & args) -> StoragePtr + { + ASTs & engine_args = args.engine_args; + + if (engine_args.size() < 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage TieredDistributedMerge requires at least 2 arguments, got {}", engine_args.size()); + + // Validate first argument - must be a table function + ASTPtr first_arg = engine_args[0]; + if (const auto * func = first_arg->as()) + { + // Check if it's a valid table function name + if (!TableFunctionFactory::instance().isTableFunctionName(func->name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "First argument must be a table function, got: {}", func->name); + + // Check if it's one of the supported remote table functions + if (func->name != "remote" && func->name != "remoteSecure" && + func->name != "cluster" && func->name != "clusterAllReplicas") + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "First argument must be one of: remote, remoteSecure, cluster, clusterAllReplicas, got: {}", func->name); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "First argument must be a table function, got: {}", first_arg->getID()); + } + + // Validate second argument - must be a SQL expression (not a string literal) + ASTPtr second_arg = engine_args[1]; + if (const auto * literal = second_arg->as()) + { + // Check if it's a string literal (which would be invalid for a SQL expression) + if (literal->value.getType() == Field::Types::String) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Second argument must be a SQL expression, got string literal"); + } + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "TieredDistributedMerge engine is not implemented yet"); + }, + { + .supports_settings = false, + .supports_parallel_insert = false, + .supports_schema_inference = false, + .source_access_type = AccessType::REMOTE, + }); } bool StorageDistributed::initializeDiskOnConfigChange(const std::set & new_added_disks) diff --git a/tests/queries/0_stateless/03372_tiered_distributed_merge.reference b/tests/queries/0_stateless/03372_tiered_distributed_merge.reference new file mode 100644 index 000000000000..2cf8256d7f20 --- /dev/null +++ b/tests/queries/0_stateless/03372_tiered_distributed_merge.reference @@ -0,0 +1,13 @@ +Code: 42. DB::Exception: Storage TieredDistributedMerge requires at least 2 arguments, got 0. (NUMBER_OF_ARGUMENTS_DOESNT_MATCH) + +Code: 42. DB::Exception: Storage TieredDistributedMerge requires at least 2 arguments, got 1. (NUMBER_OF_ARGUMENTS_DOESNT_MATCH) + +Code: 36. DB::Exception: First argument must be a table function, got: Literal_'invalid_arg'. (BAD_ARGUMENTS) + +Code: 36. DB::Exception: First argument must be a table function, got: sin. (BAD_ARGUMENTS) + +Code: 36. DB::Exception: First argument must be one of: remote, remoteSecure, cluster, clusterAllReplicas, got: url. (BAD_ARGUMENTS) + +Code: 42. DB::Exception: Storage TieredDistributedMerge requires at least 2 arguments, got 1. (NUMBER_OF_ARGUMENTS_DOESNT_MATCH) + +Code: 36. DB::Exception: Second argument must be a SQL expression, got string literal. (BAD_ARGUMENTS) \ No newline at end of file diff --git a/tests/queries/0_stateless/03372_tiered_distributed_merge.sql b/tests/queries/0_stateless/03372_tiered_distributed_merge.sql new file mode 100644 index 000000000000..a00481eadfda --- /dev/null +++ b/tests/queries/0_stateless/03372_tiered_distributed_merge.sql @@ -0,0 +1,8 @@ +-- Test TieredDistributedMerge engine registration and basic validation +CREATE TABLE test_tiered_distributed_merge_no_args ( `id` UInt32, `name` String ) ENGINE = TieredDistributedMerge(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +CREATE TABLE test_tiered_distributed_merge_one_arg ( `id` UInt32, `name` String ) ENGINE = TieredDistributedMerge(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge('invalid_arg', 1); -- { serverError BAD_ARGUMENTS } +CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge(sin(3), 1); -- { serverError BAD_ARGUMENTS } +CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge(url('http://google.com', 'RawBLOB'), 1); -- { serverError BAD_ARGUMENTS } +CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge(urlCluster('test_cluster', 'http://example.com')); -- { serverError BAD_ARGUMENTS } +CREATE TABLE test_tiered_distributed_merge_invalid_second_arg (`id` UInt32, `name` String) ENGINE = TieredDistributedMerge(remote('test_cluster', 'db', 'table'), 'invalid_predicate');-- { serverError BAD_ARGUMENTS } From 2e651c2b9796745b8778b4e10f90ec666d737535 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 3 Sep 2025 23:00:41 +0200 Subject: [PATCH 02/20] Pass-through the initialization to the first table funcion --- src/Storages/StorageDistributed.cpp | 45 +++++++++++- .../03372_tiered_distributed_merge.sql | 3 + ...red_distributed_merge_real_table.reference | 26 +++++++ ...74_tiered_distributed_merge_real_table.sql | 68 +++++++++++++++++++ 4 files changed, 141 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference create mode 100644 tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f4e7db7a51e9..86d8ca7ef512 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -85,6 +85,7 @@ #include #include +#include #include #include @@ -2202,7 +2203,49 @@ void registerStorageDistributed(StorageFactory & factory) } } - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "TieredDistributedMerge engine is not implemented yet"); + // Create the underlying StorageDistributed using the table function + const ContextPtr & context = args.getContext(); + + auto table_function = TableFunctionFactory::instance().get(first_arg, context); + if (!table_function) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); + + // Execute the table function to get the underlying storage + StoragePtr storage = table_function->execute( + first_arg, + context, + args.table_id.table_name, + args.columns, + false, // use_global_context = false + false); // is_insert_query = false + + // table function execution wraps the actual storage in a StorageTableFunctionProxy, to make initialize it lazily + // so we need to get the nested storage + if (auto proxy = std::dynamic_pointer_cast(storage)) + { + storage = proxy->getNested(); + } + + // Cast to StorageDistributed to access its methods + auto distributed_storage = std::dynamic_pointer_cast(storage); + if (!distributed_storage) + { + // Debug: Print the actual type we got + std::string actual_type = storage ? storage->getName() : "nullptr"; + throw Exception(ErrorCodes::LOGICAL_ERROR, + "TableFunctionRemote did not return a StorageDistributed or StorageProxy, got: {}", actual_type); + } + + // Fix the database and table names - this is the same pattern used in InterpreterCreateQuery + // The TableFunctionRemote creates a StorageDistributed with "_table_function" database, + // but we need to rename it to the correct database and table names + distributed_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); + + // Store the filter expression for later use in read operations + // For now, we'll just return the distributed storage + // TODO: Implement filter application in read() method + + return distributed_storage; }, { .supports_settings = false, diff --git a/tests/queries/0_stateless/03372_tiered_distributed_merge.sql b/tests/queries/0_stateless/03372_tiered_distributed_merge.sql index a00481eadfda..5c139972c9ce 100644 --- a/tests/queries/0_stateless/03372_tiered_distributed_merge.sql +++ b/tests/queries/0_stateless/03372_tiered_distributed_merge.sql @@ -6,3 +6,6 @@ CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `nam CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge(url('http://google.com', 'RawBLOB'), 1); -- { serverError BAD_ARGUMENTS } CREATE TABLE test_tiered_distributed_merge_invalid_first_arg ( `id` UInt32, `name` String) ENGINE = TieredDistributedMerge(urlCluster('test_cluster', 'http://example.com')); -- { serverError BAD_ARGUMENTS } CREATE TABLE test_tiered_distributed_merge_invalid_second_arg (`id` UInt32, `name` String) ENGINE = TieredDistributedMerge(remote('test_cluster', 'db', 'table'), 'invalid_predicate');-- { serverError BAD_ARGUMENTS } + +-- Test 8: TieredDistributedMerge with 2 arguments (valid table function + SQL expression) should work +CREATE TABLE test_tiered_distributed_merge_valid (`id` UInt32, `name` String) ENGINE = TieredDistributedMerge(remote('test_cluster', 'db', 'table'), id > 0); diff --git a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference new file mode 100644 index 000000000000..908788a2b5ba --- /dev/null +++ b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference @@ -0,0 +1,26 @@ +10 +6 +4 David 300.2 +4 David 300.2 +5 Eve 250.1 +5 Eve 250.1 +3 Charlie 150.7 +3 Charlie 150.7 +1 Alice 2022-01-01 10:00:00 100.5 +1 Alice 2022-01-01 10:00:00 100.5 +2 Bob 2022-01-02 11:00:00 200.3 +Union + Expression ((Project names + Projection)) + Expression + ReadFromMergeTree (default.test_local_table) + ReadFromRemote (Read from remote replica) +Expression (Project names) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + Projection)) + MergingAggregated + Union + Aggregating + Expression (Before GROUP BY) + Expression + ReadFromMergeTree (default.test_local_table) + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql new file mode 100644 index 000000000000..8550b1970ad4 --- /dev/null +++ b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql @@ -0,0 +1,68 @@ +-- Test TieredDistributedMerge engine with real table connection +-- This test ensures we can actually create tables and select data + +DROP TABLE IF EXISTS test_local_table; +DROP TABLE IF EXISTS test_tiered_real_connection; + +-- Create a local table for testing +CREATE TABLE test_local_table +( + `id` UInt32, + `name` String, + `event_time` DateTime, + `value` Float64 +) ENGINE = MergeTree() +ORDER BY id; + +-- Insert some test data +INSERT INTO test_local_table VALUES + (1, 'Alice', '2022-01-01 10:00:00', 100.5), + (2, 'Bob', '2022-01-02 11:00:00', 200.3), + (3, 'Charlie', '2022-01-03 12:00:00', 150.7), + (4, 'David', '2022-01-04 13:00:00', 300.2), + (5, 'Eve', '2022-01-05 14:00:00', 250.1); + +-- Create TieredDistributedMerge table that connects to localhost (current server) +-- This will create a real connection to the local table +CREATE TABLE test_tiered_real_connection +( + `id` UInt32, + `name` String, + `event_time` DateTime, + `value` Float64 +) ENGINE = TieredDistributedMerge( + remote('127.0.0.1:9000,127.0.0.2:9000', currentDatabase(), 'test_local_table'), + id > 0 +); + +-- Test that we can select data from the TieredDistributedMerge table +-- This should return the same data as the local table +SELECT count() FROM test_tiered_real_connection; + +-- Test with WHERE condition +SELECT count() FROM test_tiered_real_connection WHERE value > 200; + +-- Test with ORDER BY +SELECT id, name, value FROM test_tiered_real_connection WHERE id > 2 ORDER BY value DESC; + +-- Test with LIMIT +SELECT * FROM test_tiered_real_connection ORDER BY id LIMIT 3; + +SET prefer_localhost_replica = 1; -- avoid getting different plans due to that setting + +-- Test EXPLAIN to see the query plan +EXPLAIN SELECT * FROM test_tiered_real_connection WHERE value > 150; + +-- Test EXPLAIN with more complex query +EXPLAIN SELECT + name, + count() as count, + avg(value) as avg_value +FROM test_tiered_real_connection +WHERE event_time >= '2022-01-02' +GROUP BY name +ORDER BY avg_value DESC; + +-- Clean up +DROP TABLE IF EXISTS test_tiered_real_connection; +DROP TABLE IF EXISTS test_local_table; From 0cd7769cc4a37a699117cdd73ee7a158b38f4bc2 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 3 Sep 2025 23:38:22 +0200 Subject: [PATCH 03/20] make the filter work --- src/Storages/StorageDistributed.cpp | 22 ++++++++++-- src/Storages/StorageDistributed.h | 6 ++++ ...red_distributed_merge_real_table.reference | 35 ++++++++++++++++--- ...74_tiered_distributed_merge_real_table.sql | 26 ++++++++++++++ 4 files changed, 81 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86d8ca7ef512..4b59564dd3b5 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -996,12 +996,22 @@ void StorageDistributed::read( if (settings[Setting::allow_experimental_analyzer]) { - StorageID remote_storage_id = StorageID{remote_database, remote_table}; + // Apply additional filter if present (for TieredDistributedMerge) + // Add to filter_asts like additional_table_filters do + if (additional_filter) + { + modified_query_info.filter_asts.push_back(additional_filter); + } + + StorageID remote_storage_id = StorageID::createEmpty(); + if (!remote_table_function_ptr) + remote_storage_id = StorageID{remote_database, remote_table}; auto query_tree_distributed = buildQueryTreeDistributed(modified_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. @@ -1018,6 +1028,13 @@ void StorageDistributed::read( } else { + // Apply additional filter if present (for TieredDistributedMerge) + // Add to filter_asts like additional_table_filters do + if (additional_filter) + { + modified_query_info.filter_asts.push_back(additional_filter); + } + header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); modified_query_info.query = ClusterProxy::rewriteSelectQuery( @@ -2242,8 +2259,7 @@ void registerStorageDistributed(StorageFactory & factory) distributed_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); // Store the filter expression for later use in read operations - // For now, we'll just return the distributed storage - // TODO: Implement filter application in read() method + distributed_storage->setAdditionalFilter(second_arg); return distributed_storage; }, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 784c39e8f755..35c9d0f666f1 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -149,6 +149,9 @@ class StorageDistributed final : public IStorage, WithContext size_t getShardCount() const; + /// Set additional filter for TieredDistributedMerge engine + void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } + bool initializeDiskOnConfigChange(const std::set & new_added_disks) override; private: @@ -282,6 +285,9 @@ class StorageDistributed final : public IStorage, WithContext pcg64 rng; bool is_remote_function; + + /// Additional filter expression for TieredDistributedMerge engine + ASTPtr additional_filter; }; } diff --git a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference index 908788a2b5ba..218a4473a581 100644 --- a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference +++ b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.reference @@ -1,14 +1,11 @@ 10 6 +2 Bob 200.3 +2 Bob 200.3 4 David 300.2 4 David 300.2 5 Eve 250.1 5 Eve 250.1 -3 Charlie 150.7 -3 Charlie 150.7 -1 Alice 2022-01-01 10:00:00 100.5 -1 Alice 2022-01-01 10:00:00 100.5 -2 Bob 2022-01-02 11:00:00 200.3 Union Expression ((Project names + Projection)) Expression @@ -24,3 +21,31 @@ Expression (Project names) Expression ReadFromMergeTree (default.test_local_table) ReadFromRemote (Read from remote replica) +5 +2 Bob 200.3 +2 Bob 200.3 +4 David 300.2 +4 David 300.2 +5 Eve 250.1 +5 Eve 250.1 +0 Invalid 0.5 +0 Invalid 0.5 +1 Alice 100.5 +1 Alice 100.5 +2 Bob 200.3 +2 Bob 200.3 +2 Bob 200.3 +2 Bob 200.3 +2 Bob 200.3 +2 Bob 200.3 +0 Invalid 0.5 +0 Invalid 0.5 +1 Alice 100.5 +1 Alice 100.5 +2 Bob 200.3 +2 Bob 200.3 +Union + Expression ((Project names + Projection)) + Expression + ReadFromMergeTree (default.test_local_table) + ReadFromRemote (Read from remote replica) \ No newline at end of file diff --git a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql index 8550b1970ad4..f3d26c984e78 100644 --- a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql +++ b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql @@ -16,6 +16,7 @@ ORDER BY id; -- Insert some test data INSERT INTO test_local_table VALUES + (0, 'Invalid', '2022-01-01 10:00:00', 0.5), (1, 'Alice', '2022-01-01 10:00:00', 100.5), (2, 'Bob', '2022-01-02 11:00:00', 200.3), (3, 'Charlie', '2022-01-03 12:00:00', 150.7), @@ -63,6 +64,31 @@ WHERE event_time >= '2022-01-02' GROUP BY name ORDER BY avg_value DESC; +-- Test that the additional filter (id > 0) is working correctly +-- This should return all 5 rows since all ids are > 0 +SELECT count() FROM test_tiered_real_connection; + +-- Test with a WHERE condition that should be combined with the additional filter +-- The query should be: SELECT * FROM test_local_table WHERE (id > 0) AND (value > 200) +-- This should return rows with id > 0 AND value > 200 +SELECT id, name, value FROM test_tiered_real_connection WHERE value > 200 ORDER BY id; + +-- Test with a WHERE condition that conflicts with the additional filter +-- The query should be: SELECT * FROM test_local_table WHERE (id > 0) AND (id < 3) +-- This should return rows with id > 0 AND id < 3 (i.e., id = 1, 2) +SELECT id, name, value FROM test_tiered_real_connection WHERE id < 3 ORDER BY id; + +-- should work correctly together with additional_table_filters +SELECT id, name, value FROM test_tiered_real_connection WHERE id < 3 ORDER BY id SETTINGS additional_table_filters = {'test_tiered_real_connection' : 'id > 1'}, allow_experimental_analyzer = 0; +SELECT id, name, value FROM test_tiered_real_connection WHERE id < 3 ORDER BY id SETTINGS additional_table_filters = {'test_tiered_real_connection' : 'id > 1'}, allow_experimental_analyzer = 1; + +SELECT id, name, value FROM test_tiered_real_connection WHERE id < 3 ORDER BY id SETTINGS allow_experimental_analyzer = 0; +SELECT id, name, value FROM test_tiered_real_connection WHERE id < 3 ORDER BY id SETTINGS allow_experimental_analyzer = 1; + + +-- Test EXPLAIN to see how the additional filter is applied +EXPLAIN SELECT * FROM test_tiered_real_connection WHERE value > 200; + -- Clean up DROP TABLE IF EXISTS test_tiered_real_connection; DROP TABLE IF EXISTS test_local_table; From 1d53b706d9a294c6f5a364c96af0ade60e4a7b73 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 3 Sep 2025 23:42:45 +0200 Subject: [PATCH 04/20] whitespace --- .../0_stateless/03374_tiered_distributed_merge_real_table.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql index f3d26c984e78..08c523a10087 100644 --- a/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql +++ b/tests/queries/0_stateless/03374_tiered_distributed_merge_real_table.sql @@ -55,11 +55,11 @@ SET prefer_localhost_replica = 1; -- avoid getting different plans due to that s EXPLAIN SELECT * FROM test_tiered_real_connection WHERE value > 150; -- Test EXPLAIN with more complex query -EXPLAIN SELECT +EXPLAIN SELECT name, count() as count, avg(value) as avg_value -FROM test_tiered_real_connection +FROM test_tiered_real_connection WHERE event_time >= '2022-01-02' GROUP BY name ORDER BY avg_value DESC; From 07d1315cfdaa56fd9a323b4d7eb8e0b0b7c3de3c Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 10 Sep 2025 22:29:34 +0200 Subject: [PATCH 05/20] progress --- .../ClusterProxy/SelectStreamFactory.cpp | 98 +++- .../ClusterProxy/SelectStreamFactory.h | 3 +- .../TranslateQualifiedNamesVisitor.cpp | 11 + .../TranslateQualifiedNamesVisitor.h | 37 ++ src/Parsers/ASTIdentifier.cpp | 11 + src/Parsers/ASTIdentifier.h | 4 + src/Planner/PlannerActionsVisitor.cpp | 2 +- src/Storages/StorageDistributed.cpp | 461 ++++++++++++++++-- src/Storages/StorageDistributed.h | 34 +- ...ibuted_merge_predicate_filtering.reference | 31 ++ ..._distributed_merge_predicate_filtering.sql | 148 ++++++ 11 files changed, 783 insertions(+), 57 deletions(-) create mode 100644 tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.reference create mode 100644 tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.sql diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 2521e7e229ab..dbda11728c8b 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -67,8 +67,16 @@ ASTPtr rewriteSelectQuery( const ASTPtr & query, const std::string & remote_database, const std::string & remote_table, - ASTPtr table_function_ptr) + ASTPtr table_function_ptr, + ASTPtr additional_filter) { + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: About to call rewriteSelectQuery"); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: query: {}", query ? query->formatForErrorMessage() : "null"); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: remote_database: {}", remote_database); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: remote_table: {}", remote_table); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: table_function_ptr: {}", table_function_ptr ? table_function_ptr->formatForErrorMessage() : "null"); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: additional_filter: {}", additional_filter ? additional_filter->formatForErrorMessage() : "null"); + auto modified_query_ast = query->clone(); ASTSelectQuery & select_query = modified_query_ast->as(); @@ -80,23 +88,109 @@ ASTPtr rewriteSelectQuery( if (!context->getSettingsRef()[Setting::allow_experimental_analyzer]) { + // Apply additional filter if provided + if (additional_filter) + { + if (select_query.where()) + { + /// WHERE AND + select_query.setExpression( + ASTSelectQuery::Expression::WHERE, + makeASTFunction("and", select_query.where(), additional_filter)); + } + else + { + /// No WHERE – simply set it + select_query.setExpression( + ASTSelectQuery::Expression::WHERE, additional_filter->clone()); + } + } + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); + + // // Add debug output for semantic->table fields + // if (auto expression_list = select_query.select()) + // { + // for (const auto & child : expression_list->children) + // { + // if (auto identifier = child->as()) + // { + // // access the protected member of IdentifierSemanticImpl through a hack - ok for debugging purposes + // auto semantic = identifier->getSemantic(); + // if (semantic) + // { + // LOG_ERROR(getLogger("Debug"), "DEBUG: Column '{}' semantic properties:", identifier->name()); + // LOG_ERROR(getLogger("Debug"), " - special: {}", semantic->special); + // LOG_ERROR(getLogger("Debug"), " - can_be_alias: {}", semantic->can_be_alias); + // LOG_ERROR(getLogger("Debug"), " - covered: {}", semantic->covered); + // LOG_ERROR(getLogger("Debug"), " - membership: {}", semantic->membership ? std::to_string(*semantic->membership) : "none"); + // LOG_ERROR(getLogger("Debug"), " - table: '{}'", semantic->table); + // LOG_ERROR(getLogger("Debug"), " - legacy_compound: {}", semantic->legacy_compound); + // } + // } + // } + // } + if (table_function_ptr) + { select_query.addTableFunction(table_function_ptr); + + // Reset semantic table information for all column identifiers to prevent + // RestoreQualifiedNamesVisitor from adding wrong table names + ResetSemanticTableVisitor::Data data; + ResetSemanticTableVisitor(data).visit(modified_query_ast); + } else select_query.replaceDatabaseAndTable(remote_database, remote_table); /// Restore long column names (cause our short names are ambiguous). /// TODO: aliased table functions & CREATE TABLE AS table function cases + if (!table_function_ptr) { RestoreQualifiedNamesVisitor::Data data; data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query->as(), 0)); data.remote_table.database = remote_database; data.remote_table.table = remote_table; + + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: About to call RestoreQualifiedNamesVisitor"); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.alias: {}", data.distributed_table.alias); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.table: {}", data.distributed_table.table); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.database: {}", data.distributed_table.database); + + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.alias: {}", data.remote_table.alias); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.table: {}", data.remote_table.table); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.database: {}", data.remote_table.database); + + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); + LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: Successfully completed RestoreQualifiedNamesVisitor"); } } + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); + // Add debug output for semantic->table fields + // if (auto expression_list = select_query.select()) + // { + // for (const auto & child : expression_list->children) + // { + // if (auto identifier = child->as()) + // { + // // access the protected member of IdentifierSemanticImpl through a hack - ok for debugging purposes + // auto semantic = identifier->getSemantic(); + // if (semantic) + // { + // LOG_ERROR(getLogger("Debug"), "DEBUG: Column '{}' semantic properties:", identifier->name()); + // LOG_ERROR(getLogger("Debug"), " - special: {}", semantic->special); + // LOG_ERROR(getLogger("Debug"), " - can_be_alias: {}", semantic->can_be_alias); + // LOG_ERROR(getLogger("Debug"), " - covered: {}", semantic->covered); + // LOG_ERROR(getLogger("Debug"), " - membership: {}", semantic->membership ? std::to_string(*semantic->membership) : "none"); + // LOG_ERROR(getLogger("Debug"), " - table: '{}'", semantic->table); + // LOG_ERROR(getLogger("Debug"), " - legacy_compound: {}", semantic->legacy_compound); + // } + // } + // } + // } + /// To make local JOIN works, default database should be added to table names. /// But only for JOIN section, since the following should work using default_database: /// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value @@ -105,6 +199,8 @@ ASTPtr rewriteSelectQuery( /* only_replace_current_database_function_= */false, /* only_replace_in_join_= */true); visitor.visit(modified_query_ast); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: Successfully completed AddDefaultDatabaseVisitor"); + // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); return modified_query_ast; } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 2d04816fba8a..64ae76c39e7d 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -42,7 +42,8 @@ ASTPtr rewriteSelectQuery( const ASTPtr & query, const std::string & remote_database, const std::string & remote_table, - ASTPtr table_function_ptr = nullptr); + ASTPtr table_function_ptr = nullptr, + ASTPtr additional_filter = nullptr); using ColumnsDescriptionByShardNum = std::unordered_map; using AdditionalShardFilterGenerator = std::function; diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index c21c4d34fa84..5346bd93fd44 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -399,4 +399,15 @@ void RestoreQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &, D } } +void ResetSemanticTableMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * t = ast->as()) + visit(*t, ast, data); +} + +void ResetSemanticTableMatcher::visit(ASTIdentifier & identifier, ASTPtr &, Data &) +{ + identifier.resetSemanticTable(); +} + } diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index 00c85d08873f..e03cd2b4f38d 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -80,4 +80,41 @@ struct RestoreQualifiedNamesMatcher using RestoreQualifiedNamesVisitor = InDepthNodeVisitor; + +/// Reset semantic->table for all column identifiers in the AST. +/// +/// PROBLEM DESCRIPTION: +/// When an AST is passed through multiple query rewrites (e.g., in distributed queries), +/// the semantic->table information attached to ASTIdentifier nodes can become stale and +/// cause incorrect column qualification. This happens because: +/// +/// 1. During initial parsing, semantic->table is populated with the original table name +/// 2. When the query is rewritten (e.g., FROM clause changed from table to remote() function), +/// the AST structure is modified but semantic->table information is preserved +/// 3. Subsequent visitors like RestoreQualifiedNamesVisitor may use this stale semantic->table +/// information to incorrectly qualify column names with the original table name +/// +/// EXAMPLE PROBLEM: +/// Original query: SELECT id FROM my_table WHERE id > 0 +/// After rewrite: SELECT id FROM remote('host', 'db', 'table') WHERE id > 0 +/// Problem: RestoreQualifiedNamesVisitor sees semantic->table = "my_table" and +/// incorrectly produces: SELECT my_table.id FROM remote(...) WHERE my_table.id > 0 +/// +/// SOLUTION: +/// This visitor clears semantic->table for all column identifiers, ensuring that subsequent +/// visitors work with clean semantic information and don't apply stale table qualifications. +struct ResetSemanticTableMatcher +{ + struct Data + { + // No data needed for this visitor + }; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + static void visit(ASTPtr & ast, Data & data); + static void visit(ASTIdentifier & identifier, ASTPtr &, Data & data); +}; + +using ResetSemanticTableVisitor = InDepthNodeVisitor; + } diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index cc7a940b85e1..8c9d51cd8d59 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -169,6 +169,17 @@ void ASTIdentifier::restoreTable() } } +void ASTIdentifier::resetSemanticTable() +{ + // Only reset semantic table for column identifiers (not table identifiers) + if (semantic && !semantic->special) + { + semantic->table.clear(); + semantic->can_be_alias = true; + semantic->membership = std::nullopt; + } +} + std::shared_ptr ASTIdentifier::createTable() const { if (name_parts.size() == 1) return std::make_shared(name_parts[0]); diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 72dde7f644fb..b20a647bb036 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -52,8 +52,12 @@ class ASTIdentifier : public ASTWithAlias void updateTreeHashImpl(SipHash & hash_state, bool ignore_alias) const override; void restoreTable(); // TODO(ilezhankin): get rid of this + void resetSemanticTable(); // Reset semantic to empty string (see ResetSemanticTableVisitor) std::shared_ptr createTable() const; // returns |nullptr| if identifier is not table. + // // FIXME: remove after debugging + // const std::shared_ptr& getSemantic() const { return semantic; } + String full_name; std::vector name_parts; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index dd4070125d5e..af0f41755f59 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -348,7 +348,7 @@ class ActionNodeNameHelper } default: { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid action query tree node {}", node->formatASTForErrorMessage()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid action query tree node {} (node_type: {})", node->formatASTForErrorMessage(), static_cast(node_type)); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4b59564dd3b5..0c5eb2cd6a25 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -5,6 +5,13 @@ #include #include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -95,6 +102,7 @@ #include #include #include +#include #include #include #include @@ -583,6 +591,7 @@ bool StorageDistributed::isShardingKeySuitsQueryTreeNodeExpression( return allOutputsDependsOnlyOnAllowedNodes(sharding_key_dag, irreducibe_nodes, matches); } +// TODO: support additional table functions std::optional StorageDistributed::getOptimizedQueryProcessingStageAnalyzer(const SelectQueryInfo & query_info, const Settings & settings) const { bool optimize_sharding_key_aggregation = settings[Setting::optimize_skip_unused_shards] && settings[Setting::optimize_distributed_group_by_sharding_key] @@ -641,6 +650,7 @@ std::optional StorageDistributed::getOptimizedQueryP return QueryProcessingStage::Complete; } +// TODO: support additional table functions std::optional StorageDistributed::getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, const Settings & settings) const { bool optimize_sharding_key_aggregation = settings[Setting::optimize_skip_unused_shards] && settings[Setting::optimize_distributed_group_by_sharding_key] @@ -750,9 +760,11 @@ static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const { + /// TODO: support additional table functions return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); } +/// TODO: support additional table functions StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr /*query_context*/) const { @@ -888,7 +900,8 @@ bool rewriteJoinToGlobalJoinIfNeeded(QueryTreeNodePtr join_tree) QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, - const ASTPtr & remote_table_function) + const ASTPtr & remote_table_function, + const ASTPtr & additional_filter = nullptr) { auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -955,7 +968,45 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, replacement_table_expression->setAlias(query_info.table_expression->getAlias()); + + QueryTreeNodePtr filter; + // static auto logger = getLogger("StorageDistributed"); + + if (additional_filter) + { + const auto & context = query_info.planner_context->getQueryContext(); + + filter = buildQueryTree(additional_filter->clone(), query_context); + + // LOG_ERROR(logger, "DEBUG: About to call QueryAnalysisPass(replacement_table_expression).run(filter, context)"); + // LOG_ERROR(logger, "DEBUG: filter: {}", filter->dumpTree()); + + QueryAnalysisPass(replacement_table_expression).run(filter, context); + + // LOG_ERROR(logger, "DEBUG: filter after pass.run: {}", filter->dumpTree()); + + } + auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); + + // Apply additional filter if provided + if (filter) + { + // LOG_ERROR(logger, "DEBUG: query_tree_to_modify: {}", query_tree_to_modify->dumpTree()); + + auto & query = query_tree_to_modify->as(); + query.getWhere() = query.hasWhere() + ? mergeConditionNodes({query.getWhere(), filter}, query_context) + : std::move(filter); + + // LOG_ERROR(logger, "DEBUG: query_tree_to_modify after mergeConditionNodes: {}", query_tree_to_modify->dumpTree()); + + // QueryAnalysisPass pass; + // pass.run(query_tree_to_modify, query_context); + + // LOG_ERROR(logger, "DEBUG: query_tree_to_modify after pass.run: {}", query_tree_to_modify->dumpTree()); + } + ReplaseAliasColumnsVisitor replase_alias_columns_visitor; replase_alias_columns_visitor.visit(query_tree_to_modify); @@ -974,6 +1025,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, } return buildQueryTreeForShard(query_info.planner_context, query_tree_to_modify, /*allow_global_join_for_right_table*/ false); + } } @@ -992,56 +1044,141 @@ void StorageDistributed::read( SelectQueryInfo modified_query_info = query_info; + std::vector all_headers; + std::vector all_query_infos; + const auto & settings = local_context->getSettingsRef(); if (settings[Setting::allow_experimental_analyzer]) { - // Apply additional filter if present (for TieredDistributedMerge) - // Add to filter_asts like additional_table_filters do - if (additional_filter) - { - modified_query_info.filter_asts.push_back(additional_filter); - } - StorageID remote_storage_id = StorageID::createEmpty(); if (!remote_table_function_ptr) remote_storage_id = StorageID{remote_database, remote_table}; + // LOG_ERROR(log, "DEBUG: About to call buildQueryTreeDistributed for main query"); + // LOG_ERROR(log, "DEBUG: modified_query_info.query_tree: {}", modified_query_info.query_tree ? modified_query_info.query_tree->formatASTForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: remote_storage_id: {}", remote_storage_id.getFullNameNotQuoted()); + // LOG_ERROR(log, "DEBUG: remote_table_function_ptr: {}", remote_table_function_ptr ? remote_table_function_ptr->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: additional_filter: {}", additional_filter ? additional_filter->formatForErrorMessage() : "null"); auto query_tree_distributed = buildQueryTreeDistributed(modified_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, - remote_table_function_ptr); + remote_table_function_ptr, + additional_filter); + // LOG_ERROR(log, "DEBUG: Successfully completed buildQueryTreeDistributed for main query"); + // LOG_ERROR(log, "DEBUG: query_tree_distributed: {}", query_tree_distributed ? query_tree_distributed->formatASTForErrorMessage() : "null"); + + + // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQueryAnalyzer::getSampleBlock"); + // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); + SelectQueryOptions options = SelectQueryOptions(processed_stage).analyze(); + // LOG_ERROR(log, "DEBUG: SelectQueryOptions created"); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, options); + // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQueryAnalyzer::getSampleBlock"); - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ for (auto & column : header) column.column = column.column->convertToFullColumnIfConst(); + modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); - /// Return directly (with correct header) if no shard to query. - if (modified_query_info.getCluster()->getShardsInfo().empty()) + if (!additional_table_functions.empty()) + { + for (const auto & table_function_entry : additional_table_functions) + { + // Create a modified query info with the additional predicate + SelectQueryInfo additional_query_info = query_info; + + // LOG_ERROR(log, "DEBUG: About to call buildQueryTreeDistributed for additional table function"); + // LOG_ERROR(log, "DEBUG: additional_query_info.query_tree: {}", additional_query_info.query_tree ? additional_query_info.query_tree->formatASTForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: table_function_entry.table_function_ast: {}", table_function_entry.table_function_ast ? table_function_entry.table_function_ast->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: table_function_entry.predicate_ast: {}", table_function_entry.predicate_ast ? table_function_entry.predicate_ast->formatForErrorMessage() : "null"); + auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, + query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, + StorageID::createEmpty(), + table_function_entry.table_function_ast, + table_function_entry.predicate_ast); + // LOG_ERROR(log, "DEBUG: Successfully completed buildQueryTreeDistributed for additional table function"); + // LOG_ERROR(log, "DEBUG: additional_query_tree: {}", additional_query_tree ? additional_query_tree->formatASTForErrorMessage() : "null"); + + // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQueryAnalyzer::getSampleBlock for additional table function"); + + + // TODO: somewhere here the DESCRIBE TABLE is triggered, try to avoid it. + auto additional_header = InterpreterSelectQueryAnalyzer::getSampleBlock(additional_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + + // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQueryAnalyzer::getSampleBlock for additional table function"); + for (auto & column : additional_header) + column.column = column.column->convertToFullColumnIfConst(); + + additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); + additional_query_info.query_tree = std::move(additional_query_tree); + + all_headers.push_back(additional_header); + all_query_infos.push_back(additional_query_info); + } + } + + // For empty shards - avoid early return if we have additional table functions + if (modified_query_info.getCluster()->getShardsInfo().empty() && additional_table_functions.empty()) return; } else { - // Apply additional filter if present (for TieredDistributedMerge) - // Add to filter_asts like additional_table_filters do - if (additional_filter) - { - modified_query_info.filter_asts.push_back(additional_filter); - } + // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); + // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); + // LOG_ERROR(log, "DEBUG: header: {}", header.dumpStructure()); + // LOG_ERROR(log, "DEBUG: About to call ClusterProxy::rewriteSelectQuery"); + modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, - remote_database, remote_table, remote_table_function_ptr); + remote_database, remote_table, remote_table_function_ptr, + additional_filter); + + // LOG_ERROR(log, "DEBUG: Successfully completed ClusterProxy::rewriteSelectQuery"); + // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); - if (modified_query_info.getCluster()->getShardsInfo().empty()) + if (!additional_table_functions.empty()) + { + for (const auto & table_function_entry : additional_table_functions) + { + SelectQueryInfo additional_query_info = query_info; + + // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); + // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); + + auto additional_header = InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); + // LOG_ERROR(log, "DEBUG: additional_header: {}", additional_header.dumpStructure()); + // LOG_ERROR(log, "DEBUG: About to call ClusterProxy::rewriteSelectQuery"); + + additional_query_info.query = ClusterProxy::rewriteSelectQuery( + local_context, additional_query_info.query, + "", "", table_function_entry.table_function_ast, + table_function_entry.predicate_ast); + + // LOG_ERROR(log, "DEBUG: Successfully completed ClusterProxy::rewriteSelectQuery"); + // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); + + all_headers.push_back(additional_header); + all_query_infos.push_back(additional_query_info); + } + } + + // For empty shards - avoid early return if we have additional table functions + if (modified_query_info.getCluster()->getShardsInfo().empty() && additional_table_functions.empty()) { Pipe pipe(std::make_shared(header)); auto read_from_pipe = std::make_unique(std::move(pipe)); @@ -1053,35 +1190,159 @@ void StorageDistributed::read( } const auto & snapshot_data = assert_cast(*storage_snapshot->data); - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( + + if (!modified_query_info.getCluster()->getShardsInfo().empty()) + { + ClusterProxy::SelectStreamFactory select_stream_factory = + ClusterProxy::SelectStreamFactory( + header, + snapshot_data.objects_by_shard, + storage_snapshot, + processed_stage); + + auto shard_filter_generator = ClusterProxy::getShardFilterGeneratorForCustomKey( + *modified_query_info.getCluster(), local_context, getInMemoryMetadataPtr()->columns); + + ClusterProxy::executeQuery( + query_plan, header, - snapshot_data.objects_by_shard, - storage_snapshot, - processed_stage); - - auto shard_filter_generator = ClusterProxy::getShardFilterGeneratorForCustomKey( - *modified_query_info.getCluster(), local_context, getInMemoryMetadataPtr()->columns); - - ClusterProxy::executeQuery( - query_plan, - header, - processed_stage, - remote_storage, - remote_table_function_ptr, - select_stream_factory, - log, - local_context, - modified_query_info, - sharding_key_expr, - sharding_key_column_name, - *distributed_settings, - shard_filter_generator, - is_remote_function); - - /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. - if (!query_plan.isInitialized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline is not initialized"); + processed_stage, + remote_storage, + remote_table_function_ptr, + select_stream_factory, + log, + local_context, + modified_query_info, + sharding_key_expr, + sharding_key_column_name, + *distributed_settings, + shard_filter_generator, + is_remote_function); + + /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. + if (!query_plan.isInitialized()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline is not initialized"); + } + + std::vector additional_plans; + for (size_t i = 0; i < all_query_infos.size(); ++i) + { + auto additional_query_info = all_query_infos[i]; + const auto & storage = additional_table_functions[i].storage; + auto additional_header = all_headers[i]; + + // LOG_ERROR(log, "DEBUG: About to call storage->read for additional storage"); + // LOG_ERROR(log, "DEBUG: additional_header: {}", additional_header.dumpStructure()); + // LOG_ERROR(log, "DEBUG: storage: {}", storage->getName()); + // LOG_ERROR(log, "DEBUG: additional_query_info: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); + // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); + + + // Create a new query plan for this additional storage + QueryPlan additional_plan; + // Execute the query against the additional storage + storage->read( + additional_plan, + {}, // column names + storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), local_context), + additional_query_info, + local_context, + processed_stage, + 0, // max_block_size + 0); // num_streams + + // LOG_ERROR(log, "DEBUG: Successfully completed storage->read for additional storage"); + + additional_plans.push_back(std::move(additional_plan)); + } + + // Combine all plans using UnionStep + if (!additional_plans.empty()) + { + // Convert QueryPlan objects to QueryPlanPtr + std::vector plan_ptrs; + plan_ptrs.reserve(additional_plans.size() + 1); + + // Add the main plan to the list + plan_ptrs.push_back(std::make_unique(std::move(query_plan))); + + // Add additional plans + for (auto & plan : additional_plans) + { + plan_ptrs.push_back(std::make_unique(std::move(plan))); + } + + // Create a new query plan that unions all the results + QueryPlan union_plan; + + // Get headers from all plans + std::vector headers; + headers.reserve(plan_ptrs.size()); + for (const auto & plan_ptr : plan_ptrs) + { + headers.push_back(plan_ptr->getCurrentHeader()); + } + + // Create UnionStep to combine all plans + auto union_step = std::make_unique(std::move(headers), 0); + + + /* TODO!!: simple union step is not enough, we need to add a conversion step to match the headers + +SELECT + id % 2 AS x, + count() +FROM test_tiered_predicate_filtering_analyzer_off +GROUP BY x +ORDER BY x ASC + + +SELECT + hostName(), + id % 2 AS x, + count() +FROM test_tiered_predicate_filtering_analyzer_off +GROUP BY x +ORDER BY x ASC + +Query id: 88fbdc56-33de-468d-87ef-155121cdbea2 + + +Elapsed: 0.012 sec. + +[mfilimonov-MS-7E12] 2025.09.10 22:24:47.280713 [ 399165 ] {70b3d45e-77e5-4acd-b3af-3e1e4b6c1e58} executeQuery: Code: 60. DB::Exception: Unknown table expression identifier 'test_tiered_predicate_filtering_analyzer_off' in scope SELECT hostName(), id % 2 AS x, count() FROM test_tiered_predicate_filtering_analyzer_off GROUP BY x ORDER BY x ASC. (UNKNOWN_TABLE) (version v25.6.2.20000.altinityantalya.28000) (from [::ffff:127.0.0.1]:56130) (query 1, line 1) (in query: SELECT hostName(), id % 2 as x, count() FROM test_tiered_predicate_filtering_analyzer_off group by x ORDER BY x ASC ;), Stack trace (when copying this message, always include the lines below): + +0. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__exception/exception.h:113: Poco::Exception::Exception(String const&, int) @ 0x000000001bc06a92 +1. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.cpp:115: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x00000000107b4b5b +2. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.h:119: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x000000000a2d1e2c +3. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.h:137: DB::Exception::Exception(int, FormatStringHelperImpl::type, std::type_identity::type>, String const&, String&&) @ 0x000000000a946e8b +4. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:4475: DB::QueryAnalyzer::initializeQueryJoinTreeNode(std::shared_ptr&, DB::IdentifierResolveScope&) @ 0x0000000014f28356 +5. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:5755: DB::QueryAnalyzer::resolveQuery(std::shared_ptr const&, DB::IdentifierResolveScope&) @ 0x0000000014ef57bb +6. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:181: DB::QueryAnalyzer::resolve(std::shared_ptr&, std::shared_ptr const&, std::shared_ptr) @ 0x0000000014ef493a +7. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalysisPass.cpp:18: DB::QueryAnalysisPass::run(std::shared_ptr&, std::shared_ptr) @ 0x0000000014ef414e +8. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/QueryTreePassManager.cpp:187: DB::QueryTreePassManager::run(std::shared_ptr) @ 0x0000000014f5246a +9. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp:165: DB::buildQueryTreeAndRunPasses(std::shared_ptr const&, DB::SelectQueryOptions const&, std::shared_ptr const&, std::shared_ptr const&) @ 0x0000000015903a3c +10. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp:182: DB::InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(std::shared_ptr const&, std::shared_ptr const&, DB::SelectQueryOptions const&, std::vector> const&) @ 0x000000001590222d +11. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:634: std::__unique_if::__unique_single std::make_unique[abi:se190107]&, std::shared_ptr const&, DB::SelectQueryOptions const&>(std::shared_ptr&, std::shared_ptr const&, DB::SelectQueryOptions const&) @ 0x0000000015904c04 +12. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x000000001589c9cb +13. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/executeQuery.cpp:1455: DB::executeQueryImpl(char const*, char const*, std::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum, DB::ReadBuffer*, std::shared_ptr&) @ 0x0000000015bd48c0 +14. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/executeQuery.cpp:1715: DB::executeQuery(String const&, std::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum) @ 0x0000000015bcf388 +15. /home/mfilimonov/workspace/ClickHouse/master/src/Server/TCPHandler.cpp:721: DB::TCPHandler::runImpl() @ 0x0000000017920812 +16. /home/mfilimonov/workspace/ClickHouse/master/src/Server/TCPHandler.cpp:2727: DB::TCPHandler::run() @ 0x0000000017939bb9 +17. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Net/src/TCPServerConnection.cpp:40: Poco::Net::TCPServerConnection::start() @ 0x000000001bca8687 +18. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Net/src/TCPServerDispatcher.cpp:115: Poco::Net::TCPServerDispatcher::run() @ 0x000000001bca8b5e +19. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Foundation/src/ThreadPool.cpp:205: Poco::PooledThread::run() @ 0x000000001bc52e32 +20. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Foundation/src/Thread_POSIX.cpp:335: Poco::ThreadImpl::runnableEntry(void*) @ 0x000000001bc50a6f +21. start_thread @ 0x00000000000a27f1 +22. __GI___clone3 @ 0x0000000000133c9c + + +*/ + union_plan.unitePlans(std::move(union_step), std::move(plan_ptrs)); + + // Replace the original query plan with the union plan + query_plan = std::move(union_plan); + } } @@ -2178,7 +2439,8 @@ void registerStorageDistributed(StorageFactory & factory) .has_builtin_setting_fn = DistributedSettings::hasBuiltin, }); - // Register TieredDistributedMerge engine - Step 3: Add first argument validation + // Register TieredDistributedMerge engine + // TODO: consider moving it to a separate file / subclass of StorageDistributed factory.registerStorage("TieredDistributedMerge", [](const StorageFactory::Arguments & args) -> StoragePtr { ASTs & engine_args = args.engine_args; @@ -2208,7 +2470,7 @@ void registerStorageDistributed(StorageFactory & factory) "First argument must be a table function, got: {}", first_arg->getID()); } - // Validate second argument - must be a SQL expression (not a string literal) + // TODO: Validate second argument - must be a SQL expression (just rejecting a string literal for now) ASTPtr second_arg = engine_args[1]; if (const auto * literal = second_arg->as()) { @@ -2223,16 +2485,106 @@ void registerStorageDistributed(StorageFactory & factory) // Create the underlying StorageDistributed using the table function const ContextPtr & context = args.getContext(); + // Parse additional table function pairs (if any) + std::vector additional_table_functions; + for (size_t i = 2; i < engine_args.size(); i += 2) + { + if (i + 1 >= engine_args.size()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Table function pairs must have both table function and predicate, got odd number of arguments"); + + ASTPtr table_function_ast = engine_args[i]; + ASTPtr predicate_ast = engine_args[i + 1]; + + // Validate table function + const auto * func = table_function_ast->as(); + if (!func) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Additional table function must be a table function, got: {}", table_function_ast->getID()); + } + else if (!TableFunctionFactory::instance().isTableFunctionName(func->name)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: additional table function must be a valid table function, got: {}", i, func->name); + } + + // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster + // Check if it's a supported table function type + String table_function_name = func->name; + + // Check for ITableFunctionCluster types (ending with "Cluster") + // Use a whitelist of supported table function names for clarity and safety + static const std::unordered_set supported_table_functions = { + "remote", "remoteSecure", "cluster", "clusterAllReplicas", + "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", + "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" + }; + + if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: additional table function '{}' is not supported. " + "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " + "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " + "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); + } + + // TODO: Validate predicate - must be a SQL expression (just rejecting a string literal for now) + if (const auto * literal = predicate_ast->as()) + { + if (literal->value.getType() == Field::Types::String) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Additional predicate must be a SQL expression, got string literal"); + } + } + + // Create table function instance and execute it to get StoragePtr + auto table_function = TableFunctionFactory::instance().get(table_function_ast, context); + if (!table_function) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid additional table function in TieredDistributedMerge engine"); + + // Execute the table function to get the underlying storage + StoragePtr additional_storage = table_function->execute( + table_function_ast, + context, + args.table_id.table_name, + args.columns, // Use the same columns as the main table function + false, // use_global_context = false + false); // is_insert_query = false + + // Handle StorageTableFunctionProxy if present + if (auto proxy = std::dynamic_pointer_cast(additional_storage)) + { + additional_storage = proxy->getNested(); + } + + // additional_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); + + + additional_table_functions.emplace_back(std::move(additional_storage), table_function_ast, predicate_ast); + } + + // Now handle the first table function (which must be a TableFunctionRemote) auto table_function = TableFunctionFactory::instance().get(first_arg, context); if (!table_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); + // For schema inference, we need to determine the columns first if they're not provided + ColumnsDescription columns_to_use = args.columns; + if (columns_to_use.empty()) + { + // Get the column structure from the table function + columns_to_use = table_function->getActualTableStructure(context, true); + } + // Execute the table function to get the underlying storage StoragePtr storage = table_function->execute( first_arg, context, args.table_id.table_name, - args.columns, + columns_to_use, false, // use_global_context = false false); // is_insert_query = false @@ -2261,12 +2613,15 @@ void registerStorageDistributed(StorageFactory & factory) // Store the filter expression for later use in read operations distributed_storage->setAdditionalFilter(second_arg); + // Store additional table functions for later use + distributed_storage->setAdditionalTableFunctions(std::move(additional_table_functions)); + return distributed_storage; }, { .supports_settings = false, - .supports_parallel_insert = false, - .supports_schema_inference = false, + .supports_parallel_insert = true, + .supports_schema_inference = true, .source_access_type = AccessType::REMOTE, }); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 35c9d0f666f1..1a756de5b708 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -50,6 +50,20 @@ class StorageDistributed final : public IStorage, WithContext friend class StorageSystemDistributionQueue; public: + /// Structure to hold storage, its AST, and associated predicate + struct TableFunctionEntry + { + StoragePtr storage; + ASTPtr table_function_ast; + ASTPtr predicate_ast; + + TableFunctionEntry(StoragePtr storage_, ASTPtr table_function_ast_, ASTPtr predicate_ast_) + : storage(std::move(storage_)) + , table_function_ast(std::move(table_function_ast_)) + , predicate_ast(std::move(predicate_ast_)) + {} + }; + StorageDistributed( const StorageID & id_, const ColumnsDescription & columns_, @@ -70,7 +84,10 @@ class StorageDistributed final : public IStorage, WithContext ~StorageDistributed() override; - std::string getName() const override { return "Distributed"; } + std::string getName() const override + { + return additional_table_functions.empty() ? "Distributed" : "TieredDistributedMerge"; + } bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } @@ -152,6 +169,18 @@ class StorageDistributed final : public IStorage, WithContext /// Set additional filter for TieredDistributedMerge engine void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } + /// Set additional table functions for TieredDistributedMerge engine + void setAdditionalTableFunctions(std::vector additional_table_functions_) + { + additional_table_functions = std::move(additional_table_functions_); + } + + /// Getter methods for ClusterProxy::executeQuery + StorageID getRemoteStorageID() const { return remote_storage; } + ExpressionActionsPtr getShardingKeyExpression() const { return sharding_key_expr; } + const DistributedSettings * getDistributedSettings() const { return distributed_settings.get(); } + bool isRemoteFunction() const { return is_remote_function; } + bool initializeDiskOnConfigChange(const std::set & new_added_disks) override; private: @@ -288,6 +317,9 @@ class StorageDistributed final : public IStorage, WithContext /// Additional filter expression for TieredDistributedMerge engine ASTPtr additional_filter; + + /// Additional table functions for TieredDistributedMerge engine + std::vector additional_table_functions; }; } diff --git a/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.reference b/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.reference new file mode 100644 index 000000000000..87577cf0fc65 --- /dev/null +++ b/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.reference @@ -0,0 +1,31 @@ +Test 1: Predicate filtering with analyzer OFF +6 +3 +3 +Test 2: Virtual column _table_index with analyzer OFF +1 3 +2 3 +Test 3: Predicate filtering with analyzer ON +6 +3 +3 +Test 4: Virtual column _table_index with analyzer ON +1 3 +2 3 +Test 5: Complex predicate filtering +2 +1 1 +2 1 +Test 6: Data integrity check +1 4 David 2025-09-05 400 +1 5 Eve 2025-09-10 500 +1 6 Frank 2025-09-15 600 +2 1 Alice 2025-08-15 100 +2 2 Bob 2025-08-20 200 +2 3 Charlie 2025-08-25 300 +Test 7: Additional WHERE clause +2 +3 +Test 8: Additional WHERE clause with analyzer ON +2 +3 diff --git a/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.sql b/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.sql new file mode 100644 index 000000000000..68afff56b7fe --- /dev/null +++ b/tests/queries/0_stateless/03376_tiered_distributed_merge_predicate_filtering.sql @@ -0,0 +1,148 @@ +-- Tags: no-random-merge-tree-settings + +-- Test TieredDistributedMerge engine predicate filtering and virtual column functionality + +DROP TABLE IF EXISTS test_table1_local SYNC; +DROP TABLE IF EXISTS test_table2_local SYNC; +DROP TABLE IF EXISTS test_tiered_predicate_filtering_analyzer_off SYNC; + +-- Create local tables with data before and after watermark +CREATE TABLE test_table1_local +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt32 +) +ENGINE = MergeTree() +ORDER BY id; + +CREATE TABLE test_table2_local +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt32 +) +ENGINE = MergeTree() +ORDER BY id; + +-- Insert data before watermark (2025-09-01) +INSERT INTO test_table1_local VALUES + (11, 'Alice', '2025-08-15', 100), + (12, 'Bob', '2025-08-20', 200), + (13, 'Charlie', '2025-08-25', 300); + +INSERT INTO test_table2_local VALUES + (21, 'Alice', '2025-08-15', 100), + (22, 'Bob', '2025-08-20', 200), + (23, 'Charlie', '2025-08-25', 300); + +-- Insert data after watermark (2025-09-01) +INSERT INTO test_table1_local VALUES + (14, 'David', '2025-09-05', 400), + (15, 'Eve', '2025-09-10', 500), + (16, 'Frank', '2025-09-15', 600); + +INSERT INTO test_table2_local VALUES + (24, 'David', '2025-09-05', 400), + (25, 'Eve', '2025-09-10', 500), + (26, 'Frank', '2025-09-15', 600); + +-- Test 1: Basic predicate filtering with analyzer disabled +SET allow_experimental_analyzer = 1; + +CREATE TABLE test_tiered_predicate_filtering_analyzer_off +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt32 +) +ENGINE = TieredDistributedMerge( + remote('127.0.0.2:9000', currentDatabase(), 'test_table1_local'), + date >= '2025-09-01', + remote('127.0.0.2:9000', currentDatabase(), 'test_table2_local'), + date < '2025-09-01' +); + +-- Test predicate filtering - should return only data after watermark from table1 and before watermark from table2 +SELECT 'Test 1: Predicate filtering with analyzer OFF' as test_name; +SELECT * FROM test_tiered_predicate_filtering_analyzer_off ORDER BY id; +SELECT count() as total_rows FROM test_tiered_predicate_filtering_analyzer_off; +SELECT count() as rows_after_watermark FROM test_tiered_predicate_filtering_analyzer_off WHERE date >= '2025-09-01'; +SELECT count() as rows_before_watermark FROM test_tiered_predicate_filtering_analyzer_off WHERE date < '2025-09-01'; + +-- Test virtual column functionality +SELECT 'Test 2: Virtual column _table_index with analyzer OFF' as test_name; +SELECT _table_index, count() as row_count FROM test_tiered_predicate_filtering_analyzer_off GROUP BY _table_index ORDER BY _table_index; + +-- Test 3: Basic predicate filtering with analyzer enabled +SET allow_experimental_analyzer = 1; + +CREATE TABLE test_tiered_predicate_filtering_analyzer_on +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt32 +) +ENGINE = TieredDistributedMerge( + remote('127.0.0.1:9000,127.0.0.2:9000', currentDatabase(), 'test_table1_local'), + date >= '2025-09-01', + remote('127.0.0.1:9000,127.0.0.2:9000', currentDatabase(), 'test_table2_local'), + date < '2025-09-01' +); + +-- Test predicate filtering - should return only data after watermark from table1 and before watermark from table2 +SELECT 'Test 3: Predicate filtering with analyzer ON' as test_name; +SELECT count() as total_rows FROM test_tiered_predicate_filtering_analyzer_on; +SELECT count() as rows_after_watermark FROM test_tiered_predicate_filtering_analyzer_on WHERE date >= '2025-09-01'; +SELECT count() as rows_before_watermark FROM test_tiered_predicate_filtering_analyzer_on WHERE date < '2025-09-01'; + +-- Test virtual column functionality +SELECT 'Test 4: Virtual column _table_index with analyzer ON' as test_name; +SELECT _table_index, count() as row_count FROM test_tiered_predicate_filtering_analyzer_on GROUP BY _table_index ORDER BY _table_index; + +-- Test 5: Complex predicate with multiple conditions +SET allow_experimental_analyzer = 0; + +CREATE TABLE test_tiered_complex_predicate +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt32 +) +ENGINE = TieredDistributedMerge( + remote('127.0.0.1:9000,127.0.0.2:9000', currentDatabase(), 'test_table1_local'), + date >= '2025-09-01' AND value > 400, + remote('127.0.0.1:9000,127.0.0.2:9000', currentDatabase(), 'test_table2_local'), + date < '2025-09-01' AND value < 300 +); + +SELECT 'Test 5: Complex predicate filtering' as test_name; +SELECT count() as total_rows FROM test_tiered_complex_predicate; +SELECT _table_index, count() as row_count FROM test_tiered_complex_predicate GROUP BY _table_index ORDER BY _table_index; + +-- Test 6: Verify data integrity - check specific values +SELECT 'Test 6: Data integrity check' as test_name; +SELECT _table_index, id, name, date, value FROM test_tiered_predicate_filtering_analyzer_off ORDER BY _table_index, id; + +-- Test 7: Test with additional WHERE clause on top of engine predicates +SELECT 'Test 7: Additional WHERE clause' as test_name; +SELECT count() as alice_rows FROM test_tiered_predicate_filtering_analyzer_off WHERE name = 'Alice'; +SELECT count() as high_value_rows FROM test_tiered_predicate_filtering_analyzer_off WHERE value > 300; + +-- Test 8: Test with analyzer enabled and additional WHERE clause +SET allow_experimental_analyzer = 1; +SELECT 'Test 8: Additional WHERE clause with analyzer ON' as test_name; +SELECT count() as alice_rows FROM test_tiered_predicate_filtering_analyzer_on WHERE name = 'Alice'; +SELECT count() as high_value_rows FROM test_tiered_predicate_filtering_analyzer_on WHERE value > 300; + +-- Clean up +DROP TABLE IF EXISTS test_tiered_predicate_filtering_analyzer_off; +DROP TABLE IF EXISTS test_tiered_predicate_filtering_analyzer_on; +DROP TABLE IF EXISTS test_tiered_complex_predicate; +DROP TABLE IF EXISTS test_table1_local; +DROP TABLE IF EXISTS test_table2_local; From 11fc47e93119e2384d0af2fc39c00ce223b466b6 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 11 Sep 2025 14:50:13 +0200 Subject: [PATCH 06/20] fix the processing stage, clean debug lines --- .../ClusterProxy/SelectStreamFactory.cpp | 69 ---------- .../TranslateQualifiedNamesVisitor.h | 20 +-- src/Parsers/ASTIdentifier.h | 3 - src/Storages/StorageDistributed.cpp | 128 +----------------- 4 files changed, 13 insertions(+), 207 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index dbda11728c8b..7297cb6f70c7 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -70,13 +70,6 @@ ASTPtr rewriteSelectQuery( ASTPtr table_function_ptr, ASTPtr additional_filter) { - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: About to call rewriteSelectQuery"); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: query: {}", query ? query->formatForErrorMessage() : "null"); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: remote_database: {}", remote_database); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: remote_table: {}", remote_table); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: table_function_ptr: {}", table_function_ptr ? table_function_ptr->formatForErrorMessage() : "null"); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: additional_filter: {}", additional_filter ? additional_filter->formatForErrorMessage() : "null"); - auto modified_query_ast = query->clone(); ASTSelectQuery & select_query = modified_query_ast->as(); @@ -105,30 +98,6 @@ ASTPtr rewriteSelectQuery( ASTSelectQuery::Expression::WHERE, additional_filter->clone()); } } - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); - - // // Add debug output for semantic->table fields - // if (auto expression_list = select_query.select()) - // { - // for (const auto & child : expression_list->children) - // { - // if (auto identifier = child->as()) - // { - // // access the protected member of IdentifierSemanticImpl through a hack - ok for debugging purposes - // auto semantic = identifier->getSemantic(); - // if (semantic) - // { - // LOG_ERROR(getLogger("Debug"), "DEBUG: Column '{}' semantic properties:", identifier->name()); - // LOG_ERROR(getLogger("Debug"), " - special: {}", semantic->special); - // LOG_ERROR(getLogger("Debug"), " - can_be_alias: {}", semantic->can_be_alias); - // LOG_ERROR(getLogger("Debug"), " - covered: {}", semantic->covered); - // LOG_ERROR(getLogger("Debug"), " - membership: {}", semantic->membership ? std::to_string(*semantic->membership) : "none"); - // LOG_ERROR(getLogger("Debug"), " - table: '{}'", semantic->table); - // LOG_ERROR(getLogger("Debug"), " - legacy_compound: {}", semantic->legacy_compound); - // } - // } - // } - // } if (table_function_ptr) { @@ -144,7 +113,6 @@ ASTPtr rewriteSelectQuery( /// Restore long column names (cause our short names are ambiguous). /// TODO: aliased table functions & CREATE TABLE AS table function cases - if (!table_function_ptr) { RestoreQualifiedNamesVisitor::Data data; @@ -152,45 +120,10 @@ ASTPtr rewriteSelectQuery( data.remote_table.database = remote_database; data.remote_table.table = remote_table; - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: About to call RestoreQualifiedNamesVisitor"); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.alias: {}", data.distributed_table.alias); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.table: {}", data.distributed_table.table); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.distributed_table.database: {}", data.distributed_table.database); - - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.alias: {}", data.remote_table.alias); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.table: {}", data.remote_table.table); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: data.remote_table.database: {}", data.remote_table.database); - - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); - LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: Successfully completed RestoreQualifiedNamesVisitor"); } } - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); - // Add debug output for semantic->table fields - // if (auto expression_list = select_query.select()) - // { - // for (const auto & child : expression_list->children) - // { - // if (auto identifier = child->as()) - // { - // // access the protected member of IdentifierSemanticImpl through a hack - ok for debugging purposes - // auto semantic = identifier->getSemantic(); - // if (semantic) - // { - // LOG_ERROR(getLogger("Debug"), "DEBUG: Column '{}' semantic properties:", identifier->name()); - // LOG_ERROR(getLogger("Debug"), " - special: {}", semantic->special); - // LOG_ERROR(getLogger("Debug"), " - can_be_alias: {}", semantic->can_be_alias); - // LOG_ERROR(getLogger("Debug"), " - covered: {}", semantic->covered); - // LOG_ERROR(getLogger("Debug"), " - membership: {}", semantic->membership ? std::to_string(*semantic->membership) : "none"); - // LOG_ERROR(getLogger("Debug"), " - table: '{}'", semantic->table); - // LOG_ERROR(getLogger("Debug"), " - legacy_compound: {}", semantic->legacy_compound); - // } - // } - // } - // } - /// To make local JOIN works, default database should be added to table names. /// But only for JOIN section, since the following should work using default_database: /// - SELECT * FROM d WHERE value IN (SELECT l.value FROM l) ORDER BY value @@ -199,8 +132,6 @@ ASTPtr rewriteSelectQuery( /* only_replace_current_database_function_= */false, /* only_replace_in_join_= */true); visitor.visit(modified_query_ast); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: Successfully completed AddDefaultDatabaseVisitor"); - // LOG_ERROR(getLogger("ClusterProxy::SelectStreamFactory"), "DEBUG: modified_query_ast: {}", modified_query_ast ? modified_query_ast->dumpTree() : "null"); return modified_query_ast; } diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index e03cd2b4f38d..474ee20b72fd 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -84,31 +84,23 @@ using RestoreQualifiedNamesVisitor = InDepthNodeVisitortable for all column identifiers in the AST. /// /// PROBLEM DESCRIPTION: -/// When an AST is passed through multiple query rewrites (e.g., in distributed queries), +/// When an AST is passed through multiple query rewrites (e.g., in TieredDistributedMerge -> remote), /// the semantic->table information attached to ASTIdentifier nodes can become stale and /// cause incorrect column qualification. This happens because: /// /// 1. During initial parsing, semantic->table is populated with the original table name -/// 2. When the query is rewritten (e.g., FROM clause changed from table to remote() function), +/// 2. When the query is rewritten (e.g., FROM clause changed from table to remote() function inside TieredDistributedMerge), /// the AST structure is modified but semantic->table information is preserved -/// 3. Subsequent visitors like RestoreQualifiedNamesVisitor may use this stale semantic->table -/// information to incorrectly qualify column names with the original table name -/// -/// EXAMPLE PROBLEM: -/// Original query: SELECT id FROM my_table WHERE id > 0 -/// After rewrite: SELECT id FROM remote('host', 'db', 'table') WHERE id > 0 -/// Problem: RestoreQualifiedNamesVisitor sees semantic->table = "my_table" and -/// incorrectly produces: SELECT my_table.id FROM remote(...) WHERE my_table.id > 0 +/// 3. Subsequent visitors like RestoreQualifiedNamesVisitor called in remote() function over the same AST +/// may use this stale semantic->table information to incorrectly qualify column names with the original table name /// /// SOLUTION: /// This visitor clears semantic->table for all column identifiers, ensuring that subsequent /// visitors work with clean semantic information and don't apply stale table qualifications. struct ResetSemanticTableMatcher { - struct Data - { - // No data needed for this visitor - }; + // No data needed for this visitor + struct Data {}; static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } static void visit(ASTPtr & ast, Data & data); diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index b20a647bb036..3ea66264ca24 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -55,9 +55,6 @@ class ASTIdentifier : public ASTWithAlias void resetSemanticTable(); // Reset semantic to empty string (see ResetSemanticTableVisitor) std::shared_ptr createTable() const; // returns |nullptr| if identifier is not table. - // // FIXME: remove after debugging - // const std::shared_ptr& getSemantic() const { return semantic; } - String full_name; std::vector name_parts; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0c5eb2cd6a25..8a4d057932d9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -509,6 +509,10 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( if (to_stage == QueryProcessingStage::WithMergeableState) return QueryProcessingStage::WithMergeableState; + // TODO: check logic + if (!additional_table_functions.empty()) + nodes += additional_table_functions.size(); + /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. if (nodes == 1) @@ -551,6 +555,9 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( bool StorageDistributed::isShardingKeySuitsQueryTreeNodeExpression( const QueryTreeNodePtr & expr, const SelectQueryInfo & query_info) const { + if (!additional_table_functions.empty()) + return false; + ColumnsWithTypeAndName empty_input_columns; ColumnNodePtrWithHashSet empty_correlated_columns_set; // When comparing sharding key expressions, we need to ignore table qualifiers in column names @@ -970,7 +977,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, QueryTreeNodePtr filter; - // static auto logger = getLogger("StorageDistributed"); if (additional_filter) { @@ -978,13 +984,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, filter = buildQueryTree(additional_filter->clone(), query_context); - // LOG_ERROR(logger, "DEBUG: About to call QueryAnalysisPass(replacement_table_expression).run(filter, context)"); - // LOG_ERROR(logger, "DEBUG: filter: {}", filter->dumpTree()); - QueryAnalysisPass(replacement_table_expression).run(filter, context); - - // LOG_ERROR(logger, "DEBUG: filter after pass.run: {}", filter->dumpTree()); - } auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); @@ -992,19 +992,10 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, // Apply additional filter if provided if (filter) { - // LOG_ERROR(logger, "DEBUG: query_tree_to_modify: {}", query_tree_to_modify->dumpTree()); - auto & query = query_tree_to_modify->as(); query.getWhere() = query.hasWhere() ? mergeConditionNodes({query.getWhere(), filter}, query_context) : std::move(filter); - - // LOG_ERROR(logger, "DEBUG: query_tree_to_modify after mergeConditionNodes: {}", query_tree_to_modify->dumpTree()); - - // QueryAnalysisPass pass; - // pass.run(query_tree_to_modify, query_context); - - // LOG_ERROR(logger, "DEBUG: query_tree_to_modify after pass.run: {}", query_tree_to_modify->dumpTree()); } ReplaseAliasColumnsVisitor replase_alias_columns_visitor; @@ -1055,27 +1046,15 @@ void StorageDistributed::read( if (!remote_table_function_ptr) remote_storage_id = StorageID{remote_database, remote_table}; - // LOG_ERROR(log, "DEBUG: About to call buildQueryTreeDistributed for main query"); - // LOG_ERROR(log, "DEBUG: modified_query_info.query_tree: {}", modified_query_info.query_tree ? modified_query_info.query_tree->formatASTForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: remote_storage_id: {}", remote_storage_id.getFullNameNotQuoted()); - // LOG_ERROR(log, "DEBUG: remote_table_function_ptr: {}", remote_table_function_ptr ? remote_table_function_ptr->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: additional_filter: {}", additional_filter ? additional_filter->formatForErrorMessage() : "null"); auto query_tree_distributed = buildQueryTreeDistributed(modified_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr, additional_filter); - // LOG_ERROR(log, "DEBUG: Successfully completed buildQueryTreeDistributed for main query"); - // LOG_ERROR(log, "DEBUG: query_tree_distributed: {}", query_tree_distributed ? query_tree_distributed->formatASTForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQueryAnalyzer::getSampleBlock"); - // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); SelectQueryOptions options = SelectQueryOptions(processed_stage).analyze(); - // LOG_ERROR(log, "DEBUG: SelectQueryOptions created"); header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, options); - // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQueryAnalyzer::getSampleBlock"); /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. @@ -1094,26 +1073,15 @@ void StorageDistributed::read( // Create a modified query info with the additional predicate SelectQueryInfo additional_query_info = query_info; - // LOG_ERROR(log, "DEBUG: About to call buildQueryTreeDistributed for additional table function"); - // LOG_ERROR(log, "DEBUG: additional_query_info.query_tree: {}", additional_query_info.query_tree ? additional_query_info.query_tree->formatASTForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: table_function_entry.table_function_ast: {}", table_function_entry.table_function_ast ? table_function_entry.table_function_ast->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: table_function_entry.predicate_ast: {}", table_function_entry.predicate_ast ? table_function_entry.predicate_ast->formatForErrorMessage() : "null"); auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, StorageID::createEmpty(), table_function_entry.table_function_ast, table_function_entry.predicate_ast); - // LOG_ERROR(log, "DEBUG: Successfully completed buildQueryTreeDistributed for additional table function"); - // LOG_ERROR(log, "DEBUG: additional_query_tree: {}", additional_query_tree ? additional_query_tree->formatASTForErrorMessage() : "null"); - - // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQueryAnalyzer::getSampleBlock for additional table function"); - // TODO: somewhere here the DESCRIBE TABLE is triggered, try to avoid it. auto additional_header = InterpreterSelectQueryAnalyzer::getSampleBlock(additional_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQueryAnalyzer::getSampleBlock for additional table function"); for (auto & column : additional_header) column.column = column.column->convertToFullColumnIfConst(); @@ -1131,47 +1099,26 @@ void StorageDistributed::read( } else { - // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); - // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); - header = InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQuery(modified_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); - // LOG_ERROR(log, "DEBUG: header: {}", header.dumpStructure()); - // LOG_ERROR(log, "DEBUG: About to call ClusterProxy::rewriteSelectQuery"); - modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr, additional_filter); - // LOG_ERROR(log, "DEBUG: Successfully completed ClusterProxy::rewriteSelectQuery"); - // LOG_ERROR(log, "DEBUG: modified_query_info.query: {}", modified_query_info.query ? modified_query_info.query->formatForErrorMessage() : "null"); - if (!additional_table_functions.empty()) { for (const auto & table_function_entry : additional_table_functions) { SelectQueryInfo additional_query_info = query_info; - // LOG_ERROR(log, "DEBUG: About to call InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); - // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); - auto additional_header = InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - // LOG_ERROR(log, "DEBUG: Successfully completed InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock"); - // LOG_ERROR(log, "DEBUG: additional_header: {}", additional_header.dumpStructure()); - // LOG_ERROR(log, "DEBUG: About to call ClusterProxy::rewriteSelectQuery"); additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", table_function_entry.table_function_ast, table_function_entry.predicate_ast); - // LOG_ERROR(log, "DEBUG: Successfully completed ClusterProxy::rewriteSelectQuery"); - // LOG_ERROR(log, "DEBUG: additional_query_info.query: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); - all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); } @@ -1231,13 +1178,6 @@ void StorageDistributed::read( const auto & storage = additional_table_functions[i].storage; auto additional_header = all_headers[i]; - // LOG_ERROR(log, "DEBUG: About to call storage->read for additional storage"); - // LOG_ERROR(log, "DEBUG: additional_header: {}", additional_header.dumpStructure()); - // LOG_ERROR(log, "DEBUG: storage: {}", storage->getName()); - // LOG_ERROR(log, "DEBUG: additional_query_info: {}", additional_query_info.query ? additional_query_info.query->formatForErrorMessage() : "null"); - // LOG_ERROR(log, "DEBUG: processed_stage: {}", static_cast(processed_stage)); - - // Create a new query plan for this additional storage QueryPlan additional_plan; // Execute the query against the additional storage @@ -1251,8 +1191,6 @@ void StorageDistributed::read( 0, // max_block_size 0); // num_streams - // LOG_ERROR(log, "DEBUG: Successfully completed storage->read for additional storage"); - additional_plans.push_back(std::move(additional_plan)); } @@ -1286,58 +1224,6 @@ void StorageDistributed::read( // Create UnionStep to combine all plans auto union_step = std::make_unique(std::move(headers), 0); - - /* TODO!!: simple union step is not enough, we need to add a conversion step to match the headers - -SELECT - id % 2 AS x, - count() -FROM test_tiered_predicate_filtering_analyzer_off -GROUP BY x -ORDER BY x ASC - - -SELECT - hostName(), - id % 2 AS x, - count() -FROM test_tiered_predicate_filtering_analyzer_off -GROUP BY x -ORDER BY x ASC - -Query id: 88fbdc56-33de-468d-87ef-155121cdbea2 - - -Elapsed: 0.012 sec. - -[mfilimonov-MS-7E12] 2025.09.10 22:24:47.280713 [ 399165 ] {70b3d45e-77e5-4acd-b3af-3e1e4b6c1e58} executeQuery: Code: 60. DB::Exception: Unknown table expression identifier 'test_tiered_predicate_filtering_analyzer_off' in scope SELECT hostName(), id % 2 AS x, count() FROM test_tiered_predicate_filtering_analyzer_off GROUP BY x ORDER BY x ASC. (UNKNOWN_TABLE) (version v25.6.2.20000.altinityantalya.28000) (from [::ffff:127.0.0.1]:56130) (query 1, line 1) (in query: SELECT hostName(), id % 2 as x, count() FROM test_tiered_predicate_filtering_analyzer_off group by x ORDER BY x ASC ;), Stack trace (when copying this message, always include the lines below): - -0. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__exception/exception.h:113: Poco::Exception::Exception(String const&, int) @ 0x000000001bc06a92 -1. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.cpp:115: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x00000000107b4b5b -2. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.h:119: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x000000000a2d1e2c -3. /home/mfilimonov/workspace/ClickHouse/master/src/Common/Exception.h:137: DB::Exception::Exception(int, FormatStringHelperImpl::type, std::type_identity::type>, String const&, String&&) @ 0x000000000a946e8b -4. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:4475: DB::QueryAnalyzer::initializeQueryJoinTreeNode(std::shared_ptr&, DB::IdentifierResolveScope&) @ 0x0000000014f28356 -5. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:5755: DB::QueryAnalyzer::resolveQuery(std::shared_ptr const&, DB::IdentifierResolveScope&) @ 0x0000000014ef57bb -6. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalyzer.cpp:181: DB::QueryAnalyzer::resolve(std::shared_ptr&, std::shared_ptr const&, std::shared_ptr) @ 0x0000000014ef493a -7. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/Resolve/QueryAnalysisPass.cpp:18: DB::QueryAnalysisPass::run(std::shared_ptr&, std::shared_ptr) @ 0x0000000014ef414e -8. /home/mfilimonov/workspace/ClickHouse/master/src/Analyzer/QueryTreePassManager.cpp:187: DB::QueryTreePassManager::run(std::shared_ptr) @ 0x0000000014f5246a -9. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp:165: DB::buildQueryTreeAndRunPasses(std::shared_ptr const&, DB::SelectQueryOptions const&, std::shared_ptr const&, std::shared_ptr const&) @ 0x0000000015903a3c -10. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp:182: DB::InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(std::shared_ptr const&, std::shared_ptr const&, DB::SelectQueryOptions const&, std::vector> const&) @ 0x000000001590222d -11. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:634: std::__unique_if::__unique_single std::make_unique[abi:se190107]&, std::shared_ptr const&, DB::SelectQueryOptions const&>(std::shared_ptr&, std::shared_ptr const&, DB::SelectQueryOptions const&) @ 0x0000000015904c04 -12. /home/mfilimonov/workspace/ClickHouse/master/contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x000000001589c9cb -13. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/executeQuery.cpp:1455: DB::executeQueryImpl(char const*, char const*, std::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum, DB::ReadBuffer*, std::shared_ptr&) @ 0x0000000015bd48c0 -14. /home/mfilimonov/workspace/ClickHouse/master/src/Interpreters/executeQuery.cpp:1715: DB::executeQuery(String const&, std::shared_ptr, DB::QueryFlags, DB::QueryProcessingStage::Enum) @ 0x0000000015bcf388 -15. /home/mfilimonov/workspace/ClickHouse/master/src/Server/TCPHandler.cpp:721: DB::TCPHandler::runImpl() @ 0x0000000017920812 -16. /home/mfilimonov/workspace/ClickHouse/master/src/Server/TCPHandler.cpp:2727: DB::TCPHandler::run() @ 0x0000000017939bb9 -17. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Net/src/TCPServerConnection.cpp:40: Poco::Net::TCPServerConnection::start() @ 0x000000001bca8687 -18. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Net/src/TCPServerDispatcher.cpp:115: Poco::Net::TCPServerDispatcher::run() @ 0x000000001bca8b5e -19. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Foundation/src/ThreadPool.cpp:205: Poco::PooledThread::run() @ 0x000000001bc52e32 -20. /home/mfilimonov/workspace/ClickHouse/master/base/poco/Foundation/src/Thread_POSIX.cpp:335: Poco::ThreadImpl::runnableEntry(void*) @ 0x000000001bc50a6f -21. start_thread @ 0x00000000000a27f1 -22. __GI___clone3 @ 0x0000000000133c9c - - -*/ union_plan.unitePlans(std::move(union_step), std::move(plan_ptrs)); // Replace the original query plan with the union plan From 25afccb92c224659c93dd85cdd0f10940f55beb0 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 18:20:55 +0200 Subject: [PATCH 07/20] some drafts --- src/Storages/StorageDistributed.cpp | 150 ++++++++++++++++++++-------- 1 file changed, 106 insertions(+), 44 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8a4d057932d9..7e6ceae543ca 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -910,6 +910,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const ASTPtr & remote_table_function, const ASTPtr & additional_filter = nullptr) { + auto dbg_log = getLogger("TieredDistributedMerge/buildQueryTreeDistributed"); auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -956,6 +957,18 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, query_analysis_pass.run(node, query_context); } + // Try to log replacement table expression columns for table function as well + try + { + if (table_function_node->isResolved()) + { + auto snapshot = table_function_node->getStorageSnapshot(); + auto cols = snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals()); + LOG_TRACE(dbg_log, "Replacement table expression (table function) columns: {}", ColumnsDescription{cols}.toString()); + } + } + catch (...) {} + replacement_table_expression = std::move(table_function_node); } else @@ -963,6 +976,11 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); auto column_names_and_types = distributed_storage_snapshot->getColumns(get_column_options); + try + { + LOG_TRACE(dbg_log, "Replacement table expression (remote table) columns: {}", ColumnsDescription{column_names_and_types}.toString()); + } + catch (...) {} auto storage = std::make_shared(remote_storage_id, ColumnsDescription{column_names_and_types}); auto table_node = std::make_shared(std::move(storage), query_context); @@ -980,11 +998,26 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, if (additional_filter) { - const auto & context = query_info.planner_context->getQueryContext(); - + // Build filter tree in the same query context that is used for the replacement table expression filter = buildQueryTree(additional_filter->clone(), query_context); - QueryAnalysisPass(replacement_table_expression).run(filter, context); + // Ensure the replacement table expression is resolved (has storage/snapshot) before analyzing the filter + if (auto * tf = replacement_table_expression->as(); tf && !tf->isResolved()) + { + QueryAnalysisPass analyze_table_expression; + QueryTreeNodePtr tmp = replacement_table_expression; + analyze_table_expression.run(tmp, query_context); + replacement_table_expression = std::move(tmp); + } + + // Analyze the filter against the resolved replacement table expression so identifiers/types + // are bound to the correct subtable header/snapshot, including virtual/hive columns. + QueryAnalysisPass(replacement_table_expression).run(filter, query_context); + try + { + LOG_TRACE(dbg_log, "Additional filter after analysis: {}", filter->formatASTForErrorMessage()); + } + catch (...) {} } auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); @@ -998,6 +1031,21 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } + try + { + LOG_TRACE(dbg_log, "Query tree after replacement + reanalysis: {}", query_tree_to_modify->formatASTForErrorMessage()); + } + catch (...) {} + + // Also log a sample header after reanalysis (analyze-only) to see resolved types + try + { + SelectQueryOptions opts = SelectQueryOptions().analyze(); + Block sample = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_to_modify, query_context, opts); + LOG_TRACE(dbg_log, "Sample header after reanalysis: {}", sample.dumpStructure()); + } + catch (...) {} + ReplaseAliasColumnsVisitor replase_alias_columns_visitor; replase_alias_columns_visitor.visit(query_tree_to_modify); @@ -1065,6 +1113,9 @@ void StorageDistributed::read( modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); + // Clear any pre-typed filter DAG from the original query_info to avoid mismatches + // after table substitution. Planner will rebuild filters from the re-analyzed tree. + modified_query_info.filter_actions_dag.reset(); if (!additional_table_functions.empty()) { @@ -1079,14 +1130,10 @@ void StorageDistributed::read( table_function_entry.table_function_ast, table_function_entry.predicate_ast); - // TODO: somewhere here the DESCRIBE TABLE is triggered, try to avoid it. - auto additional_header = InterpreterSelectQueryAnalyzer::getSampleBlock(additional_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - - for (auto & column : additional_header) - column.column = column.column->convertToFullColumnIfConst(); - additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); + additional_query_info.filter_actions_dag.reset(); + LOG_DEBUG(log, "TieredDistributedMerge: additional storage #{} query: {}", i, additional_query_info.query->formatForLogging()); all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); @@ -1112,14 +1159,13 @@ void StorageDistributed::read( { SelectQueryInfo additional_query_info = query_info; - auto additional_header = InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", table_function_entry.table_function_ast, table_function_entry.predicate_ast); - all_headers.push_back(additional_header); + LOG_DEBUG(log, "TieredDistributedMerge: additional storage #{} query: {}", i, additional_query_info.query->formatForLogging()); + all_query_infos.push_back(additional_query_info); } } @@ -1136,6 +1182,8 @@ void StorageDistributed::read( } } + // No-op here: filter DAG will be produced by the planner from the re-analyzed query tree. + const auto & snapshot_data = assert_cast(*storage_snapshot->data); if (!modified_query_info.getCluster()->getShardsInfo().empty()) @@ -1174,22 +1222,36 @@ void StorageDistributed::read( std::vector additional_plans; for (size_t i = 0; i < all_query_infos.size(); ++i) { + // Use the rewritten additional query info built above for this storage auto additional_query_info = all_query_infos[i]; - const auto & storage = additional_table_functions[i].storage; - auto additional_header = all_headers[i]; // Create a new query plan for this additional storage QueryPlan additional_plan; - // Execute the query against the additional storage - storage->read( - additional_plan, - {}, // column names - storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), local_context), - additional_query_info, - local_context, - processed_stage, - 0, // max_block_size - 0); // num_streams + + // Build a full SELECT plan for the additional storage so that + // projection/aggregation (e.g. count()) is represented in the plan output header. + const auto & settings_local = local_context->getSettingsRef(); + if (settings_local[Setting::allow_experimental_analyzer]) + { + // Use the analyzer pipeline + // Build plan to the same processing stage as the main path + SelectQueryOptions add_opts(processed_stage); + InterpreterSelectQueryAnalyzer add_interpreter( + additional_query_info.query_tree, + local_context, + add_opts); + additional_plan = std::move(add_interpreter).extractQueryPlan(); + } + else + { + // Classic pipeline + SelectQueryOptions add_opts(processed_stage); + InterpreterSelectQuery add_interpreter( + additional_query_info.query, + local_context, + add_opts); + add_interpreter.buildQueryPlan(additional_plan); + } additional_plans.push_back(std::move(additional_plan)); } @@ -2395,26 +2457,26 @@ void registerStorageDistributed(StorageFactory & factory) "Argument #{}: additional table function must be a valid table function, got: {}", i, func->name); } - // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster - // Check if it's a supported table function type - String table_function_name = func->name; - - // Check for ITableFunctionCluster types (ending with "Cluster") - // Use a whitelist of supported table function names for clarity and safety - static const std::unordered_set supported_table_functions = { - "remote", "remoteSecure", "cluster", "clusterAllReplicas", - "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", - "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" - }; - - if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument #{}: additional table function '{}' is not supported. " - "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " - "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " - "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); - } + // // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster + // // Check if it's a supported table function type + // String table_function_name = func->name; + + // // Check for ITableFunctionCluster types (ending with "Cluster") + // // Use a whitelist of supported table function names for clarity and safety + // static const std::unordered_set supported_table_functions = { + // "remote", "remoteSecure", "cluster", "clusterAllReplicas", + // "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", + // "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" + // }; + + // if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) + // { + // throw Exception(ErrorCodes::BAD_ARGUMENTS, + // "Argument #{}: additional table function '{}' is not supported. " + // "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " + // "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " + // "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); + // } // TODO: Validate predicate - must be a SQL expression (just rejecting a string literal for now) if (const auto * literal = predicate_ast->as()) From a84f6936f2d7f7c970138cc9bfc915e14b745b49 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 18:21:06 +0200 Subject: [PATCH 08/20] Revert "some drafts" This reverts commit 25afccb92c224659c93dd85cdd0f10940f55beb0. --- src/Storages/StorageDistributed.cpp | 150 ++++++++-------------------- 1 file changed, 44 insertions(+), 106 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 7e6ceae543ca..8a4d057932d9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -910,7 +910,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const ASTPtr & remote_table_function, const ASTPtr & additional_filter = nullptr) { - auto dbg_log = getLogger("TieredDistributedMerge/buildQueryTreeDistributed"); auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -957,18 +956,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, query_analysis_pass.run(node, query_context); } - // Try to log replacement table expression columns for table function as well - try - { - if (table_function_node->isResolved()) - { - auto snapshot = table_function_node->getStorageSnapshot(); - auto cols = snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals()); - LOG_TRACE(dbg_log, "Replacement table expression (table function) columns: {}", ColumnsDescription{cols}.toString()); - } - } - catch (...) {} - replacement_table_expression = std::move(table_function_node); } else @@ -976,11 +963,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); auto column_names_and_types = distributed_storage_snapshot->getColumns(get_column_options); - try - { - LOG_TRACE(dbg_log, "Replacement table expression (remote table) columns: {}", ColumnsDescription{column_names_and_types}.toString()); - } - catch (...) {} auto storage = std::make_shared(remote_storage_id, ColumnsDescription{column_names_and_types}); auto table_node = std::make_shared(std::move(storage), query_context); @@ -998,26 +980,11 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, if (additional_filter) { - // Build filter tree in the same query context that is used for the replacement table expression - filter = buildQueryTree(additional_filter->clone(), query_context); + const auto & context = query_info.planner_context->getQueryContext(); - // Ensure the replacement table expression is resolved (has storage/snapshot) before analyzing the filter - if (auto * tf = replacement_table_expression->as(); tf && !tf->isResolved()) - { - QueryAnalysisPass analyze_table_expression; - QueryTreeNodePtr tmp = replacement_table_expression; - analyze_table_expression.run(tmp, query_context); - replacement_table_expression = std::move(tmp); - } + filter = buildQueryTree(additional_filter->clone(), query_context); - // Analyze the filter against the resolved replacement table expression so identifiers/types - // are bound to the correct subtable header/snapshot, including virtual/hive columns. - QueryAnalysisPass(replacement_table_expression).run(filter, query_context); - try - { - LOG_TRACE(dbg_log, "Additional filter after analysis: {}", filter->formatASTForErrorMessage()); - } - catch (...) {} + QueryAnalysisPass(replacement_table_expression).run(filter, context); } auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); @@ -1031,21 +998,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } - try - { - LOG_TRACE(dbg_log, "Query tree after replacement + reanalysis: {}", query_tree_to_modify->formatASTForErrorMessage()); - } - catch (...) {} - - // Also log a sample header after reanalysis (analyze-only) to see resolved types - try - { - SelectQueryOptions opts = SelectQueryOptions().analyze(); - Block sample = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_to_modify, query_context, opts); - LOG_TRACE(dbg_log, "Sample header after reanalysis: {}", sample.dumpStructure()); - } - catch (...) {} - ReplaseAliasColumnsVisitor replase_alias_columns_visitor; replase_alias_columns_visitor.visit(query_tree_to_modify); @@ -1113,9 +1065,6 @@ void StorageDistributed::read( modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); - // Clear any pre-typed filter DAG from the original query_info to avoid mismatches - // after table substitution. Planner will rebuild filters from the re-analyzed tree. - modified_query_info.filter_actions_dag.reset(); if (!additional_table_functions.empty()) { @@ -1130,10 +1079,14 @@ void StorageDistributed::read( table_function_entry.table_function_ast, table_function_entry.predicate_ast); + // TODO: somewhere here the DESCRIBE TABLE is triggered, try to avoid it. + auto additional_header = InterpreterSelectQueryAnalyzer::getSampleBlock(additional_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + + for (auto & column : additional_header) + column.column = column.column->convertToFullColumnIfConst(); + additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); - additional_query_info.filter_actions_dag.reset(); - LOG_DEBUG(log, "TieredDistributedMerge: additional storage #{} query: {}", i, additional_query_info.query->formatForLogging()); all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); @@ -1159,13 +1112,14 @@ void StorageDistributed::read( { SelectQueryInfo additional_query_info = query_info; + auto additional_header = InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", table_function_entry.table_function_ast, table_function_entry.predicate_ast); - LOG_DEBUG(log, "TieredDistributedMerge: additional storage #{} query: {}", i, additional_query_info.query->formatForLogging()); - + all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); } } @@ -1182,8 +1136,6 @@ void StorageDistributed::read( } } - // No-op here: filter DAG will be produced by the planner from the re-analyzed query tree. - const auto & snapshot_data = assert_cast(*storage_snapshot->data); if (!modified_query_info.getCluster()->getShardsInfo().empty()) @@ -1222,36 +1174,22 @@ void StorageDistributed::read( std::vector additional_plans; for (size_t i = 0; i < all_query_infos.size(); ++i) { - // Use the rewritten additional query info built above for this storage auto additional_query_info = all_query_infos[i]; + const auto & storage = additional_table_functions[i].storage; + auto additional_header = all_headers[i]; // Create a new query plan for this additional storage QueryPlan additional_plan; - - // Build a full SELECT plan for the additional storage so that - // projection/aggregation (e.g. count()) is represented in the plan output header. - const auto & settings_local = local_context->getSettingsRef(); - if (settings_local[Setting::allow_experimental_analyzer]) - { - // Use the analyzer pipeline - // Build plan to the same processing stage as the main path - SelectQueryOptions add_opts(processed_stage); - InterpreterSelectQueryAnalyzer add_interpreter( - additional_query_info.query_tree, - local_context, - add_opts); - additional_plan = std::move(add_interpreter).extractQueryPlan(); - } - else - { - // Classic pipeline - SelectQueryOptions add_opts(processed_stage); - InterpreterSelectQuery add_interpreter( - additional_query_info.query, - local_context, - add_opts); - add_interpreter.buildQueryPlan(additional_plan); - } + // Execute the query against the additional storage + storage->read( + additional_plan, + {}, // column names + storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), local_context), + additional_query_info, + local_context, + processed_stage, + 0, // max_block_size + 0); // num_streams additional_plans.push_back(std::move(additional_plan)); } @@ -2457,26 +2395,26 @@ void registerStorageDistributed(StorageFactory & factory) "Argument #{}: additional table function must be a valid table function, got: {}", i, func->name); } - // // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster - // // Check if it's a supported table function type - // String table_function_name = func->name; - - // // Check for ITableFunctionCluster types (ending with "Cluster") - // // Use a whitelist of supported table function names for clarity and safety - // static const std::unordered_set supported_table_functions = { - // "remote", "remoteSecure", "cluster", "clusterAllReplicas", - // "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", - // "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" - // }; - - // if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) - // { - // throw Exception(ErrorCodes::BAD_ARGUMENTS, - // "Argument #{}: additional table function '{}' is not supported. " - // "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " - // "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " - // "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); - // } + // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster + // Check if it's a supported table function type + String table_function_name = func->name; + + // Check for ITableFunctionCluster types (ending with "Cluster") + // Use a whitelist of supported table function names for clarity and safety + static const std::unordered_set supported_table_functions = { + "remote", "remoteSecure", "cluster", "clusterAllReplicas", + "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", + "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" + }; + + if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: additional table function '{}' is not supported. " + "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " + "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " + "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); + } // TODO: Validate predicate - must be a SQL expression (just rejecting a string literal for now) if (const auto * literal = predicate_ast->as()) From 9370cdeb81e71bf8bc7e1822437ceec2f58dc7b4 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 19:21:53 +0200 Subject: [PATCH 09/20] relax checks, support not only table functions as additional storage --- src/Storages/StorageDistributed.cpp | 115 ++++++++++++++-------------- src/Storages/StorageDistributed.h | 15 ++-- 2 files changed, 69 insertions(+), 61 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8a4d057932d9..4b337a39a745 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1175,7 +1175,34 @@ void StorageDistributed::read( for (size_t i = 0; i < all_query_infos.size(); ++i) { auto additional_query_info = all_query_infos[i]; - const auto & storage = additional_table_functions[i].storage; + // Get storage from either table function execution or StorageID resolution + StoragePtr storage; + if (additional_table_functions[i].storage_id.has_value()) + { + // For table identifiers, resolve the StorageID + storage = DatabaseCatalog::instance().getTable(additional_table_functions[i].storage_id.value(), local_context); + } + else + { + // For table functions, execute the AST + auto table_function = TableFunctionFactory::instance().get(additional_table_functions[i].table_function_ast, local_context); + if (!table_function) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); + + storage = table_function->execute( + additional_table_functions[i].table_function_ast, + local_context, + getStorageID().table_name, // Use the current table name + {}, // columns - will be determined from storage + false, // use_global_context = false + false); // is_insert_query = false + + // Handle StorageTableFunctionProxy if present + if (auto proxy = std::dynamic_pointer_cast(storage)) + { + storage = proxy->getNested(); + } + } auto additional_header = all_headers[i]; // Create a new query plan for this additional storage @@ -2382,40 +2409,6 @@ void registerStorageDistributed(StorageFactory & factory) ASTPtr table_function_ast = engine_args[i]; ASTPtr predicate_ast = engine_args[i + 1]; - // Validate table function - const auto * func = table_function_ast->as(); - if (!func) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Additional table function must be a table function, got: {}", table_function_ast->getID()); - } - else if (!TableFunctionFactory::instance().isTableFunctionName(func->name)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument #{}: additional table function must be a valid table function, got: {}", i, func->name); - } - - // Additional table functions can be either TableFunctionRemote or ITableFunctionCluster - // Check if it's a supported table function type - String table_function_name = func->name; - - // Check for ITableFunctionCluster types (ending with "Cluster") - // Use a whitelist of supported table function names for clarity and safety - static const std::unordered_set supported_table_functions = { - "remote", "remoteSecure", "cluster", "clusterAllReplicas", - "s3Cluster", "urlCluster", "fileCluster", "S3Cluster", "AzureCluster", "HDFSCluster", - "IcebergS3Cluster", "IcebergAzureCluster", "IcebergHDFSCluster", "DeltaLakeCluster", "HudiCluster" - }; - - if (supported_table_functions.find(table_function_name) == supported_table_functions.end()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument #{}: additional table function '{}' is not supported. " - "TieredDistributedMerge engine requires additional table functions to be either ITableFunctionCluster-based " - "(like s3Cluster, urlCluster, DeltaLakeCluster, etc.) or TableFunctionRemote-based " - "(like remote, remoteSecure, cluster, clusterAllReplicas).", i, table_function_name); - } - // TODO: Validate predicate - must be a SQL expression (just rejecting a string literal for now) if (const auto * literal = predicate_ast->as()) { @@ -2425,31 +2418,41 @@ void registerStorageDistributed(StorageFactory & factory) "Additional predicate must be a SQL expression, got string literal"); } } + + // Validate table function or table identifier + if (const auto * func = table_function_ast->as()) + { + // It's a table function - validate it + if (!TableFunctionFactory::instance().isTableFunctionName(func->name)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: additional table function must be a valid table function, got: {}", i, func->name); + } - // Create table function instance and execute it to get StoragePtr - auto table_function = TableFunctionFactory::instance().get(table_function_ast, context); - if (!table_function) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid additional table function in TieredDistributedMerge engine"); - - // Execute the table function to get the underlying storage - StoragePtr additional_storage = table_function->execute( - table_function_ast, - context, - args.table_id.table_name, - args.columns, // Use the same columns as the main table function - false, // use_global_context = false - false); // is_insert_query = false - - // Handle StorageTableFunctionProxy if present - if (auto proxy = std::dynamic_pointer_cast(additional_storage)) + // It's a table function - store the AST for later execution + additional_table_functions.emplace_back(table_function_ast, predicate_ast); + } + else if (const auto * identifier = table_function_ast->as()) { - additional_storage = proxy->getNested(); + // It's a table identifier - validate it can be parsed as StorageID + try + { + // Parse table identifier to get StorageID + StorageID storage_id(table_function_ast); + additional_table_functions.emplace_back(table_function_ast, predicate_ast, storage_id); + } + catch (const Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: invalid table identifier '{}': {}", i, identifier->name(), e.message()); + } + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: additional argument must be either a table function or table identifier, got: {}", i, table_function_ast->getID()); } - // additional_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); - - - additional_table_functions.emplace_back(std::move(additional_storage), table_function_ast, predicate_ast); } // Now handle the first table function (which must be a TableFunctionRemote) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 1a756de5b708..991b220bde8a 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -50,18 +50,23 @@ class StorageDistributed final : public IStorage, WithContext friend class StorageSystemDistributionQueue; public: - /// Structure to hold storage, its AST, and associated predicate + /// Structure to hold table function AST, predicate, and optional StorageID for table identifiers struct TableFunctionEntry { - StoragePtr storage; ASTPtr table_function_ast; ASTPtr predicate_ast; + std::optional storage_id; // For table identifiers instead of table functions - TableFunctionEntry(StoragePtr storage_, ASTPtr table_function_ast_, ASTPtr predicate_ast_) - : storage(std::move(storage_)) - , table_function_ast(std::move(table_function_ast_)) + TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_) + : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) {} + + TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_) + : table_function_ast(std::move(table_function_ast_)) + , predicate_ast(std::move(predicate_ast_)) + , storage_id(std::move(storage_id_)) + {} }; StorageDistributed( From 96047e7711f778455234780de9f846ec568e7e90 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 19:42:26 +0200 Subject: [PATCH 10/20] better storage registration --- src/Storages/StorageDistributed.cpp | 3 +++ src/Storages/registerStorages.cpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4b337a39a745..442cd97e6613 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2351,7 +2351,10 @@ void registerStorageDistributed(StorageFactory & factory) .source_access_type = AccessType::REMOTE, .has_builtin_setting_fn = DistributedSettings::hasBuiltin, }); +} +void registerStorageTieredDistributedMerge(StorageFactory & factory) +{ // Register TieredDistributedMerge engine // TODO: consider moving it to a separate file / subclass of StorageDistributed factory.registerStorage("TieredDistributedMerge", [](const StorageFactory::Arguments & args) -> StoragePtr diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 70f6d44b0422..6c3e5f3d8284 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -13,6 +13,7 @@ void registerStorageNull(StorageFactory & factory); void registerStorageMerge(StorageFactory & factory); void registerStorageBuffer(StorageFactory & factory); void registerStorageDistributed(StorageFactory & factory); +void registerStorageTieredDistributedMerge(StorageFactory & factory); void registerStorageMemory(StorageFactory & factory); void registerStorageFile(StorageFactory & factory); void registerStorageURL(StorageFactory & factory); @@ -115,6 +116,7 @@ void registerStorages() registerStorageMerge(factory); registerStorageBuffer(factory); registerStorageDistributed(factory); + registerStorageTieredDistributedMerge(factory); registerStorageMemory(factory); registerStorageFile(factory); registerStorageURL(factory); From 73c5f0e9666931083446e6ce642cb7406f5c180d Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 19:54:12 +0200 Subject: [PATCH 11/20] better arg validation --- src/Storages/StorageDistributed.cpp | 39 ++++++++++++++++------------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 442cd97e6613..de205fff3233 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2386,20 +2386,21 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) "First argument must be a table function, got: {}", first_arg->getID()); } - // TODO: Validate second argument - must be a SQL expression (just rejecting a string literal for now) + // Create the underlying StorageDistributed using the table function + const ContextPtr & context = args.getContext(); + + // Validate second argument - must be a SQL expression ASTPtr second_arg = engine_args[1]; - if (const auto * literal = second_arg->as()) + try { - // Check if it's a string literal (which would be invalid for a SQL expression) - if (literal->value.getType() == Field::Types::String) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Second argument must be a SQL expression, got string literal"); - } + auto syntax_result = TreeRewriter(context).analyze(second_arg, args.columns.getAllPhysical()); + ExpressionAnalyzer(second_arg, syntax_result, context).getActions(true); + } + catch (const Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Second argument must be a valid SQL expression: {}", e.message()); } - - // Create the underlying StorageDistributed using the table function - const ContextPtr & context = args.getContext(); // Parse additional table function pairs (if any) std::vector additional_table_functions; @@ -2412,14 +2413,16 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) ASTPtr table_function_ast = engine_args[i]; ASTPtr predicate_ast = engine_args[i + 1]; - // TODO: Validate predicate - must be a SQL expression (just rejecting a string literal for now) - if (const auto * literal = predicate_ast->as()) + // Validate predicate - must be a SQL expression + try { - if (literal->value.getType() == Field::Types::String) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Additional predicate must be a SQL expression, got string literal"); - } + auto syntax_result = TreeRewriter(context).analyze(predicate_ast, args.columns.getAllPhysical()); + ExpressionAnalyzer(predicate_ast, syntax_result, context).getActions(true); + } + catch (const Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{} must be a valid SQL expression: {}", i + 1, e.message()); } // Validate table function or table identifier From 82a4fbd48b01980ebb7ddf9a660540733f995b4f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 20:12:35 +0200 Subject: [PATCH 12/20] draft of getInMemoryMetadata / getStorageSnapshot for TieredDistributedMerge --- src/Storages/StorageDistributed.cpp | 149 ++++++++++++++++++++++++++-- src/Storages/StorageDistributed.h | 9 ++ 2 files changed, 150 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index de205fff3233..3e23771c1785 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include @@ -28,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -765,11 +767,6 @@ static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr return false; } -StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const -{ - /// TODO: support additional table functions - return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); -} /// TODO: support additional table functions StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( @@ -1188,7 +1185,7 @@ void StorageDistributed::read( auto table_function = TableFunctionFactory::instance().get(additional_table_functions[i].table_function_ast, local_context); if (!table_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); - + storage = table_function->execute( additional_table_functions[i].table_function_ast, local_context, @@ -1196,7 +1193,7 @@ void StorageDistributed::read( {}, // columns - will be determined from storage false, // use_global_context = false false); // is_insert_query = false - + // Handle StorageTableFunctionProxy if present if (auto proxy = std::dynamic_pointer_cast(storage)) { @@ -2424,7 +2421,7 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument #{} must be a valid SQL expression: {}", i + 1, e.message()); } - + // Validate table function or table identifier if (const auto * func = table_function_ast->as()) { @@ -2545,4 +2542,140 @@ bool StorageDistributed::initializeDiskOnConfigChange(const std::set & n return true; } + +StorageInMemoryMetadata StorageDistributed::getInMemoryMetadata() const +{ + if (additional_table_functions.empty()) + { + // For regular Distributed engine, use base implementation + return IStorage::getInMemoryMetadata(); + } + + // For TieredDistributedMerge engine, merge schemas from all layers + auto metadata = IStorage::getInMemoryMetadata(); + + // Get merged columns from all layers + auto merged_columns = getColumnsDescriptionFromLayers(getContext()); + if (!merged_columns.empty()) + { + metadata.setColumns(merged_columns); + } + + // Note: Virtual columns for TieredDistributedMerge should be set in constructor + // or in a non-const method + + return metadata; +} + +StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const +{ + if (additional_table_functions.empty()) + { + // For regular Distributed engine, use existing implementation + return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); + } + + // For TieredDistributedMerge engine, create snapshot with virtual columns + auto snapshot_data = std::make_unique(); + + if (!requiresObjectColumns(metadata_snapshot->getColumns(), nullptr)) + return std::make_shared(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data)); + + // For Object columns, we need to collect objects from all layers + snapshot_data->objects_by_shard = getExtendedObjectsOfRemoteTables( + *getCluster(), + StorageID{remote_database, remote_table}, + metadata_snapshot->getColumns(), + getContext()); + + auto object_columns = DB::getConcreteObjectColumns( + snapshot_data->objects_by_shard.begin(), + snapshot_data->objects_by_shard.end(), + metadata_snapshot->getColumns(), + [](const auto & shard_num_and_columns) -> const auto & { return shard_num_and_columns.second; }); + + return std::make_shared(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data)); +} + +ColumnsDescription StorageDistributed::getColumnsDescriptionFromLayers(const ContextPtr & query_context) const +{ + ColumnsDescription result; + + // Start with the main distributed table columns + auto main_metadata = IStorage::getInMemoryMetadata(); + result = main_metadata.getColumns(); + + // Add columns from additional table functions + for (const auto & layer : additional_table_functions) + { + try + { + auto layer_snapshot = getStorageSnapshotForLayer(layer, query_context); + if (layer_snapshot) + { + auto layer_columns = layer_snapshot->getAllColumnsDescription(); + + // Merge columns using supertype logic (similar to StorageMerge) + for (const auto & column : layer_columns) + { + if (!result.has(column.name)) + { + result.add(column); + } + else if (column != result.get(column.name)) + { + result.modify(column.name, [&column](ColumnDescription & existing) + { + existing.type = getLeastSupertypeOrVariant(DataTypes{existing.type, column.type}); + if (existing.default_desc != column.default_desc) + existing.default_desc = {}; + }); + } + } + } + } + catch (const Exception & e) + { + LOG_WARNING(log, "Failed to get schema from layer {}: {}", layer.table_function_ast->formatForErrorMessage(), e.message()); + } + } + + return result; +} + +StorageSnapshotPtr StorageDistributed::getStorageSnapshotForLayer(const TableFunctionEntry & layer, const ContextPtr & query_context) const +{ + try + { + if (layer.storage_id.has_value()) + { + // It's a table identifier - get storage directly + auto storage = DatabaseCatalog::instance().getTable(layer.storage_id.value(), query_context); + return storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), query_context); + } + else + { + // It's a table function - execute it to get storage + auto table_function = TableFunctionFactory::instance().get(layer.table_function_ast, query_context); + auto storage = table_function->execute(layer.table_function_ast, query_context, ""); + return storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), query_context); + } + } + catch (const Exception & e) + { + LOG_WARNING(log, "Failed to get storage snapshot for layer: {}", e.message()); + return nullptr; + } +} + +VirtualColumnsDescription StorageDistributed::createVirtualsForTieredDistributedMerge() const +{ + auto desc = createVirtuals(); // Get base virtuals from regular Distributed + + // Add _table_index virtual column for TieredDistributedMerge + desc.addEphemeral("_table_index", std::make_shared(), "Index of the table layer (0 for main layer, 1+ for additional layers)"); + + return desc; +} + } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 991b220bde8a..5aba428fa4ac 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -120,6 +120,9 @@ class StorageDistributed final : public IStorage, WithContext StorageSnapshotPtr getStorageSnapshotForQuery( const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr query_context) const override; + /// Override for TieredDistributedMerge to merge schemas from all layers + StorageInMemoryMetadata getInMemoryMetadata() const override; + QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; @@ -325,6 +328,12 @@ class StorageDistributed final : public IStorage, WithContext /// Additional table functions for TieredDistributedMerge engine std::vector additional_table_functions; + +private: + /// Helper methods for TieredDistributedMerge + ColumnsDescription getColumnsDescriptionFromLayers(const ContextPtr & query_context) const; + StorageSnapshotPtr getStorageSnapshotForLayer(const TableFunctionEntry & layer, const ContextPtr & query_context) const; + VirtualColumnsDescription createVirtualsForTieredDistributedMerge() const; }; } From 382b88598a18ce53d2f31e3bae375e62cc3cccbe Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 20:12:42 +0200 Subject: [PATCH 13/20] Revert "draft of getInMemoryMetadata / getStorageSnapshot for TieredDistributedMerge" This reverts commit 82a4fbd48b01980ebb7ddf9a660540733f995b4f. --- src/Storages/StorageDistributed.cpp | 149 ++-------------------------- src/Storages/StorageDistributed.h | 9 -- 2 files changed, 8 insertions(+), 150 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 3e23771c1785..de205fff3233 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include @@ -29,7 +28,6 @@ #include #include #include -#include #include #include #include @@ -767,6 +765,11 @@ static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr return false; } +StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const +{ + /// TODO: support additional table functions + return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); +} /// TODO: support additional table functions StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( @@ -1185,7 +1188,7 @@ void StorageDistributed::read( auto table_function = TableFunctionFactory::instance().get(additional_table_functions[i].table_function_ast, local_context); if (!table_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); - + storage = table_function->execute( additional_table_functions[i].table_function_ast, local_context, @@ -1193,7 +1196,7 @@ void StorageDistributed::read( {}, // columns - will be determined from storage false, // use_global_context = false false); // is_insert_query = false - + // Handle StorageTableFunctionProxy if present if (auto proxy = std::dynamic_pointer_cast(storage)) { @@ -2421,7 +2424,7 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument #{} must be a valid SQL expression: {}", i + 1, e.message()); } - + // Validate table function or table identifier if (const auto * func = table_function_ast->as()) { @@ -2542,140 +2545,4 @@ bool StorageDistributed::initializeDiskOnConfigChange(const std::set & n return true; } - -StorageInMemoryMetadata StorageDistributed::getInMemoryMetadata() const -{ - if (additional_table_functions.empty()) - { - // For regular Distributed engine, use base implementation - return IStorage::getInMemoryMetadata(); - } - - // For TieredDistributedMerge engine, merge schemas from all layers - auto metadata = IStorage::getInMemoryMetadata(); - - // Get merged columns from all layers - auto merged_columns = getColumnsDescriptionFromLayers(getContext()); - if (!merged_columns.empty()) - { - metadata.setColumns(merged_columns); - } - - // Note: Virtual columns for TieredDistributedMerge should be set in constructor - // or in a non-const method - - return metadata; -} - -StorageSnapshotPtr StorageDistributed::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const -{ - if (additional_table_functions.empty()) - { - // For regular Distributed engine, use existing implementation - return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); - } - - // For TieredDistributedMerge engine, create snapshot with virtual columns - auto snapshot_data = std::make_unique(); - - if (!requiresObjectColumns(metadata_snapshot->getColumns(), nullptr)) - return std::make_shared(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data)); - - // For Object columns, we need to collect objects from all layers - snapshot_data->objects_by_shard = getExtendedObjectsOfRemoteTables( - *getCluster(), - StorageID{remote_database, remote_table}, - metadata_snapshot->getColumns(), - getContext()); - - auto object_columns = DB::getConcreteObjectColumns( - snapshot_data->objects_by_shard.begin(), - snapshot_data->objects_by_shard.end(), - metadata_snapshot->getColumns(), - [](const auto & shard_num_and_columns) -> const auto & { return shard_num_and_columns.second; }); - - return std::make_shared(*this, metadata_snapshot, std::move(object_columns), std::move(snapshot_data)); -} - -ColumnsDescription StorageDistributed::getColumnsDescriptionFromLayers(const ContextPtr & query_context) const -{ - ColumnsDescription result; - - // Start with the main distributed table columns - auto main_metadata = IStorage::getInMemoryMetadata(); - result = main_metadata.getColumns(); - - // Add columns from additional table functions - for (const auto & layer : additional_table_functions) - { - try - { - auto layer_snapshot = getStorageSnapshotForLayer(layer, query_context); - if (layer_snapshot) - { - auto layer_columns = layer_snapshot->getAllColumnsDescription(); - - // Merge columns using supertype logic (similar to StorageMerge) - for (const auto & column : layer_columns) - { - if (!result.has(column.name)) - { - result.add(column); - } - else if (column != result.get(column.name)) - { - result.modify(column.name, [&column](ColumnDescription & existing) - { - existing.type = getLeastSupertypeOrVariant(DataTypes{existing.type, column.type}); - if (existing.default_desc != column.default_desc) - existing.default_desc = {}; - }); - } - } - } - } - catch (const Exception & e) - { - LOG_WARNING(log, "Failed to get schema from layer {}: {}", layer.table_function_ast->formatForErrorMessage(), e.message()); - } - } - - return result; -} - -StorageSnapshotPtr StorageDistributed::getStorageSnapshotForLayer(const TableFunctionEntry & layer, const ContextPtr & query_context) const -{ - try - { - if (layer.storage_id.has_value()) - { - // It's a table identifier - get storage directly - auto storage = DatabaseCatalog::instance().getTable(layer.storage_id.value(), query_context); - return storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), query_context); - } - else - { - // It's a table function - execute it to get storage - auto table_function = TableFunctionFactory::instance().get(layer.table_function_ast, query_context); - auto storage = table_function->execute(layer.table_function_ast, query_context, ""); - return storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), query_context); - } - } - catch (const Exception & e) - { - LOG_WARNING(log, "Failed to get storage snapshot for layer: {}", e.message()); - return nullptr; - } -} - -VirtualColumnsDescription StorageDistributed::createVirtualsForTieredDistributedMerge() const -{ - auto desc = createVirtuals(); // Get base virtuals from regular Distributed - - // Add _table_index virtual column for TieredDistributedMerge - desc.addEphemeral("_table_index", std::make_shared(), "Index of the table layer (0 for main layer, 1+ for additional layers)"); - - return desc; -} - } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 5aba428fa4ac..991b220bde8a 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -120,9 +120,6 @@ class StorageDistributed final : public IStorage, WithContext StorageSnapshotPtr getStorageSnapshotForQuery( const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr query_context) const override; - /// Override for TieredDistributedMerge to merge schemas from all layers - StorageInMemoryMetadata getInMemoryMetadata() const override; - QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; @@ -328,12 +325,6 @@ class StorageDistributed final : public IStorage, WithContext /// Additional table functions for TieredDistributedMerge engine std::vector additional_table_functions; - -private: - /// Helper methods for TieredDistributedMerge - ColumnsDescription getColumnsDescriptionFromLayers(const ContextPtr & query_context) const; - StorageSnapshotPtr getStorageSnapshotForLayer(const TableFunctionEntry & layer, const ContextPtr & query_context) const; - VirtualColumnsDescription createVirtualsForTieredDistributedMerge() const; }; } From 5a050717f11afcb292b03985de5976e0e5ed877e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 20:26:35 +0200 Subject: [PATCH 14/20] proper plan for layers --- src/Storages/StorageDistributed.cpp | 50 +++++++---------------------- 1 file changed, 11 insertions(+), 39 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index de205fff3233..73432b31605b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -1175,50 +1176,21 @@ void StorageDistributed::read( for (size_t i = 0; i < all_query_infos.size(); ++i) { auto additional_query_info = all_query_infos[i]; - // Get storage from either table function execution or StorageID resolution - StoragePtr storage; - if (additional_table_functions[i].storage_id.has_value()) - { - // For table identifiers, resolve the StorageID - storage = DatabaseCatalog::instance().getTable(additional_table_functions[i].storage_id.value(), local_context); - } - else - { - // For table functions, execute the AST - auto table_function = TableFunctionFactory::instance().get(additional_table_functions[i].table_function_ast, local_context); - if (!table_function) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in TieredDistributedMerge engine"); - - storage = table_function->execute( - additional_table_functions[i].table_function_ast, - local_context, - getStorageID().table_name, // Use the current table name - {}, // columns - will be determined from storage - false, // use_global_context = false - false); // is_insert_query = false - - // Handle StorageTableFunctionProxy if present - if (auto proxy = std::dynamic_pointer_cast(storage)) - { - storage = proxy->getNested(); - } - } auto additional_header = all_headers[i]; - // Create a new query plan for this additional storage - QueryPlan additional_plan; - // Execute the query against the additional storage - storage->read( - additional_plan, - {}, // column names - storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), local_context), - additional_query_info, + // This properly handles both analyzer and legacy modes with converting actions + auto additional_plan_ptr = createLocalPlan( + additional_query_info.query, + additional_header, local_context, processed_stage, - 0, // max_block_size - 0); // num_streams + 0, // shard_num - not applicable for local plans + 1, // shard_count - not applicable for local plans + false, // has_missing_objects + false, // build_logical_plan + ""); // default_database - additional_plans.push_back(std::move(additional_plan)); + additional_plans.push_back(std::move(*additional_plan_ptr)); } // Combine all plans using UnionStep From fc0994153bfb7b4e7653e358a6566f7f1f4eaa20 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 21:25:16 +0200 Subject: [PATCH 15/20] single header for all subtables --- src/Storages/StorageDistributed.cpp | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 73432b31605b..6c303b886d3d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1036,7 +1036,6 @@ void StorageDistributed::read( SelectQueryInfo modified_query_info = query_info; - std::vector all_headers; std::vector all_query_infos; const auto & settings = local_context->getSettingsRef(); @@ -1080,16 +1079,10 @@ void StorageDistributed::read( table_function_entry.table_function_ast, table_function_entry.predicate_ast); - // TODO: somewhere here the DESCRIBE TABLE is triggered, try to avoid it. - auto additional_header = InterpreterSelectQueryAnalyzer::getSampleBlock(additional_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - - for (auto & column : additional_header) - column.column = column.column->convertToFullColumnIfConst(); additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); - all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); } } @@ -1113,14 +1106,11 @@ void StorageDistributed::read( { SelectQueryInfo additional_query_info = query_info; - auto additional_header = InterpreterSelectQuery(additional_query_info.query, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", table_function_entry.table_function_ast, table_function_entry.predicate_ast); - all_headers.push_back(additional_header); all_query_infos.push_back(additional_query_info); } } @@ -1176,12 +1166,11 @@ void StorageDistributed::read( for (size_t i = 0; i < all_query_infos.size(); ++i) { auto additional_query_info = all_query_infos[i]; - auto additional_header = all_headers[i]; // This properly handles both analyzer and legacy modes with converting actions auto additional_plan_ptr = createLocalPlan( additional_query_info.query, - additional_header, + header, local_context, processed_stage, 0, // shard_num - not applicable for local plans From 5cfc7d740ee63d2637e7e5e861a30c26af386065 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 22:01:28 +0200 Subject: [PATCH 16/20] cosmetics --- src/Storages/StorageDistributed.cpp | 7 +++++++ src/Storages/StorageDistributed.h | 5 +---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 6c303b886d3d..ff9a578cda09 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1211,6 +1211,7 @@ void StorageDistributed::read( // Create UnionStep to combine all plans auto union_step = std::make_unique(std::move(headers), 0); + union_step->setStepDescription("TieredDistributedMerge"); union_plan.unitePlans(std::move(union_step), std::move(plan_ptrs)); @@ -2197,6 +2198,12 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const } } +void StorageDistributed::setAdditionalTableFunctions(std::vector additional_table_functions_) +{ + additional_table_functions = std::move(additional_table_functions_); + log = getLogger("TieredDistributedMerge (" + getStorageID().table_name + ")"); +} + void registerStorageDistributed(StorageFactory & factory) { factory.registerStorage("Distributed", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 991b220bde8a..08d34abee4f9 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -175,10 +175,7 @@ class StorageDistributed final : public IStorage, WithContext void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } /// Set additional table functions for TieredDistributedMerge engine - void setAdditionalTableFunctions(std::vector additional_table_functions_) - { - additional_table_functions = std::move(additional_table_functions_); - } + void setAdditionalTableFunctions(std::vector additional_table_functions_); /// Getter methods for ClusterProxy::executeQuery StorageID getRemoteStorageID() const { return remote_storage; } From 05d44f4955d9587bf933e62e4f466553797c8981 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 1 Oct 2025 23:05:51 +0200 Subject: [PATCH 17/20] add virtual column registration --- src/Storages/StorageDistributed.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ff9a578cda09..ac6c4fbc8470 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2202,6 +2202,11 @@ void StorageDistributed::setAdditionalTableFunctions(std::vector(), "Index of the table function in TieredDistributedMerge (0 for main table, 1+ for additional table functions)"); + setVirtuals(virtuals); } void registerStorageDistributed(StorageFactory & factory) From 3b0417a50eb15d139e4de313f81f7f1e9e694f12 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 2 Oct 2025 22:20:21 +0200 Subject: [PATCH 18/20] Add support for table names instead of table functions for layers 2,... --- src/Storages/StorageDistributed.cpp | 74 ++++++++++++++++++++++++----- 1 file changed, 62 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ac6c4fbc8470..2d4f1709d433 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -205,6 +205,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int STORAGE_REQUIRES_PARAMETER; extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_DATABASE; + extern const int UNKNOWN_TABLE; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INFINITE_LOOP; @@ -1075,11 +1077,10 @@ void StorageDistributed::read( auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, - StorageID::createEmpty(), - table_function_entry.table_function_ast, + table_function_entry.storage_id ? *table_function_entry.storage_id : StorageID::createEmpty(), + table_function_entry.storage_id ? nullptr : table_function_entry.table_function_ast, table_function_entry.predicate_ast); - additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); @@ -1106,10 +1107,21 @@ void StorageDistributed::read( { SelectQueryInfo additional_query_info = query_info; - additional_query_info.query = ClusterProxy::rewriteSelectQuery( - local_context, additional_query_info.query, - "", "", table_function_entry.table_function_ast, - table_function_entry.predicate_ast); + if (table_function_entry.storage_id) + { + additional_query_info.query = ClusterProxy::rewriteSelectQuery( + local_context, additional_query_info.query, + table_function_entry.storage_id->database_name, table_function_entry.storage_id->table_name, + nullptr, + table_function_entry.predicate_ast); + } + else + { + additional_query_info.query = ClusterProxy::rewriteSelectQuery( + local_context, additional_query_info.query, + "", "", table_function_entry.table_function_ast, + table_function_entry.predicate_ast); + } all_query_infos.push_back(additional_query_info); } @@ -2397,7 +2409,7 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument #{} must be a valid SQL expression: {}", i + 1, e.message()); } - + // Validate table function or table identifier if (const auto * func = table_function_ast->as()) { @@ -2411,19 +2423,57 @@ void registerStorageTieredDistributedMerge(StorageFactory & factory) // It's a table function - store the AST for later execution additional_table_functions.emplace_back(table_function_ast, predicate_ast); } - else if (const auto * identifier = table_function_ast->as()) + else if (const auto * ast_identifier = table_function_ast->as()) { - // It's a table identifier - validate it can be parsed as StorageID + // It's an identifier - try to convert it to a table identifier + auto table_identifier = ast_identifier->createTable(); + if (!table_identifier) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: identifier '{}' cannot be converted to table identifier", i, ast_identifier->name()); + } + try { // Parse table identifier to get StorageID - StorageID storage_id(table_function_ast); + StorageID storage_id(table_identifier); + + // Sanity check: ensure the table identifier is fully qualified (has database name) + if (storage_id.database_name.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: table identifier '{}' must be fully qualified (database.table)", i, ast_identifier->name()); + } + + // Sanity check: verify the table exists + try + { + auto database = DatabaseCatalog::instance().getDatabase(storage_id.database_name, context); + if (!database) + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, + "Database '{}' does not exist", storage_id.database_name); + } + + auto table = database->tryGetTable(storage_id.table_name, context); + if (!table) + { + throw Exception(ErrorCodes::UNKNOWN_TABLE, + "Table '{}.{}' does not exist", storage_id.database_name, storage_id.table_name); + } + } + catch (const Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Argument #{}: table '{}' validation failed: {}", i, ast_identifier->name(), e.message()); + } + additional_table_functions.emplace_back(table_function_ast, predicate_ast, storage_id); } catch (const Exception & e) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument #{}: invalid table identifier '{}': {}", i, identifier->name(), e.message()); + "Argument #{}: invalid table identifier '{}': {}", i, ast_identifier->name(), e.message()); } } else From d486f050be5fad1603f61a2368bff28d6d2ed97b Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 2 Oct 2025 22:41:44 +0200 Subject: [PATCH 19/20] bit safer main plan processing --- src/Storages/StorageDistributed.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2d4f1709d433..4ce12ab8d712 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1199,10 +1199,11 @@ void StorageDistributed::read( { // Convert QueryPlan objects to QueryPlanPtr std::vector plan_ptrs; - plan_ptrs.reserve(additional_plans.size() + 1); + plan_ptrs.reserve(additional_plans.size() + query_plan.isInitialized() ? 1 : 0); // Add the main plan to the list - plan_ptrs.push_back(std::make_unique(std::move(query_plan))); + if (query_plan.isInitialized()) + plan_ptrs.push_back(std::make_unique(std::move(query_plan))); // Add additional plans for (auto & plan : additional_plans) From 19d8785db1f2b5ce9a1424ce4821494237f16306 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 2 Oct 2025 22:58:29 +0200 Subject: [PATCH 20/20] try to workaround the issue with distributed_aggregation_memory_efficient --- src/Storages/StorageDistributed.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4ce12ab8d712..3acc6400c94c 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1042,6 +1042,17 @@ void StorageDistributed::read( const auto & settings = local_context->getSettingsRef(); + // Disable memory_bound settings when additional table functions are present + // to avoid UNKNOWN_AGGREGATED_DATA_VARIANT when mixing different aggregation variants + // from remote shards (with memory_bound) and local layers (without memory_bound) + // FIXME: we can push additional_query_info into ClusterProxy::executeQuery to avoid this hack + // TODO: test is needed + if (!additional_table_functions.empty()) + { + const_cast(local_context.get())->setSetting("enable_memory_bound_merging_of_aggregation_results", false); + const_cast(local_context.get())->setSetting("distributed_aggregation_memory_efficient", false); + } + if (settings[Setting::allow_experimental_analyzer]) { StorageID remote_storage_id = StorageID::createEmpty(); @@ -1199,7 +1210,7 @@ void StorageDistributed::read( { // Convert QueryPlan objects to QueryPlanPtr std::vector plan_ptrs; - plan_ptrs.reserve(additional_plans.size() + query_plan.isInitialized() ? 1 : 0); + plan_ptrs.reserve(additional_plans.size() + (query_plan.isInitialized() ? 1 : 0)); // Add the main plan to the list if (query_plan.isInitialized())