From cd1a38f95da4c69af64be2024061a4a25e0bd29f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 15:07:05 +0100 Subject: [PATCH 01/26] rename files --- .../table-engines/special/{tiered-distributed.md => hybrid.md} | 2 +- ...3642_tiered_distributed.reference => 03642_hybrid.reference} | 0 .../{03642_tiered_distributed.sql => 03642_hybrid.sql} | 0 3 files changed, 1 insertion(+), 1 deletion(-) rename docs/en/engines/table-engines/special/{tiered-distributed.md => hybrid.md} (98%) rename tests/queries/0_stateless/{03642_tiered_distributed.reference => 03642_hybrid.reference} (100%) rename tests/queries/0_stateless/{03642_tiered_distributed.sql => 03642_hybrid.sql} (100%) diff --git a/docs/en/engines/table-engines/special/tiered-distributed.md b/docs/en/engines/table-engines/special/hybrid.md similarity index 98% rename from docs/en/engines/table-engines/special/tiered-distributed.md rename to docs/en/engines/table-engines/special/hybrid.md index 95a986ff48de..da3578c11535 100644 --- a/docs/en/engines/table-engines/special/tiered-distributed.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -1,6 +1,6 @@ --- description: 'Hybrid unions multiple data sources behind per-layer predicates so queries behave like a single table while data is migrated or tiered.' -slug: /engines/table-engines/special/tiered-distributed +slug: /engines/table-engines/special/hybrid title: 'Hybrid Table Engine' sidebar_label: 'Hybrid' sidebar_position: 11 diff --git a/tests/queries/0_stateless/03642_tiered_distributed.reference b/tests/queries/0_stateless/03642_hybrid.reference similarity index 100% rename from tests/queries/0_stateless/03642_tiered_distributed.reference rename to tests/queries/0_stateless/03642_hybrid.reference diff --git a/tests/queries/0_stateless/03642_tiered_distributed.sql b/tests/queries/0_stateless/03642_hybrid.sql similarity index 100% rename from tests/queries/0_stateless/03642_tiered_distributed.sql rename to tests/queries/0_stateless/03642_hybrid.sql From 6ad282eb87963932ce879741a89280115e041c40 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 15:15:55 +0100 Subject: [PATCH 02/26] rename layers to segments --- .../engines/table-engines/special/hybrid.md | 18 ++--- src/Storages/StorageDistributed.cpp | 2 +- .../0_stateless/03642_hybrid.reference | 18 ++--- tests/queries/0_stateless/03642_hybrid.sql | 66 +++++++++---------- 4 files changed, 52 insertions(+), 52 deletions(-) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index da3578c11535..8e2819190961 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -1,5 +1,5 @@ --- -description: 'Hybrid unions multiple data sources behind per-layer predicates so queries behave like a single table while data is migrated or tiered.' +description: 'Hybrid unions multiple data sources behind per-segment predicates so queries behave like a single table while data is migrated or tiered.' slug: /engines/table-engines/special/hybrid title: 'Hybrid Table Engine' sidebar_label: 'Hybrid' @@ -9,7 +9,7 @@ sidebar_position: 11 # Hybrid table engine `Hybrid` builds on top of the [Distributed](./distributed.md) table engine. It lets you expose several data sources as one logical table and assign every source its own predicate. -The engine rewrites incoming queries so that each layer receives the original query plus its predicate. This keeps all of the Distributed optimisations (remote aggregation, `skip_unused_shards`, +The engine rewrites incoming queries so that each segment receives the original query plus its predicate. This keeps all of the Distributed optimisations (remote aggregation, `skip_unused_shards`, global JOIN pushdown, and so on) while you duplicate or migrate data across clusters, storage types, or formats. It keeps the same execution pipeline as `engine=Distributed` but can read from multiple underlying sources simultaneously—similar to `engine=Merge`—while still pushing logic down to each source. @@ -20,7 +20,7 @@ Typical use cases include: - Tiered storage, for example fresh data on a local cluster and historical data in S3. - Gradual roll-outs where only a subset of rows should be served from a new backend. -By giving mutually exclusive predicates to the layers (for example, `date < watermark` and `date >= watermark`), you ensure that each row is read from exactly one source. +By giving mutually exclusive predicates to the segments (for example, `date < watermark` and `date >= watermark`), you ensure that each row is read from exactly one source. ## Engine definition @@ -39,14 +39,14 @@ You must pass at least two arguments – the first table function and its predic ### Arguments and behaviour - `table_function_n` must be a valid table function (for example `remote`, `remoteSecure`, `cluster`, `clusterAllReplicas`, `s3Cluster`) or a fully qualified table name (`database.table`). The first argument must be a table function—such as `remote` or `cluster`—because it instantiates the underlying `Distributed` storage. -- `predicate_n` must be an expression that can be evaluated on the table columns. The engine adds it to the layer's query with an additional `AND`, so expressions like `event_date >= '2025-09-01'` or `id BETWEEN 10 AND 15` are typical. -- The query planner picks the same processing stage for every layer as it does for the base `Distributed` plan, so remote aggregation, ORDER BY pushdown, `skip_unused_shards`, and the legacy/analyzer execution modes behave the same way. +- `predicate_n` must be an expression that can be evaluated on the table columns. The engine adds it to the segment's query with an additional `AND`, so expressions like `event_date >= '2025-09-01'` or `id BETWEEN 10 AND 15` are typical. +- The query planner picks the same processing stage for every segment as it does for the base `Distributed` plan, so remote aggregation, ORDER BY pushdown, `skip_unused_shards`, and the legacy/analyzer execution modes behave the same way. - `INSERT` statements are forwarded to the first table function only. If you need multi-destination writes, use explicit `INSERT` statements into the respective sources. -- Align schemas across the layers. ClickHouse builds a common header; if the physical types differ you may need to add casts on one side or in the query, just as you would when reading from heterogeneous replicas. +- Align schemas across the segments. ClickHouse builds a common header; if the physical types differ you may need to add casts on one side or in the query, just as you would when reading from heterogeneous replicas. ## Example: local cluster plus S3 historical tier -The following commands illustrate a two-layer layout. Hot data stays on a local ClickHouse cluster, while historical rows come from public S3 Parquet files. +The following commands illustrate a two-segment layout. Hot data stays on a local ClickHouse cluster, while historical rows come from public S3 Parquet files. ```sql -- Local MergeTree table that keeps current data @@ -79,7 +79,7 @@ CREATE OR REPLACE TABLE btc_blocks ENGINE = Hybrid( s3('s3://aws-public-blockchain/v1.0/btc/blocks/**.parquet', NOSIGN), date < '2025-09-01' ) AS btc_blocks_local; --- Writes target the first (remote) layer +-- Writes target the first (remote) segment INSERT INTO btc_blocks SELECT * FROM s3('s3://aws-public-blockchain/v1.0/btc/blocks/**.parquet', NOSIGN) @@ -103,4 +103,4 @@ GROUP BY date ORDER BY date; ``` -Because the predicates are applied inside every layer, queries such as `ORDER BY`, `GROUP BY`, `LIMIT`, `JOIN`, and `EXPLAIN` behave as if you were reading from a single `Distributed` table. When sources expose different physical types (for example `FixedString(64)` versus `String` in Parquet), add explicit casts during ingestion or in the query, as shown above. +Because the predicates are applied inside every segment, queries such as `ORDER BY`, `GROUP BY`, `LIMIT`, `JOIN`, and `EXPLAIN` behave as if you were reading from a single `Distributed` table. When sources expose different physical types (for example `FixedString(64)` versus `String` in Parquet), add explicit casts during ingestion or in the query, as shown above. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a3ffad98a47b..c50a67d0b9bf 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2196,7 +2196,7 @@ void StorageDistributed::setHybridLayout(std::vector additio log = getLogger("Hybrid (" + getStorageID().table_name + ")"); auto virtuals = createVirtuals(); - // or _layer_index? + // or _segment_index? virtuals.addEphemeral("_table_index", std::make_shared(), "Index of the table function in Hybrid (0 for main table, 1+ for additional table functions)"); setVirtuals(virtuals); } diff --git a/tests/queries/0_stateless/03642_hybrid.reference b/tests/queries/0_stateless/03642_hybrid.reference index d4ac38a21257..fc5db02aa8cc 100644 --- a/tests/queries/0_stateless/03642_hybrid.reference +++ b/tests/queries/0_stateless/03642_hybrid.reference @@ -27,7 +27,7 @@ CREATE TABLE default.test_tiered_distributed_numbers_range\n(\n `number` UInt 2 3 4 -Create Hybrid table with two remote layers as table +Create Hybrid table with two remote segments as table CREATE TABLE default.test_tiered_distributed_numbers_dual\n(\n `number` UInt64\n)\nENGINE = Hybrid(remote(\'localhost:9000\', \'system.numbers\'), number < 5, remote(\'localhost:9000\', system.numbers), (number >= 10) AND (number <= 15))\nCOMMENT \'Generates all natural numbers, starting from 0 (to 2^64 - 1, and then again) in sorted order.\' 0 1 @@ -63,23 +63,23 @@ Create Hybrid table combining remote function and local table 13 14 15 -Verify Hybrid skips layer with always false predicate on the first layer +Verify Hybrid skips segment with always false predicate on the first segment 10 11 12 13 14 15 -Verify Hybrid skips layer with always false predicate on the second layer +Verify Hybrid skips segment with always false predicate on the second segment 0 1 2 -Prepare local MergeTree table for multi-layer tests +Prepare local MergeTree table for multi-segment tests Populate local table with sample data -Create Hybrid table with three layer pairs -Count rows across all layers +Create Hybrid table with three segment pairs +Count rows across all segments 6 -Count rows from layers with id > 4 +Count rows from segments with id > 4 1 Count rows where value > 200 3 @@ -127,7 +127,7 @@ Verify additional_table_filters works consistently (legacy analyser) 2 Bob 200.3 Verify additional_table_filters works consistently (new analyser) 2 Bob 200.3 -Clean up Hybrid table with three layer pairs +Clean up Hybrid table with three segment pairs Clean up local helper table Drop predicate filtering fixtures if they exist Create local tables representing before/after watermark partitions @@ -135,7 +135,7 @@ Create second local table with different value type Insert rows before watermark into both tables Insert rows after watermark into both tables Create Hybrid table with analyzer disabled during reads -Insert row via Hybrid table (should go to first layer) +Insert row via Hybrid table (should go to first segment) Verify that inserted row landed in first table 17 John 2025-09-25 400 Verify that second table did not receive the inserted row diff --git a/tests/queries/0_stateless/03642_hybrid.sql b/tests/queries/0_stateless/03642_hybrid.sql index 4fd346bba166..78adac6da028 100644 --- a/tests/queries/0_stateless/03642_hybrid.sql +++ b/tests/queries/0_stateless/03642_hybrid.sql @@ -49,7 +49,7 @@ SHOW CREATE TABLE test_tiered_distributed_numbers_range; SELECT * FROM test_tiered_distributed_numbers_range ORDER BY number; DROP TABLE IF EXISTS test_tiered_distributed_numbers_range SYNC; -SELECT 'Create Hybrid table with two remote layers as table'; +SELECT 'Create Hybrid table with two remote segments as table'; DROP TABLE IF EXISTS test_tiered_distributed_numbers_dual SYNC; CREATE TABLE test_tiered_distributed_numbers_dual ENGINE = Hybrid( remote('localhost:9000', system.numbers), number < 5, @@ -73,7 +73,7 @@ CREATE TABLE test_tiered_distributed_numbers_mixed SELECT * FROM test_tiered_distributed_numbers_mixed ORDER BY number; DROP TABLE IF EXISTS test_tiered_distributed_numbers_mixed SYNC; -SELECT 'Verify Hybrid skips layer with always false predicate on the first layer'; +SELECT 'Verify Hybrid skips segment with always false predicate on the first segment'; DROP TABLE IF EXISTS test_tiered_distributed_numbers_skip_first SYNC; CREATE TABLE test_tiered_distributed_numbers_skip_first ( @@ -85,7 +85,7 @@ CREATE TABLE test_tiered_distributed_numbers_skip_first SELECT * FROM test_tiered_distributed_numbers_skip_first ORDER BY number; DROP TABLE IF EXISTS test_tiered_distributed_numbers_skip_first SYNC; -SELECT 'Verify Hybrid skips layer with always false predicate on the second layer'; +SELECT 'Verify Hybrid skips segment with always false predicate on the second segment'; DROP TABLE IF EXISTS test_tiered_distributed_numbers_skip_second SYNC; CREATE TABLE test_tiered_distributed_numbers_skip_second ( @@ -99,7 +99,7 @@ DROP TABLE IF EXISTS test_tiered_distributed_numbers_skip_second SYNC; ----------------------------- -SELECT 'Prepare local MergeTree table for multi-layer tests'; +SELECT 'Prepare local MergeTree table for multi-segment tests'; DROP TABLE IF EXISTS test_tiered_local_data SYNC; CREATE TABLE test_tiered_local_data ( @@ -119,10 +119,10 @@ INSERT INTO test_tiered_local_data VALUES (4, 'David', '2022-01-04 13:00:00', 300.2), (5, 'Eve', '2022-01-05 14:00:00', 250.1); -SELECT 'Create Hybrid table with three layer pairs'; -DROP TABLE IF EXISTS test_tiered_multi_layer SYNC; +SELECT 'Create Hybrid table with three segment pairs'; +DROP TABLE IF EXISTS test_tiered_multi_segment SYNC; -CREATE TABLE test_tiered_multi_layer +CREATE TABLE test_tiered_multi_segment ( `id` UInt32, `name` String, @@ -138,52 +138,52 @@ ENGINE = Hybrid( id > 3 ); -SELECT 'Count rows across all layers'; -SELECT count() FROM test_tiered_multi_layer; -SELECT 'Count rows from layers with id > 4'; -SELECT count() FROM test_tiered_multi_layer WHERE id > 4; +SELECT 'Count rows across all segments'; +SELECT count() FROM test_tiered_multi_segment; +SELECT 'Count rows from segments with id > 4'; +SELECT count() FROM test_tiered_multi_segment WHERE id > 4; SELECT 'Count rows where value > 200'; -SELECT count() FROM test_tiered_multi_layer WHERE value > 200; +SELECT count() FROM test_tiered_multi_segment WHERE value > 200; SELECT 'Count rows named Alice'; -SELECT count() AS alice_rows FROM test_tiered_multi_layer WHERE name = 'Alice'; +SELECT count() AS alice_rows FROM test_tiered_multi_segment WHERE name = 'Alice'; SELECT 'Select rows ordered by value descending (id > 2)'; -SELECT id, name, value FROM test_tiered_multi_layer WHERE id > 2 ORDER BY value DESC; +SELECT id, name, value FROM test_tiered_multi_segment WHERE id > 2 ORDER BY value DESC; SELECT 'Limit results ordered by id'; -SELECT * FROM test_tiered_multi_layer ORDER BY id LIMIT 3; +SELECT * FROM test_tiered_multi_segment ORDER BY id LIMIT 3; SELECT 'Explain plan for filter on value'; -EXPLAIN SELECT * FROM test_tiered_multi_layer WHERE value > 150 SETTINGS prefer_localhost_replica=0, enable_analyzer=0; -EXPLAIN SELECT * FROM test_tiered_multi_layer WHERE value > 150 SETTINGS prefer_localhost_replica=0, enable_analyzer=1; -EXPLAIN SELECT * FROM test_tiered_multi_layer WHERE value > 150 SETTINGS prefer_localhost_replica=1, enable_analyzer=0; -EXPLAIN SELECT * FROM test_tiered_multi_layer WHERE value > 150 SETTINGS prefer_localhost_replica=1, enable_analyzer=1; +EXPLAIN SELECT * FROM test_tiered_multi_segment WHERE value > 150 SETTINGS prefer_localhost_replica=0, enable_analyzer=0; +EXPLAIN SELECT * FROM test_tiered_multi_segment WHERE value > 150 SETTINGS prefer_localhost_replica=0, enable_analyzer=1; +EXPLAIN SELECT * FROM test_tiered_multi_segment WHERE value > 150 SETTINGS prefer_localhost_replica=1, enable_analyzer=0; +EXPLAIN SELECT * FROM test_tiered_multi_segment WHERE value > 150 SETTINGS prefer_localhost_replica=1, enable_analyzer=1; SELECT 'Aggregate values across name when filtering by event_time'; SELECT name, count() AS count, avg(value) AS avg_value -FROM test_tiered_multi_layer +FROM test_tiered_multi_segment WHERE event_time >= '2022-01-02' GROUP BY name ORDER BY avg_value DESC; SELECT 'Verify additional_table_filters works consistently (legacy analyser)'; SELECT id, name, value -FROM test_tiered_multi_layer +FROM test_tiered_multi_segment WHERE id < 3 ORDER BY id -SETTINGS additional_table_filters = {'test_tiered_multi_layer' : 'id > 1'}, allow_experimental_analyzer = 0; +SETTINGS additional_table_filters = {'test_tiered_multi_segment' : 'id > 1'}, allow_experimental_analyzer = 0; SELECT 'Verify additional_table_filters works consistently (new analyser)'; SELECT id, name, value -FROM test_tiered_multi_layer +FROM test_tiered_multi_segment WHERE id < 3 ORDER BY id -SETTINGS additional_table_filters = {'test_tiered_multi_layer' : 'id > 1'}, allow_experimental_analyzer = 1; +SETTINGS additional_table_filters = {'test_tiered_multi_segment' : 'id > 1'}, allow_experimental_analyzer = 1; -SELECT 'Clean up Hybrid table with three layer pairs'; -DROP TABLE IF EXISTS test_tiered_multi_layer SYNC; +SELECT 'Clean up Hybrid table with three segment pairs'; +DROP TABLE IF EXISTS test_tiered_multi_segment SYNC; SELECT 'Clean up local helper table'; DROP TABLE IF EXISTS test_tiered_local_data SYNC; @@ -254,7 +254,7 @@ ENGINE = Hybrid( date < '2025-09-01' ); -SELECT 'Insert row via Hybrid table (should go to first layer)'; +SELECT 'Insert row via Hybrid table (should go to first segment)'; INSERT INTO test_tiered_watermark SETTINGS distributed_foreground_insert = 1 VALUES (17, 'John', '2025-09-25', 400); @@ -326,13 +326,13 @@ DROP TABLE IF EXISTS test_tiered_watermark_before SYNC; -- 1. Integration tests (similar to tests/queries/0_stateless) -- - Base SELECT with date split: part in Distributed, part in S3 -> results should match a manual UNION ALL (with correct ORDER BY/aggregation). -- - GROUP BY / ORDER BY / LIMIT: confirm the stage is selected correctly, finalization happens at the top, rows_before_limit_at_least is correct (createLocalPlan already keeps LIMIT). --- - JOIN: with a small table on the initiator; check GLOBAL JOIN scenarios. Ensure remote layers behave the same as remote shard subqueries created through createLocalPlan. --- - skipUnusedShards: with analyzer ensure layer conditions are respected (where FILTER DAG is available). --- - Constants: hostName()/now() in SELECT across several layers -> ensure no discrepancies. --- - EXPLAIN PLAN/PIPELINE: show child plans for layers and remote plans. +-- - JOIN: with a small table on the initiator; check GLOBAL JOIN scenarios. Ensure remote segments behave the same as remote shard subqueries created through createLocalPlan. +-- - skipUnusedShards: with analyzer ensure segment conditions are respected (where FILTER DAG is available). +-- - Constants: hostName()/now() in SELECT across several segments -> ensure no discrepancies. +-- - EXPLAIN PLAN/PIPELINE: show child plans for segments and remote plans. -- - Subqueries in logs. -- - Different column sets/types: supertype in snapshot, converting actions on read. --- - Object columns: same as Distributed — use ColumnsDescriptionByShardNum for layers if needed (optional for local layers; already implemented for Distributed). +-- - Object columns: same as Distributed — use ColumnsDescriptionByShardNum for segments if needed (optional for local segments; already implemented for Distributed). -- Condition with dictGet('a1_watermarks_dict', ...) @@ -342,4 +342,4 @@ DROP TABLE IF EXISTS test_tiered_watermark_before SYNC; -- TODO: -- test for distributed_aggregation_memory_efficient & enable_memory_bound_merging_of_aggregation_results -- to avoid UNKNOWN_AGGREGATED_DATA_VARIANT when mixing different aggregation variants --- from remote shards (with memory_bound) and local layers (without memory_bound) +-- from remote shards (with memory_bound) and local segments (without memory_bound) From 5d47f072dc9e598c114e28e7f61956b77525ed70 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 22:52:47 +0100 Subject: [PATCH 03/26] store headers --- src/Storages/StorageDistributed.cpp | 29 ++++++++++++++++++----------- src/Storages/StorageDistributed.h | 14 +++++++++----- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c50a67d0b9bf..42d3b09d2db9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2190,8 +2190,9 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const } } -void StorageDistributed::setHybridLayout(std::vector additional_table_functions_) +void StorageDistributed::setHybridLayout(ColumnsDescription base_segment_columns_, std::vector additional_table_functions_) { + base_segment_columns = std::move(base_segment_columns_); additional_table_functions = std::move(additional_table_functions_); log = getLogger("Hybrid (" + getStorageID().table_name + ")"); @@ -2348,13 +2349,13 @@ void registerStorageHybrid(StorageFactory & factory) if (!table_function) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid table function in Hybrid engine"); - // For schema inference, we need to determine the columns first if they're not provided + // Capture the physical columns reported by the first segment (table function) + ColumnsDescription first_segment_columns = table_function->getActualTableStructure(local_context, true); + + // For schema inference, prefer user-provided columns, otherwise use the physical ones 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(local_context, true); - } + columns_to_use = first_segment_columns; // Execute the table function to get the underlying storage StoragePtr storage = table_function->execute( @@ -2429,11 +2430,11 @@ void registerStorageHybrid(StorageFactory & factory) // TableFunctionFactory::get mutates the AST in-place inside TableFunctionRemote::parseArguments. ASTPtr normalized_table_function_ast = table_function_ast->clone(); auto additional_table_function = TableFunctionFactory::instance().get(normalized_table_function_ast, local_context); - (void)additional_table_function; + ColumnsDescription segment_columns = additional_table_function->getActualTableStructure(local_context, true); replaceCurrentDatabaseFunction(normalized_table_function_ast, local_context); - // It's a table function - store the AST for later execution - additional_table_functions.emplace_back(normalized_table_function_ast, predicate_ast); + // It's a table function - store the AST and cached schema for later execution + additional_table_functions.emplace_back(normalized_table_function_ast, predicate_ast, std::move(segment_columns)); } else if (const auto * ast_identifier = table_function_ast->as()) { @@ -2445,6 +2446,7 @@ void registerStorageHybrid(StorageFactory & factory) "Argument #{}: identifier '{}' cannot be converted to table identifier", i, ast_identifier->name()); } + StoragePtr validated_table; try { // Parse table identifier to get StorageID @@ -2473,6 +2475,7 @@ void registerStorageHybrid(StorageFactory & factory) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table '{}.{}' does not exist", storage_id.database_name, storage_id.table_name); } + validated_table = table; } catch (const Exception & e) { @@ -2480,7 +2483,11 @@ void registerStorageHybrid(StorageFactory & factory) "Argument #{}: table '{}' validation failed: {}", i, ast_identifier->name(), e.message()); } - additional_table_functions.emplace_back(table_function_ast, predicate_ast, storage_id); + ColumnsDescription segment_columns; + if (validated_table) + segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); + + additional_table_functions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); } catch (const Exception & e) { @@ -2502,7 +2509,7 @@ void registerStorageHybrid(StorageFactory & factory) distributed_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); // Store additional table functions for later use - distributed_storage->setHybridLayout(std::move(additional_table_functions)); + distributed_storage->setHybridLayout(std::move(first_segment_columns), std::move(additional_table_functions)); return distributed_storage; }, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 273a1c233c81..788c8fa2537f 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -50,22 +50,25 @@ class StorageDistributed final : public IStorage, WithContext friend class StorageSystemDistributionQueue; public: - /// Structure to hold table function AST, predicate, and optional StorageID for table identifiers + /// Structure to hold table function AST, predicate, optional StorageID, and cached physical columns for the segment struct TableFunctionEntry { ASTPtr table_function_ast; ASTPtr predicate_ast; std::optional storage_id; // For table identifiers instead of table functions + ColumnsDescription actual_columns; - TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_) + TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, ColumnsDescription actual_columns_) : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) + , actual_columns(std::move(actual_columns_)) {} - TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_) + TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_, ColumnsDescription actual_columns_) : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) , storage_id(std::move(storage_id_)) + , actual_columns(std::move(actual_columns_)) {} }; @@ -176,8 +179,8 @@ class StorageDistributed final : public IStorage, WithContext /// Set additional filter for Hybrid engine void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } - /// Set additional table functions for Hybrid engine - void setHybridLayout(std::vector additional_table_functions_); + /// Set additional table functions for Hybrid engine along with cached schema info + void setHybridLayout(ColumnsDescription base_segment_columns_, std::vector additional_table_functions_); /// Getter methods for ClusterProxy::executeQuery StorageID getRemoteStorageID() const { return remote_storage; } @@ -325,6 +328,7 @@ class StorageDistributed final : public IStorage, WithContext /// Additional table functions for Hybrid engine std::vector additional_table_functions; + ColumnsDescription base_segment_columns; }; } From e59a1f35d26cc1d2c9f8b03a6b84aae92c1cd5f2 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 23:06:18 +0100 Subject: [PATCH 04/26] more renames --- src/Storages/StorageDistributed.cpp | 64 ++++++++++++++--------------- src/Storages/StorageDistributed.h | 16 ++++---- 2 files changed, 40 insertions(+), 40 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 42d3b09d2db9..1be2a0c13798 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -548,8 +548,8 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( return QueryProcessingStage::WithMergeableState; // TODO: check logic - if (!additional_table_functions.empty()) - nodes += additional_table_functions.size(); + if (!segments.empty()) + nodes += segments.size(); /// If there is only one node, the query can be fully processed by the /// shard, initiator will work as a proxy only. @@ -593,7 +593,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( bool StorageDistributed::isShardingKeySuitsQueryTreeNodeExpression( const QueryTreeNodePtr & expr, const SelectQueryInfo & query_info) const { - if (!additional_table_functions.empty()) + if (!segments.empty()) return false; ColumnsWithTypeAndName empty_input_columns; @@ -636,7 +636,7 @@ bool StorageDistributed::isShardingKeySuitsQueryTreeNodeExpression( return allOutputsDependsOnlyOnAllowedNodes(sharding_key_dag, irreducibe_nodes, matches); } -// TODO: support additional table functions +// TODO: support additional segments 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] @@ -695,7 +695,7 @@ std::optional StorageDistributed::getOptimizedQueryP return QueryProcessingStage::Complete; } -// TODO: support additional table functions +// TODO: support additional segments 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] @@ -805,11 +805,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 + /// TODO: support additional segments return getStorageSnapshotForQuery(metadata_snapshot, nullptr, query_context); } -/// TODO: support additional table functions +/// TODO: support additional segments StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( const StorageMetadataPtr & metadata_snapshot, const ASTPtr & query, ContextPtr /*query_context*/) const { @@ -1101,18 +1101,18 @@ void StorageDistributed::read( modified_query_info.query_tree = std::move(query_tree_distributed); - if (!additional_table_functions.empty()) + if (!segments.empty()) { - for (const auto & table_function_entry : additional_table_functions) + for (const auto & segment : segments) { // Create a modified query info with the additional predicate SelectQueryInfo additional_query_info = query_info; auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, - 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); + segment.storage_id ? *segment.storage_id : StorageID::createEmpty(), + segment.storage_id ? nullptr : segment.table_function_ast, + segment.predicate_ast); additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); @@ -1121,8 +1121,8 @@ void StorageDistributed::read( } } - // For empty shards - avoid early return if we have additional table functions - if (modified_query_info.getCluster()->getShardsInfo().empty() && additional_table_functions.empty()) + // For empty shards - avoid early return if we have additional segments + if (modified_query_info.getCluster()->getShardsInfo().empty() && segments.empty()) return; } else @@ -1134,34 +1134,34 @@ void StorageDistributed::read( remote_database, remote_table, remote_table_function_ptr, additional_filter); - if (!additional_table_functions.empty()) + if (!segments.empty()) { - for (const auto & table_function_entry : additional_table_functions) + for (const auto & segment : segments) { SelectQueryInfo additional_query_info = query_info; - if (table_function_entry.storage_id) + if (segment.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, + segment.storage_id->database_name, segment.storage_id->table_name, nullptr, - table_function_entry.predicate_ast); + segment.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); + "", "", segment.table_function_ast, + segment.predicate_ast); } additional_query_infos.push_back(std::move(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()) + // For empty shards - avoid early return if we have additional segments + if (modified_query_info.getCluster()->getShardsInfo().empty() && segments.empty()) { Pipe pipe(std::make_shared(header)); auto read_from_pipe = std::make_unique(std::move(pipe)); @@ -2190,15 +2190,15 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const } } -void StorageDistributed::setHybridLayout(ColumnsDescription base_segment_columns_, std::vector additional_table_functions_) +void StorageDistributed::setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_) { base_segment_columns = std::move(base_segment_columns_); - additional_table_functions = std::move(additional_table_functions_); + segments = std::move(segments_); log = getLogger("Hybrid (" + getStorageID().table_name + ")"); auto virtuals = createVirtuals(); // or _segment_index? - virtuals.addEphemeral("_table_index", std::make_shared(), "Index of the table function in Hybrid (0 for main table, 1+ for additional table functions)"); + virtuals.addEphemeral("_table_index", std::make_shared(), "Index of the table function in Hybrid (0 for main table, 1+ for additional segments)"); setVirtuals(virtuals); } @@ -2404,7 +2404,7 @@ void registerStorageHybrid(StorageFactory & factory) distributed_storage->setAdditionalFilter(second_arg); // Parse additional table function pairs (if any) - std::vector additional_table_functions; + std::vector segment_definitions; for (size_t i = 2; i < engine_args.size(); i += 2) { if (i + 1 >= engine_args.size()) @@ -2434,7 +2434,7 @@ void registerStorageHybrid(StorageFactory & factory) replaceCurrentDatabaseFunction(normalized_table_function_ast, local_context); // It's a table function - store the AST and cached schema for later execution - additional_table_functions.emplace_back(normalized_table_function_ast, predicate_ast, std::move(segment_columns)); + segment_definitions.emplace_back(normalized_table_function_ast, predicate_ast, std::move(segment_columns)); } else if (const auto * ast_identifier = table_function_ast->as()) { @@ -2485,9 +2485,9 @@ void registerStorageHybrid(StorageFactory & factory) ColumnsDescription segment_columns; if (validated_table) - segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); + segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); - additional_table_functions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); + segment_definitions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); } catch (const Exception & e) { @@ -2508,8 +2508,8 @@ void registerStorageHybrid(StorageFactory & factory) // 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 additional table functions for later use - distributed_storage->setHybridLayout(std::move(first_segment_columns), std::move(additional_table_functions)); + // Store segment definitions for later use + distributed_storage->setHybridLayout(std::move(first_segment_columns), std::move(segment_definitions)); return distributed_storage; }, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 788c8fa2537f..5a268cddfbb1 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -51,20 +51,20 @@ class StorageDistributed final : public IStorage, WithContext public: /// Structure to hold table function AST, predicate, optional StorageID, and cached physical columns for the segment - struct TableFunctionEntry + struct HybridSegment { ASTPtr table_function_ast; ASTPtr predicate_ast; std::optional storage_id; // For table identifiers instead of table functions ColumnsDescription actual_columns; - TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, ColumnsDescription actual_columns_) + HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_, ColumnsDescription actual_columns_) : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) , actual_columns(std::move(actual_columns_)) {} - TableFunctionEntry(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_, ColumnsDescription actual_columns_) + HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_, ColumnsDescription actual_columns_) : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) , storage_id(std::move(storage_id_)) @@ -94,7 +94,7 @@ class StorageDistributed final : public IStorage, WithContext std::string getName() const override { - return (additional_table_functions.empty() && !additional_filter) + return (segments.empty() && !additional_filter) ? "Distributed" : "Hybrid"; } @@ -179,8 +179,8 @@ class StorageDistributed final : public IStorage, WithContext /// Set additional filter for Hybrid engine void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } - /// Set additional table functions for Hybrid engine along with cached schema info - void setHybridLayout(ColumnsDescription base_segment_columns_, std::vector additional_table_functions_); + /// Set segment definitions for Hybrid engine along with cached schema info + void setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_); /// Getter methods for ClusterProxy::executeQuery StorageID getRemoteStorageID() const { return remote_storage; } @@ -326,8 +326,8 @@ class StorageDistributed final : public IStorage, WithContext /// Additional filter expression for Hybrid engine ASTPtr additional_filter; - /// Additional table functions for Hybrid engine - std::vector additional_table_functions; + /// Additional segments for Hybrid engine + std::vector segments; ColumnsDescription base_segment_columns; }; From 6ac3b3cdb2a0e10d677e7ae9e8ad946a3b4d0198 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 23:16:05 +0100 Subject: [PATCH 05/26] more renames2 --- src/Storages/StorageDistributed.cpp | 8 ++++---- src/Storages/StorageDistributed.h | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1be2a0c13798..8480b3057559 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1087,7 +1087,7 @@ void StorageDistributed::read( query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr, - additional_filter); + base_segment_predicate); Block block = *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. @@ -1105,7 +1105,7 @@ void StorageDistributed::read( { for (const auto & segment : segments) { - // Create a modified query info with the additional predicate + // Create a modified query info with the segment predicate SelectQueryInfo additional_query_info = query_info; auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, @@ -1132,7 +1132,7 @@ void StorageDistributed::read( modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr, - additional_filter); + base_segment_predicate); if (!segments.empty()) { @@ -2401,7 +2401,7 @@ void registerStorageHybrid(StorageFactory & factory) ASTPtr second_arg = engine_args[1]; validate_predicate(second_arg, 1); - distributed_storage->setAdditionalFilter(second_arg); + distributed_storage->setBaseSegmentPredicate(second_arg); // Parse additional table function pairs (if any) std::vector segment_definitions; diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 5a268cddfbb1..a2fbb2919c3b 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -94,7 +94,7 @@ class StorageDistributed final : public IStorage, WithContext std::string getName() const override { - return (segments.empty() && !additional_filter) + return (segments.empty() && !base_segment_predicate) ? "Distributed" : "Hybrid"; } @@ -176,8 +176,8 @@ class StorageDistributed final : public IStorage, WithContext size_t getShardCount() const; - /// Set additional filter for Hybrid engine - void setAdditionalFilter(ASTPtr filter) { additional_filter = std::move(filter); } + /// Set optional predicate applied to the base segment + void setBaseSegmentPredicate(ASTPtr predicate) { base_segment_predicate = std::move(predicate); } /// Set segment definitions for Hybrid engine along with cached schema info void setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_); @@ -324,7 +324,7 @@ class StorageDistributed final : public IStorage, WithContext bool is_remote_function; /// Additional filter expression for Hybrid engine - ASTPtr additional_filter; + ASTPtr base_segment_predicate; /// Additional segments for Hybrid engine std::vector segments; From abf4a3bb0ff48006b319d05869fde6e5fb204db7 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 11 Nov 2025 23:26:32 +0100 Subject: [PATCH 06/26] test for missing column --- .../0_stateless/03642_hybrid.reference | 1 + tests/queries/0_stateless/03642_hybrid.sql | 35 +++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/tests/queries/0_stateless/03642_hybrid.reference b/tests/queries/0_stateless/03642_hybrid.reference index fc5db02aa8cc..e2b4689b6595 100644 --- a/tests/queries/0_stateless/03642_hybrid.reference +++ b/tests/queries/0_stateless/03642_hybrid.reference @@ -74,6 +74,7 @@ Verify Hybrid skips segment with always false predicate on the second segment 0 1 2 +Hybrid raises when a segment is missing a column used by the base schema Prepare local MergeTree table for multi-segment tests Populate local table with sample data Create Hybrid table with three segment pairs diff --git a/tests/queries/0_stateless/03642_hybrid.sql b/tests/queries/0_stateless/03642_hybrid.sql index 78adac6da028..7a9e38b9a953 100644 --- a/tests/queries/0_stateless/03642_hybrid.sql +++ b/tests/queries/0_stateless/03642_hybrid.sql @@ -97,6 +97,41 @@ CREATE TABLE test_tiered_distributed_numbers_skip_second SELECT * FROM test_tiered_distributed_numbers_skip_second ORDER BY number; DROP TABLE IF EXISTS test_tiered_distributed_numbers_skip_second SYNC; +SELECT 'Hybrid raises when a segment is missing a column used by the base schema'; +DROP TABLE IF EXISTS test_hybrid_segment_full SYNC; +DROP TABLE IF EXISTS test_hybrid_segment_partial SYNC; +DROP TABLE IF EXISTS test_hybrid_missing_column SYNC; + +CREATE TABLE test_hybrid_segment_full +( + `id` UInt32, + `value` UInt32 +) +ENGINE = MergeTree() +ORDER BY id; + +CREATE TABLE test_hybrid_segment_partial +( + `id` UInt32 +) +ENGINE = MergeTree() +ORDER BY id; + +INSERT INTO test_hybrid_segment_full VALUES (1, 10), (2, 20); +INSERT INTO test_hybrid_segment_partial VALUES (3), (4); + +CREATE TABLE test_hybrid_missing_column ENGINE = Hybrid( + remote('localhost:9000', currentDatabase(), 'test_hybrid_segment_full'), id < 3, + remote('localhost:9000', currentDatabase(), 'test_hybrid_segment_partial'), id >= 3 +); + +SELECT id, value FROM test_hybrid_missing_column ORDER BY id SETTINGS enable_analyzer = 0; -- { serverError UNKNOWN_IDENTIFIER } +SELECT id, value FROM test_hybrid_missing_column ORDER BY id SETTINGS enable_analyzer = 1; -- { serverError UNKNOWN_IDENTIFIER } + +DROP TABLE IF EXISTS test_hybrid_missing_column SYNC; +DROP TABLE IF EXISTS test_hybrid_segment_partial SYNC; +DROP TABLE IF EXISTS test_hybrid_segment_full SYNC; + ----------------------------- SELECT 'Prepare local MergeTree table for multi-segment tests'; From 31101051e5ef27647672358a9cd7583f5f8214d9 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 19 Nov 2025 20:26:59 +0100 Subject: [PATCH 07/26] allow using tables from the current database implicitly --- src/Storages/StorageDistributed.cpp | 23 ++++++++++++-- .../04182_hybrid_unqualified_table.reference | 3 ++ .../04182_hybrid_unqualified_table.sql | 31 +++++++++++++++++++ 3 files changed, 54 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/04182_hybrid_unqualified_table.reference create mode 100644 tests/queries/0_stateless/04182_hybrid_unqualified_table.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8480b3057559..49938aea0c2e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2452,11 +2452,28 @@ void registerStorageHybrid(StorageFactory & factory) // Parse table identifier to get StorageID StorageID storage_id(table_identifier); - // Sanity check: ensure the table identifier is fully qualified (has database name) + // Fill database for unqualified identifiers using current database (or the target table database). if (storage_id.database_name.empty()) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Argument #{}: table identifier '{}' must be fully qualified (database.table)", i, ast_identifier->name()); + String default_database = local_context->getCurrentDatabase(); + if (default_database.empty()) + default_database = args.table_id.database_name; + + if (default_database.empty()) + { + throw Exception(ErrorCodes::UNKNOWN_DATABASE, + "Argument #{}: table identifier '{}' does not specify database and no default database is selected", + i, ast_identifier->name()); + } + + storage_id.database_name = default_database; + + // Update AST so the table definition stores a fully qualified name. + auto qualified_identifier = std::make_shared(storage_id.database_name, storage_id.table_name); + qualified_identifier->alias = ast_identifier->alias; + qualified_identifier->prefer_alias_to_column_name = ast_identifier->prefer_alias_to_column_name; + table_function_ast = qualified_identifier; + engine_args[i] = table_function_ast; } // Sanity check: verify the table exists diff --git a/tests/queries/0_stateless/04182_hybrid_unqualified_table.reference b/tests/queries/0_stateless/04182_hybrid_unqualified_table.reference new file mode 100644 index 000000000000..98bae7afae05 --- /dev/null +++ b/tests/queries/0_stateless/04182_hybrid_unqualified_table.reference @@ -0,0 +1,3 @@ +Hybrid allows unqualified local tables by default +3 +1 diff --git a/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql b/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql new file mode 100644 index 000000000000..8b51910bbe7e --- /dev/null +++ b/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql @@ -0,0 +1,31 @@ +SELECT 'Hybrid allows unqualified local tables by default'; + +DROP TABLE IF EXISTS test_hybrid_unqualified_segment SYNC; +DROP TABLE IF EXISTS test_hybrid_unqualified SYNC; + +CREATE TABLE test_hybrid_unqualified_segment +( + `number` UInt64 +) +ENGINE = MergeTree() +ORDER BY tuple(); + +INSERT INTO test_hybrid_unqualified_segment VALUES (10), (20); + +CREATE TABLE test_hybrid_unqualified +( + `number` UInt64 +) +ENGINE = Hybrid( + remote('localhost:9000', system.numbers), number = 0, + test_hybrid_unqualified_segment, number >= 10 +); + +SELECT count() FROM test_hybrid_unqualified; + +SELECT positionCaseInsensitive(engine_full, concat(currentDatabase(), '.test_hybrid_unqualified_segment')) > 0 +FROM system.tables +WHERE database = currentDatabase() AND name = 'test_hybrid_unqualified'; + +DROP TABLE IF EXISTS test_hybrid_unqualified SYNC; +DROP TABLE IF EXISTS test_hybrid_unqualified_segment SYNC; From bcb0f1f5b2e13070b2f3481a44e9481e843ee1c5 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 19 Nov 2025 21:04:56 +0100 Subject: [PATCH 08/26] Mark Hybrid experimental --- docs/en/engines/table-engines/special/hybrid.md | 8 ++++++++ src/Core/Settings.cpp | 3 +++ src/Core/SettingsChangesHistory.cpp | 1 + src/Databases/enableAllExperimentalSettings.cpp | 1 + src/Storages/StorageDistributed.cpp | 11 +++++++++-- tests/queries/0_stateless/03642_hybrid.sql | 8 +++++++- .../0_stateless/04182_hybrid_unqualified_table.sql | 2 ++ 7 files changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index 8e2819190961..cc4a29f2d3b0 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -22,6 +22,14 @@ Typical use cases include: By giving mutually exclusive predicates to the segments (for example, `date < watermark` and `date >= watermark`), you ensure that each row is read from exactly one source. +## Enable the engine + +The Hybrid engine is experimental. Enable it per session (or in the user profile) before creating tables: + +```sql +SET allow_experimental_hybrid_table = 1; +``` + ## Engine definition ```sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 5a7530b752b8..f2bbc7c9329f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6925,6 +6925,9 @@ Allow the obsolete Object data type Allows creation of tables with the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine. Possible values: - 0 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is disabled. - 1 — the [TimeSeries](../../engines/table-engines/integrations/time-series.md) table engine is enabled. +)", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_hybrid_table, false, R"( +Allows creation of tables with the [Hybrid](../../engines/table-engines/special/hybrid.md) table engine. )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b8f5f9bea73c..65f956349a31 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -109,6 +109,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_ytsaurus_table_engine", false, false, "New setting."}, {"allow_experimental_ytsaurus_table_function", false, false, "New setting."}, {"allow_experimental_ytsaurus_dictionary_source", false, false, "New setting."}, + {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, {"per_part_index_stats", false, false, "New setting."}, {"allow_experimental_iceberg_compaction", 0, 0, "New setting "}, {"delta_lake_snapshot_version", -1, -1, "New setting"}, diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index 705a27c0905e..215ed5ad9248 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -61,6 +61,7 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("allow_experimental_ytsaurus_table_engine", 1); context->setSetting("allow_experimental_ytsaurus_dictionary_source", 1); context->setSetting("allow_experimental_time_series_aggregate_functions", 1); + context->setSetting("allow_experimental_hybrid_table", 1); context->setSetting("allow_experimental_lightweight_update", 1); context->setSetting("allow_experimental_insert_into_iceberg", 1); context->setSetting("allow_experimental_iceberg_compaction", 1); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 49938aea0c2e..8e092008340a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -201,6 +201,7 @@ namespace Setting extern const SettingsUInt64 allow_experimental_parallel_reading_from_replicas; extern const SettingsBool prefer_global_in_and_join; extern const SettingsBool enable_global_with_statement; + extern const SettingsBool allow_experimental_hybrid_table; } namespace DistributedSetting @@ -234,6 +235,7 @@ namespace ErrorCodes extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int SUPPORT_IS_DISABLED; } namespace ActionLocks @@ -2308,8 +2310,6 @@ void registerStorageDistributed(StorageFactory & factory) void registerStorageHybrid(StorageFactory & factory) { - // Register Hybrid engine - // TODO: consider moving it to a separate file / subclass of StorageDistributed factory.registerStorage("Hybrid", [](const StorageFactory::Arguments & args) -> StoragePtr { ASTs & engine_args = args.engine_args; @@ -2323,6 +2323,13 @@ void registerStorageHybrid(StorageFactory & factory) if (!local_context) local_context = global_context; + if (args.mode <= LoadingStrictnessLevel::CREATE + && !local_context->getSettingsRef()[Setting::allow_experimental_hybrid_table]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental Hybrid table engine is not enabled (the setting 'allow_experimental_hybrid_table')"); + } + // Validate first argument - must be a table function ASTPtr first_arg = engine_args[0]; if (const auto * func = first_arg->as()) diff --git a/tests/queries/0_stateless/03642_hybrid.sql b/tests/queries/0_stateless/03642_hybrid.sql index 7a9e38b9a953..98bbd207cf21 100644 --- a/tests/queries/0_stateless/03642_hybrid.sql +++ b/tests/queries/0_stateless/03642_hybrid.sql @@ -1,4 +1,10 @@ --- Test Hybrid engine registration and basic validation +SELECT 'Hybrid creation requires allow_experimental_hybrid_table'; +SET allow_experimental_hybrid_table = 0; +CREATE TABLE test_hybrid_requires_setting (`dummy` UInt8) ENGINE = Hybrid(remote('localhost:9000'), 1); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE IF EXISTS test_hybrid_requires_setting SYNC; + +SET allow_experimental_hybrid_table = 1; + SELECT 'Check Hybrid engine is registered'; SELECT name FROM system.table_engines WHERE name = 'Hybrid'; diff --git a/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql b/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql index 8b51910bbe7e..672beea1afc3 100644 --- a/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql +++ b/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql @@ -1,3 +1,5 @@ +SET allow_experimental_hybrid_table = 1; + SELECT 'Hybrid allows unqualified local tables by default'; DROP TABLE IF EXISTS test_hybrid_unqualified_segment SYNC; From a13a1e58a6b03cc8317aed52f43beabd8b0bb3a5 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 19 Nov 2025 22:36:00 +0100 Subject: [PATCH 09/26] almost working draft --- src/Core/Settings.cpp | 3 + src/Core/SettingsChangesHistory.cpp | 2 + .../ClusterProxy/SelectStreamFactory.cpp | 49 +++- .../ClusterProxy/SelectStreamFactory.h | 6 +- src/Storages/StorageDistributed.cpp | 209 +++++++++++++++++- 5 files changed, 259 insertions(+), 10 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f2bbc7c9329f..1587d782c8b3 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6928,6 +6928,9 @@ Allows creation of tables with the [TimeSeries](../../engines/table-engines/inte )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_hybrid_table, false, R"( Allows creation of tables with the [Hybrid](../../engines/table-engines/special/hybrid.md) table engine. +)", EXPERIMENTAL) \ + DECLARE(Bool, hybrid_table_auto_cast_columns, false, R"( +Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 65f956349a31..27a9b22f8d01 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -54,6 +54,8 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."} + {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, + {"hybrid_table_auto_cast_columns", false, false, "New setting to automatically cast Hybrid table columns when segments disagree on types."} }); addSettingsChanges(settings_changes_history, "25.8", { diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 8259d57fa7f0..7b27fcd7f814 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -15,12 +15,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -60,6 +62,46 @@ namespace FailPoints namespace ClusterProxy { +namespace +{ +void applyHybridCastsToAST( + ASTPtr & node, + const ColumnsDescription * metadata_columns, + const NameSet * columns_to_cast) +{ + if (!metadata_columns || !columns_to_cast || columns_to_cast->empty() || !node) + return; + + if (auto * func = node->as(); func && func->name == "_CAST") + return; + + if (auto * identifier = node->as()) + { + String candidate = identifier->name(); + String short_name = candidate; + + auto dot_pos = candidate.rfind('.'); + if (dot_pos != String::npos && dot_pos + 1 < candidate.size()) + short_name = candidate.substr(dot_pos + 1); + + if (columns_to_cast->contains(short_name)) + { + if (auto expected_column_opt = metadata_columns->tryGetPhysical(short_name)) + { + auto cast_ast = addTypeConversionToAST(node->clone(), expected_column_opt->type->getName()); + const auto & alias = identifier->alias.empty() ? short_name : identifier->alias; + cast_ast->setAlias(alias); + node = cast_ast; + return; + } + } + } + + for (auto & child : node->children) + applyHybridCastsToAST(child, metadata_columns, columns_to_cast); +} +} + /// select query has database, table and table function names as AST pointers /// Creates a copy of query, changes database, table and table function names. ASTPtr rewriteSelectQuery( @@ -68,7 +110,9 @@ ASTPtr rewriteSelectQuery( const std::string & remote_database, const std::string & remote_table, ASTPtr table_function_ptr, - ASTPtr additional_filter) + ASTPtr additional_filter, + const NameSet * columns_to_cast, + const ColumnsDescription * metadata_columns) { auto modified_query_ast = query->clone(); @@ -122,6 +166,9 @@ ASTPtr rewriteSelectQuery( RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); } + + if (columns_to_cast && !columns_to_cast->empty() && metadata_columns) + applyHybridCastsToAST(modified_query_ast, metadata_columns, columns_to_cast); } /// To make local JOIN works, default database should be added to table names. diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 47646a80e1aa..ea2fc195fe11 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -3,10 +3,12 @@ #include #include #include +#include #include #include #include #include +#include #include namespace DB @@ -42,7 +44,9 @@ ASTPtr rewriteSelectQuery( const std::string & remote_database, const std::string & remote_table, ASTPtr table_function_ptr = nullptr, - ASTPtr additional_filter = nullptr); + ASTPtr additional_filter = nullptr, + const NameSet * columns_to_cast = nullptr, + const ColumnsDescription * metadata_columns = nullptr); using ColumnsDescriptionByShardNum = std::unordered_map; using AdditionalShardFilterGenerator = std::function; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8e092008340a..660c12378016 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -104,6 +104,7 @@ #include #include +#include #include #include @@ -117,7 +118,7 @@ #include #include #include - +#include #include #include @@ -169,6 +170,113 @@ void replaceCurrentDatabaseFunction(ASTPtr & ast, const ContextPtr & context) for (auto & child : ast->children) replaceCurrentDatabaseFunction(child, context); } + +NameSet collectColumnsToCast( + const ColumnsDescription & metadata_columns, + const ColumnsDescription & actual_columns) +{ + NameSet names; + if (metadata_columns.empty() || actual_columns.empty()) + return names; + + for (const auto & expected_column : metadata_columns.getAllPhysical()) + { + auto actual_column_opt = actual_columns.tryGetPhysical(expected_column.name); + if (!actual_column_opt) + continue; + + if (!actual_column_opt->type->equals(*expected_column.type)) + names.emplace(expected_column.name); + } + + return names; +} + +void applyHybridCastsToQueryTree( + QueryTreeNodePtr & query_tree, + const QueryTreeNodePtr & replacement_table_expression, + const ColumnsDescription & metadata_columns, + const NameSet & columns_to_cast, + const ColumnsDescription & actual_columns, + const ContextPtr & context) +{ + if (columns_to_cast.empty()) + return; + + class HybridCastVisitor : public InDepthQueryTreeVisitor + { + public: + HybridCastVisitor( + const QueryTreeNodePtr & table_expression_, + const ColumnsDescription & metadata_columns_, + const NameSet & columns_to_cast_, + const ColumnsDescription & actual_columns_, + ContextPtr context_) + : table_expression(table_expression_) + , metadata_columns(metadata_columns_) + , columns_to_cast(columns_to_cast_) + , actual_columns(actual_columns_) + , context(std::move(context_)) + {} + + bool shouldTraverseTopToBottom() const { return false; } + + static bool needChildVisit(QueryTreeNodePtr & /*parent*/, QueryTreeNodePtr & child) + { + auto child_type = child->getNodeType(); + return !(child_type == QueryTreeNodeType::QUERY || child_type == QueryTreeNodeType::UNION); + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source = column_node->getColumnSourceOrNull(); + if (column_source != table_expression) + return; + + const auto & column_name = column_node->getColumnName(); + if (!columns_to_cast.contains(column_name)) + return; + + auto expected_column_opt = metadata_columns.tryGetPhysical(column_name); + if (!expected_column_opt) + return; + + auto actual_column_opt = actual_columns.tryGetPhysical(column_name); + const auto & source_column = actual_column_opt ? *actual_column_opt : *expected_column_opt; + + auto column_clone = std::static_pointer_cast(column_node->clone()); + column_clone->setColumnType(source_column.type); + + auto cast_node = buildCastFunction(column_clone, expected_column_opt->type, context); + const auto & alias = node->getAlias(); + if (!alias.empty()) + cast_node->setAlias(alias); + else + cast_node->setAlias(column_name); + + node = cast_node; + } + + private: + QueryTreeNodePtr table_expression; + const ColumnsDescription & metadata_columns; + const NameSet & columns_to_cast; + const ColumnsDescription & actual_columns; + ContextPtr context; + }; + + HybridCastVisitor visitor( + replacement_table_expression, + metadata_columns, + columns_to_cast, + actual_columns, + context); + visitor.visit(query_tree); +} } namespace Setting @@ -202,6 +310,7 @@ namespace Setting extern const SettingsBool prefer_global_in_and_join; extern const SettingsBool enable_global_with_statement; extern const SettingsBool allow_experimental_hybrid_table; + extern const SettingsBool hybrid_table_auto_cast_columns; } namespace DistributedSetting @@ -948,7 +1057,10 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, const ASTPtr & remote_table_function, - const ASTPtr & additional_filter = nullptr) + const ASTPtr & additional_filter = nullptr, + const NameSet * columns_to_cast = nullptr, + const ColumnsDescription * metadata_columns = nullptr, + const ColumnsDescription * actual_columns_for_cast = nullptr) { auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -1027,6 +1139,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, QueryAnalysisPass(replacement_table_expression).run(filter, context); } + auto replacement_table_expression_ptr = replacement_table_expression; auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); // Apply additional filter if provided @@ -1038,6 +1151,18 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } + if (columns_to_cast && !columns_to_cast->empty() && metadata_columns) + { + const ColumnsDescription * source_columns = actual_columns_for_cast ? actual_columns_for_cast : metadata_columns; + applyHybridCastsToQueryTree( + query_tree_to_modify, + replacement_table_expression_ptr, + *metadata_columns, + *columns_to_cast, + *source_columns, + query_context); + } + ReplaseAliasColumnsVisitor replase_alias_columns_visitor; replase_alias_columns_visitor.visit(query_tree_to_modify); @@ -1078,6 +1203,57 @@ void StorageDistributed::read( std::vector additional_query_infos; const auto & settings = local_context->getSettingsRef(); + auto metadata_ptr = getInMemoryMetadataPtr(); + const auto & metadata_columns = metadata_ptr->getColumns(); + + const bool enable_hybrid_column_casts = settings[Setting::hybrid_table_auto_cast_columns] + && (!base_segment_columns.empty() || !segments.empty()); + + NameSet columns_to_cast_names; + if (enable_hybrid_column_casts) + { + if (!base_segment_columns.empty()) + { + auto names = collectColumnsToCast(metadata_columns, base_segment_columns); + columns_to_cast_names.insert(names.begin(), names.end()); + } + + for (const auto & segment : segments) + { + auto names = collectColumnsToCast(metadata_columns, segment.actual_columns); + columns_to_cast_names.insert(names.begin(), names.end()); + } + } + + const bool need_hybrid_casts = !columns_to_cast_names.empty(); + const auto * columns_to_cast_ptr = need_hybrid_casts ? &columns_to_cast_names : nullptr; + const auto * base_actual_columns_for_cast = !base_segment_columns.empty() ? &base_segment_columns : &metadata_columns; + + auto describe_segment_target = [&](const HybridSegment & segment) -> String + { + if (segment.storage_id) + return segment.storage_id->getNameForLogs(); + if (segment.table_function_ast) + return segment.table_function_ast->formatForLogging(); + return String{""}; + }; + + const bool log_hybrid_casts = need_hybrid_casts && log->is(Poco::Message::PRIO_TRACE); + auto log_rewritten_query = [&](const String & target, const ASTPtr & ast) + { + if (!log_hybrid_casts || !ast) + return; + + LOG_TRACE(log, "rewriteSelectQuery (target: {}) -> {}", target, ast->formatForLogging()); + }; + + String base_target; + if (remote_table_function_ptr) + base_target = remote_table_function_ptr->formatForLogging(); + else if (!remote_database.empty()) + base_target = remote_database + "." + remote_table; + else + base_target = remote_table; if (settings[Setting::allow_experimental_analyzer]) { @@ -1089,7 +1265,10 @@ void StorageDistributed::read( query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr, - base_segment_predicate); + base_segment_predicate, + columns_to_cast_ptr, + &metadata_columns, + base_actual_columns_for_cast); Block block = *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. @@ -1102,6 +1281,7 @@ void StorageDistributed::read( modified_query_info.query = queryNodeToDistributedSelectQuery(query_tree_distributed); modified_query_info.query_tree = std::move(query_tree_distributed); + log_rewritten_query(base_target, modified_query_info.query); if (!segments.empty()) { @@ -1109,15 +1289,20 @@ void StorageDistributed::read( { // Create a modified query info with the segment predicate SelectQueryInfo additional_query_info = query_info; + const auto * segment_actual_columns = !segment.actual_columns.empty() ? &segment.actual_columns : &metadata_columns; auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, segment.storage_id ? *segment.storage_id : StorageID::createEmpty(), segment.storage_id ? nullptr : segment.table_function_ast, - segment.predicate_ast); + segment.predicate_ast, + columns_to_cast_ptr, + &metadata_columns, + segment_actual_columns); additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); + log_rewritten_query(describe_segment_target(segment), additional_query_info.query); additional_query_infos.push_back(std::move(additional_query_info)); } @@ -1134,7 +1319,10 @@ void StorageDistributed::read( modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr, - base_segment_predicate); + base_segment_predicate, + columns_to_cast_ptr, + &metadata_columns); + log_rewritten_query(base_target, modified_query_info.query); if (!segments.empty()) { @@ -1148,16 +1336,21 @@ void StorageDistributed::read( local_context, additional_query_info.query, segment.storage_id->database_name, segment.storage_id->table_name, nullptr, - segment.predicate_ast); + segment.predicate_ast, + columns_to_cast_ptr, + &metadata_columns); } else { additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", segment.table_function_ast, - segment.predicate_ast); + segment.predicate_ast, + columns_to_cast_ptr, + &metadata_columns); } + log_rewritten_query(describe_segment_target(segment), additional_query_info.query); additional_query_infos.push_back(std::move(additional_query_info)); } } @@ -1186,7 +1379,7 @@ void StorageDistributed::read( processed_stage); auto shard_filter_generator = ClusterProxy::getShardFilterGeneratorForCustomKey( - *modified_query_info.getCluster(), local_context, getInMemoryMetadataPtr()->columns); + *modified_query_info.getCluster(), local_context, metadata_ptr->columns); ClusterProxy::executeQuery( query_plan, From 0e0e76bfcb29d66a56139cb7ecb8e9f046dea048 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 20 Nov 2025 00:26:14 +0100 Subject: [PATCH 10/26] better --- .../engines/table-engines/special/hybrid.md | 4 + .../ClusterProxy/SelectStreamFactory.cpp | 49 +---------- .../ClusterProxy/SelectStreamFactory.h | 6 +- src/Storages/StorageDistributed.cpp | 15 ++-- ...ybrid.reference => 03643_hybrid.reference} | 1 + .../{03642_hybrid.sql => 03643_hybrid.sql} | 2 +- .../03644_hybrid_auto_cast.reference | 13 +++ .../0_stateless/03644_hybrid_auto_cast.sql | 85 +++++++++++++++++++ ... 03644_hybrid_unqualified_table.reference} | 0 ...sql => 03644_hybrid_unqualified_table.sql} | 0 10 files changed, 112 insertions(+), 63 deletions(-) rename tests/queries/0_stateless/{03642_hybrid.reference => 03643_hybrid.reference} (99%) rename tests/queries/0_stateless/{03642_hybrid.sql => 03643_hybrid.sql} (99%) create mode 100644 tests/queries/0_stateless/03644_hybrid_auto_cast.reference create mode 100644 tests/queries/0_stateless/03644_hybrid_auto_cast.sql rename tests/queries/0_stateless/{04182_hybrid_unqualified_table.reference => 03644_hybrid_unqualified_table.reference} (100%) rename tests/queries/0_stateless/{04182_hybrid_unqualified_table.sql => 03644_hybrid_unqualified_table.sql} (100%) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index cc4a29f2d3b0..0d456e302438 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -30,6 +30,10 @@ The Hybrid engine is experimental. Enable it per session (or in the user profile SET allow_experimental_hybrid_table = 1; ``` +### Automatic Type Alignment + +Hybrid segments can evolve independently, so the same logical column may use different physical types. When you set the experimental `hybrid_table_auto_cast_columns = 1` (requires `allow_experimental_analyzer = 1`), the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. This prevents header mismatches without having to edit each query. + ## Engine definition ```sql diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 7b27fcd7f814..8259d57fa7f0 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -15,14 +15,12 @@ #include #include #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -62,46 +60,6 @@ namespace FailPoints namespace ClusterProxy { -namespace -{ -void applyHybridCastsToAST( - ASTPtr & node, - const ColumnsDescription * metadata_columns, - const NameSet * columns_to_cast) -{ - if (!metadata_columns || !columns_to_cast || columns_to_cast->empty() || !node) - return; - - if (auto * func = node->as(); func && func->name == "_CAST") - return; - - if (auto * identifier = node->as()) - { - String candidate = identifier->name(); - String short_name = candidate; - - auto dot_pos = candidate.rfind('.'); - if (dot_pos != String::npos && dot_pos + 1 < candidate.size()) - short_name = candidate.substr(dot_pos + 1); - - if (columns_to_cast->contains(short_name)) - { - if (auto expected_column_opt = metadata_columns->tryGetPhysical(short_name)) - { - auto cast_ast = addTypeConversionToAST(node->clone(), expected_column_opt->type->getName()); - const auto & alias = identifier->alias.empty() ? short_name : identifier->alias; - cast_ast->setAlias(alias); - node = cast_ast; - return; - } - } - } - - for (auto & child : node->children) - applyHybridCastsToAST(child, metadata_columns, columns_to_cast); -} -} - /// select query has database, table and table function names as AST pointers /// Creates a copy of query, changes database, table and table function names. ASTPtr rewriteSelectQuery( @@ -110,9 +68,7 @@ ASTPtr rewriteSelectQuery( const std::string & remote_database, const std::string & remote_table, ASTPtr table_function_ptr, - ASTPtr additional_filter, - const NameSet * columns_to_cast, - const ColumnsDescription * metadata_columns) + ASTPtr additional_filter) { auto modified_query_ast = query->clone(); @@ -166,9 +122,6 @@ ASTPtr rewriteSelectQuery( RestoreQualifiedNamesVisitor(data).visit(modified_query_ast); } - - if (columns_to_cast && !columns_to_cast->empty() && metadata_columns) - applyHybridCastsToAST(modified_query_ast, metadata_columns, columns_to_cast); } /// To make local JOIN works, default database should be added to table names. diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index ea2fc195fe11..47646a80e1aa 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -3,12 +3,10 @@ #include #include #include -#include #include #include #include #include -#include #include namespace DB @@ -44,9 +42,7 @@ ASTPtr rewriteSelectQuery( const std::string & remote_database, const std::string & remote_table, ASTPtr table_function_ptr = nullptr, - ASTPtr additional_filter = nullptr, - const NameSet * columns_to_cast = nullptr, - const ColumnsDescription * metadata_columns = nullptr); + ASTPtr additional_filter = nullptr); using ColumnsDescriptionByShardNum = std::unordered_map; using AdditionalShardFilterGenerator = std::function; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 660c12378016..372bae41bc54 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1229,6 +1229,9 @@ void StorageDistributed::read( const auto * columns_to_cast_ptr = need_hybrid_casts ? &columns_to_cast_names : nullptr; const auto * base_actual_columns_for_cast = !base_segment_columns.empty() ? &base_segment_columns : &metadata_columns; + if (need_hybrid_casts && !settings[Setting::allow_experimental_analyzer]) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting 'hybrid_table_auto_cast_columns' is supported only with allow_experimental_analyzer=1"); + auto describe_segment_target = [&](const HybridSegment & segment) -> String { if (segment.storage_id) @@ -1319,9 +1322,7 @@ void StorageDistributed::read( modified_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, modified_query_info.query, remote_database, remote_table, remote_table_function_ptr, - base_segment_predicate, - columns_to_cast_ptr, - &metadata_columns); + base_segment_predicate); log_rewritten_query(base_target, modified_query_info.query); if (!segments.empty()) @@ -1336,18 +1337,14 @@ void StorageDistributed::read( local_context, additional_query_info.query, segment.storage_id->database_name, segment.storage_id->table_name, nullptr, - segment.predicate_ast, - columns_to_cast_ptr, - &metadata_columns); + segment.predicate_ast); } else { additional_query_info.query = ClusterProxy::rewriteSelectQuery( local_context, additional_query_info.query, "", "", segment.table_function_ast, - segment.predicate_ast, - columns_to_cast_ptr, - &metadata_columns); + segment.predicate_ast); } log_rewritten_query(describe_segment_target(segment), additional_query_info.query); diff --git a/tests/queries/0_stateless/03642_hybrid.reference b/tests/queries/0_stateless/03643_hybrid.reference similarity index 99% rename from tests/queries/0_stateless/03642_hybrid.reference rename to tests/queries/0_stateless/03643_hybrid.reference index e2b4689b6595..aef70a61a949 100644 --- a/tests/queries/0_stateless/03642_hybrid.reference +++ b/tests/queries/0_stateless/03643_hybrid.reference @@ -1,3 +1,4 @@ +Hybrid creation requires allow_experimental_hybrid_table Check Hybrid engine is registered Hybrid Ensure no leftovers before validation checks diff --git a/tests/queries/0_stateless/03642_hybrid.sql b/tests/queries/0_stateless/03643_hybrid.sql similarity index 99% rename from tests/queries/0_stateless/03642_hybrid.sql rename to tests/queries/0_stateless/03643_hybrid.sql index 98bbd207cf21..86ef79259250 100644 --- a/tests/queries/0_stateless/03642_hybrid.sql +++ b/tests/queries/0_stateless/03643_hybrid.sql @@ -346,7 +346,7 @@ DROP TABLE IF EXISTS test_tiered_watermark SYNC; DROP TABLE IF EXISTS test_tiered_watermark_after SYNC; DROP TABLE IF EXISTS test_tiered_watermark_before SYNC; --- TODO: +-- TODO: - addressed by 03644_hybrid_auto_cast.sql -- Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Conversion from AggregateFunction(sum, Decimal(38, 0)) to AggregateFunction(sum, UInt32) is not supported: while converting source column `sum(__table1.value)` to destination column `sum(__table1.value)`. (CANNOT_CONVERT_TYPE) -- SELECT sum(value) FROM test_tiered_watermark; diff --git a/tests/queries/0_stateless/03644_hybrid_auto_cast.reference b/tests/queries/0_stateless/03644_hybrid_auto_cast.reference new file mode 100644 index 000000000000..869ac32216b6 --- /dev/null +++ b/tests/queries/0_stateless/03644_hybrid_auto_cast.reference @@ -0,0 +1,13 @@ +hybrid_table_auto_cast_columns = 0, enable_analyzer = 1 (headers mismatch) +hybrid_table_auto_cast_columns = 0, enable_analyzer = 0 (headers mismatch) +1 +hybrid_table_auto_cast_columns = 0, enable_analyzer = 1 manual cast +600 +1 +hybrid_table_auto_cast_columns = 0, enable_analyzer = 0 manual cast +600 +1 +hybrid_table_auto_cast_columns = 1, enable_analyzer = 1 +600 +1 +hybrid_table_auto_cast_columns = 1, enable_analyzer = 0 (analizer required) diff --git a/tests/queries/0_stateless/03644_hybrid_auto_cast.sql b/tests/queries/0_stateless/03644_hybrid_auto_cast.sql new file mode 100644 index 000000000000..eb5447572418 --- /dev/null +++ b/tests/queries/0_stateless/03644_hybrid_auto_cast.sql @@ -0,0 +1,85 @@ +SET allow_experimental_hybrid_table = 1, + prefer_localhost_replica = 0; + +DROP TABLE IF EXISTS test_tiered_watermark_after; +DROP TABLE IF EXISTS test_tiered_watermark_before; +DROP TABLE IF EXISTS test_tiered_watermark; + +CREATE TABLE test_tiered_watermark_after +( + `id` UInt32, + `name` String, + `date` Date, + `value` UInt64, + `categories` Array(UInt32) +) +ENGINE = MergeTree() +ORDER BY id; + +CREATE TABLE test_tiered_watermark_before +( + `id` Int32, + `name` Nullable(String), + `date` Date, + `value` Decimal128(0), + `categories` Array(Int64) +) +ENGINE = MergeTree() +ORDER BY id; + +INSERT INTO test_tiered_watermark_after VALUES + (11, 'Alice', '2025-08-15', 100, [100, 10]), + (12, 'Bob', '2025-08-20', 200, [200, 20]), + (13, 'Charlie', '2025-08-25', 300, [300, 30]), + (14, 'David', '2025-09-05', 400, [400, 40]), + (15, 'Eve', '2025-09-10', 500, [500, 50]), + (16, 'Frank', '2025-09-15', 600, [600, 60]); + +INSERT INTO test_tiered_watermark_before VALUES + (21, 'Alice', '2025-08-15', 100, [100, 10]), + (22, 'Bob', '2025-08-20', 200, [200, 20]), + (23, 'Charlie', '2025-08-25', 300, [300, 30]), + (24, 'David', '2025-09-05', 400, [400, 40]), + (25, 'Eve', '2025-09-10', 500, [500, 50]), + (26, 'Frank', '2025-09-15', 600, [600, 60]); + +CREATE TABLE test_tiered_watermark +ENGINE = Hybrid( + remote('127.0.0.1:9000', currentDatabase(), 'test_tiered_watermark_after'), + date >= '2025-09-01', + remote('127.0.0.1:9000', currentDatabase(), 'test_tiered_watermark_before'), + date < '2025-09-01' +); + +-- the problem +SELECT 'hybrid_table_auto_cast_columns = 0, enable_analyzer = 1 (headers mismatch)'; +SET hybrid_table_auto_cast_columns = 0, enable_analyzer = 1; +SELECT max(value) FROM test_tiered_watermark; -- { serverError CANNOT_CONVERT_TYPE } +SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_tiered_watermark; -- { serverError THERE_IS_NO_COLUMN } + +SELECT 'hybrid_table_auto_cast_columns = 0, enable_analyzer = 0 (headers mismatch)'; +SET hybrid_table_auto_cast_columns = 0, enable_analyzer = 0; +SELECT max(value) FROM test_tiered_watermark; -- { serverError CANNOT_CONVERT_TYPE } +SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_tiered_watermark; -- works w/o analyzer + +-- workaround - explicit cast +SELECT 'hybrid_table_auto_cast_columns = 0, enable_analyzer = 1 manual cast'; +SET hybrid_table_auto_cast_columns = 0, enable_analyzer = 1; +SELECT max(value::UInt32) FROM test_tiered_watermark; +SELECT sum(if(arrayExists(x -> (x IN (10)), categories::Array(UInt32)), 1, 0)) AS x FROM test_tiered_watermark; + +SELECT 'hybrid_table_auto_cast_columns = 0, enable_analyzer = 0 manual cast'; +SET hybrid_table_auto_cast_columns = 0, enable_analyzer = 0; +SELECT max(value::UInt32) FROM test_tiered_watermark; +SELECT sum(if(arrayExists(x -> (x IN (10)), categories::Array(UInt32)), 1, 0)) AS x FROM test_tiered_watermark; + +-- feature to add casts automatically +SELECT 'hybrid_table_auto_cast_columns = 1, enable_analyzer = 1'; +SET hybrid_table_auto_cast_columns = 1, enable_analyzer = 1; +SELECT max(value) FROM test_tiered_watermark; +SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_tiered_watermark; + +SELECT 'hybrid_table_auto_cast_columns = 1, enable_analyzer = 0 (analizer required)'; +SET hybrid_table_auto_cast_columns = 1, enable_analyzer = 0; +SELECT max(value) FROM test_tiered_watermark; -- { serverError NOT_IMPLEMENTED } +SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_tiered_watermark; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/04182_hybrid_unqualified_table.reference b/tests/queries/0_stateless/03644_hybrid_unqualified_table.reference similarity index 100% rename from tests/queries/0_stateless/04182_hybrid_unqualified_table.reference rename to tests/queries/0_stateless/03644_hybrid_unqualified_table.reference diff --git a/tests/queries/0_stateless/04182_hybrid_unqualified_table.sql b/tests/queries/0_stateless/03644_hybrid_unqualified_table.sql similarity index 100% rename from tests/queries/0_stateless/04182_hybrid_unqualified_table.sql rename to tests/queries/0_stateless/03644_hybrid_unqualified_table.sql From cac851f6444060569c087927f76821e64a595da4 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 20 Nov 2025 00:47:52 +0100 Subject: [PATCH 11/26] fix merge issue --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 27a9b22f8d01..8abc8579a01a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -53,7 +53,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, - {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."} {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, {"hybrid_table_auto_cast_columns", false, false, "New setting to automatically cast Hybrid table columns when segments disagree on types."} }); From 2f1e11ddf8b5b02f25e694608aa8dd3d3a593273 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 20 Nov 2025 00:51:19 +0100 Subject: [PATCH 12/26] misplaced setting --- src/Core/SettingsChangesHistory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8abc8579a01a..67183ff24300 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -54,7 +54,8 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, - {"hybrid_table_auto_cast_columns", false, false, "New setting to automatically cast Hybrid table columns when segments disagree on types."} + {"hybrid_table_auto_cast_columns", false, false, "New setting to automatically cast Hybrid table columns when segments disagree on types."}, + {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."} }); addSettingsChanges(settings_changes_history, "25.8", { @@ -110,7 +111,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_ytsaurus_table_engine", false, false, "New setting."}, {"allow_experimental_ytsaurus_table_function", false, false, "New setting."}, {"allow_experimental_ytsaurus_dictionary_source", false, false, "New setting."}, - {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, {"per_part_index_stats", false, false, "New setting."}, {"allow_experimental_iceberg_compaction", 0, 0, "New setting "}, {"delta_lake_snapshot_version", -1, -1, "New setting"}, From 6edfd3ac38c159ff35369dd5a84bd51d0a026509 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 12:36:06 +0100 Subject: [PATCH 13/26] better explantation for cached header --- docs/en/engines/table-engines/special/hybrid.md | 2 ++ src/Core/Settings.cpp | 2 +- src/Storages/StorageDistributed.cpp | 3 +++ src/Storages/StorageDistributed.h | 4 +++- 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index 0d456e302438..0afae098544c 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -34,6 +34,8 @@ SET allow_experimental_hybrid_table = 1; Hybrid segments can evolve independently, so the same logical column may use different physical types. When you set the experimental `hybrid_table_auto_cast_columns = 1` (requires `allow_experimental_analyzer = 1`), the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. This prevents header mismatches without having to edit each query. +Segment schemas are cached when you create or attach a Hybrid table. If you alter a segment later (for example change a column type), refresh the Hybrid table (detach/attach or recreate it) so the cached headers stay in sync with the new schema; otherwise the auto-cast feature may miss the change and queries can still fail with header/type errors. + ## Engine definition ```sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 1587d782c8b3..9f68b40dda50 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6930,7 +6930,7 @@ Allows creation of tables with the [TimeSeries](../../engines/table-engines/inte Allows creation of tables with the [Hybrid](../../engines/table-engines/special/hybrid.md) table engine. )", EXPERIMENTAL) \ DECLARE(Bool, hybrid_table_auto_cast_columns, false, R"( -Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. +Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. Segment schemas are cached when the Hybrid table is created or attached; if a segment schema changes later, detach/attach or recreate the Hybrid table so the cached headers stay in sync. )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 372bae41bc54..4a34717e5265 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2384,6 +2384,9 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const void StorageDistributed::setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_) { + /// Hybrid keeps a snapshot of segment schemas captured during CREATE/ATTACH to avoid re-reading (possibly remote) headers on every query. + /// A TTL-based cache was considered but deemed overkill for this experimental feature. + /// Subsequent segment DDL changes are not auto-detected; reattach/recreate the Hybrid table to refresh. base_segment_columns = std::move(base_segment_columns_); segments = std::move(segments_); log = getLogger("Hybrid (" + getStorageID().table_name + ")"); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index a2fbb2919c3b..f9dcfe9648ba 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -50,7 +50,9 @@ class StorageDistributed final : public IStorage, WithContext friend class StorageSystemDistributionQueue; public: - /// Structure to hold table function AST, predicate, optional StorageID, and cached physical columns for the segment + /// Structure to hold table function AST, predicate, optional StorageID, and cached physical columns for the segment. + /// Cached columns let us detect schema mismatches and enable features like hybrid_table_auto_cast_columns without + /// re-fetching remote headers on every query. struct HybridSegment { ASTPtr table_function_ast; From 78a9ba11a6ff3f43b49709a19f732c0e5e50adc7 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 12:40:43 +0100 Subject: [PATCH 14/26] hybrid_table_auto_cast_columns=true --- docs/en/engines/table-engines/special/hybrid.md | 2 +- src/Core/Settings.cpp | 4 ++-- src/Core/SettingsChangesHistory.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index 0afae098544c..f7ed0de2172b 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -32,7 +32,7 @@ SET allow_experimental_hybrid_table = 1; ### Automatic Type Alignment -Hybrid segments can evolve independently, so the same logical column may use different physical types. When you set the experimental `hybrid_table_auto_cast_columns = 1` (requires `allow_experimental_analyzer = 1`), the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. This prevents header mismatches without having to edit each query. +Hybrid segments can evolve independently, so the same logical column may use different physical types. With the experimental `hybrid_table_auto_cast_columns = 1` (default, requires `allow_experimental_analyzer = 1`), the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. This prevents header mismatches without having to edit each query, and you can opt out by setting the flag to `0` if it causes issues in your setup. Segment schemas are cached when you create or attach a Hybrid table. If you alter a segment later (for example change a column type), refresh the Hybrid table (detach/attach or recreate it) so the cached headers stay in sync with the new schema; otherwise the auto-cast feature may miss the change and queries can still fail with header/type errors. diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9f68b40dda50..80e3b119f215 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6929,8 +6929,8 @@ Allows creation of tables with the [TimeSeries](../../engines/table-engines/inte DECLARE(Bool, allow_experimental_hybrid_table, false, R"( Allows creation of tables with the [Hybrid](../../engines/table-engines/special/hybrid.md) table engine. )", EXPERIMENTAL) \ - DECLARE(Bool, hybrid_table_auto_cast_columns, false, R"( -Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. Segment schemas are cached when the Hybrid table is created or attached; if a segment schema changes later, detach/attach or recreate the Hybrid table so the cached headers stay in sync. + DECLARE(Bool, hybrid_table_auto_cast_columns, true, R"( +Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. Enabled by default; disable it if it causes issues. Segment schemas are cached when the Hybrid table is created or attached; if a segment schema changes later, detach/attach or recreate the Hybrid table so the cached headers stay in sync. )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 67183ff24300..3e3f5e3f7608 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -54,7 +54,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, - {"hybrid_table_auto_cast_columns", false, false, "New setting to automatically cast Hybrid table columns when segments disagree on types."}, + {"hybrid_table_auto_cast_columns", true, true, "New setting to automatically cast Hybrid table columns when segments disagree on types. Default enabled."}, {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."} }); addSettingsChanges(settings_changes_history, "25.8", From d48f1bf0ecc2c230707d6276d55017e17806e88c Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 12:45:37 +0100 Subject: [PATCH 15/26] Tightened buildQueryTreeDistributed to avoid unused default params and make the expected non-null columns explicit --- src/Storages/StorageDistributed.cpp | 23 +++++++++++------------ 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4a34717e5265..e1e2fcfe0678 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1057,10 +1057,10 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, const ASTPtr & remote_table_function, - const ASTPtr & additional_filter = nullptr, - const NameSet * columns_to_cast = nullptr, - const ColumnsDescription * metadata_columns = nullptr, - const ColumnsDescription * actual_columns_for_cast = nullptr) + const ASTPtr & additional_filter, + const NameSet * columns_to_cast, + const ColumnsDescription & metadata_columns, + const ColumnsDescription & actual_columns_for_cast) { auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -1151,15 +1151,14 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } - if (columns_to_cast && !columns_to_cast->empty() && metadata_columns) + if (columns_to_cast && !columns_to_cast->empty()) { - const ColumnsDescription * source_columns = actual_columns_for_cast ? actual_columns_for_cast : metadata_columns; applyHybridCastsToQueryTree( query_tree_to_modify, replacement_table_expression_ptr, - *metadata_columns, + metadata_columns, *columns_to_cast, - *source_columns, + actual_columns_for_cast, query_context); } @@ -1270,8 +1269,8 @@ void StorageDistributed::read( remote_table_function_ptr, base_segment_predicate, columns_to_cast_ptr, - &metadata_columns, - base_actual_columns_for_cast); + metadata_columns, + *base_actual_columns_for_cast); Block block = *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. @@ -1300,8 +1299,8 @@ void StorageDistributed::read( segment.storage_id ? nullptr : segment.table_function_ast, segment.predicate_ast, columns_to_cast_ptr, - &metadata_columns, - segment_actual_columns); + metadata_columns, + *segment_actual_columns); additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); From df02469c6d77a513777022a89414f2a0dcc71673 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 12:47:37 +0100 Subject: [PATCH 16/26] redundant move/copy --- src/Storages/StorageDistributed.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e1e2fcfe0678..f50a7f472ce2 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1127,7 +1127,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, replacement_table_expression->setAlias(query_info.table_expression->getAlias()); - QueryTreeNodePtr filter; if (additional_filter) @@ -1139,8 +1138,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, QueryAnalysisPass(replacement_table_expression).run(filter, context); } - auto replacement_table_expression_ptr = replacement_table_expression; - auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); + auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, replacement_table_expression); // Apply additional filter if provided if (filter) @@ -1155,7 +1153,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, { applyHybridCastsToQueryTree( query_tree_to_modify, - replacement_table_expression_ptr, + replacement_table_expression, metadata_columns, *columns_to_cast, actual_columns_for_cast, From c152623f3a060538dc9358f14e1a1ea30ad3af43 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 15:09:09 +0100 Subject: [PATCH 17/26] better comment for HybridCastVisitor --- src/Storages/StorageDistributed.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f50a7f472ce2..9a66103da2db 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -203,6 +203,21 @@ void applyHybridCastsToQueryTree( if (columns_to_cast.empty()) return; + // Visitor replaces all usages of the column with CAST(column, type) in the query tree. + // + // It normalizes headers coming from different segments when table structure in some segments + // differs from the Hybrid table definition. For example column X is UInt32 in the Hybrid table, + // but Int64 in an additional segment. + // + // Without these casts ConvertingActions may fail to reconcile mismatched headers when casts are impossible + // (e.g. AggregateFunction states carry hashed data tied to their argument type and cannot be recast), for example: + // "Conversion from AggregateFunction(uniq, Decimal(38, 0)) to AggregateFunction(uniq, UInt64) is not supported" + // (CANNOT_CONVERT_TYPE). + // + // Per-segment casts are not reliable because WithMergeState strips aliases, so merged pipelines + // from different segments would return different headers (with or without CAST), leading to errors + // like "Cannot find column `max(value)` in source stream, there are only columns: [max(_CAST(value, 'UInt64'))]" + // (THERE_IS_NO_COLUMN). class HybridCastVisitor : public InDepthQueryTreeVisitor { public: From bb3a5f205e042087e47526ae210c737d23907807 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 15:41:37 +0100 Subject: [PATCH 18/26] better signature, avoid pointer --- src/Storages/StorageDistributed.cpp | 25 ++++++++++++------------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9a66103da2db..26064c85f02a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -228,9 +228,9 @@ void applyHybridCastsToQueryTree( const ColumnsDescription & actual_columns_, ContextPtr context_) : table_expression(table_expression_) - , metadata_columns(metadata_columns_) + , hybrid_schema(metadata_columns_) , columns_to_cast(columns_to_cast_) - , actual_columns(actual_columns_) + , segment_columns(actual_columns_) , context(std::move(context_)) {} @@ -256,11 +256,11 @@ void applyHybridCastsToQueryTree( if (!columns_to_cast.contains(column_name)) return; - auto expected_column_opt = metadata_columns.tryGetPhysical(column_name); + auto expected_column_opt = hybrid_schema.tryGetPhysical(column_name); if (!expected_column_opt) return; - auto actual_column_opt = actual_columns.tryGetPhysical(column_name); + auto actual_column_opt = segment_columns.tryGetPhysical(column_name); const auto & source_column = actual_column_opt ? *actual_column_opt : *expected_column_opt; auto column_clone = std::static_pointer_cast(column_node->clone()); @@ -278,9 +278,9 @@ void applyHybridCastsToQueryTree( private: QueryTreeNodePtr table_expression; - const ColumnsDescription & metadata_columns; + const ColumnsDescription & hybrid_schema; const NameSet & columns_to_cast; - const ColumnsDescription & actual_columns; + const ColumnsDescription & segment_columns; ContextPtr context; }; @@ -1073,7 +1073,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageID & remote_storage_id, const ASTPtr & remote_table_function, const ASTPtr & additional_filter, - const NameSet * columns_to_cast, + const NameSet & columns_to_cast, const ColumnsDescription & metadata_columns, const ColumnsDescription & actual_columns_for_cast) { @@ -1164,13 +1164,13 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } - if (columns_to_cast && !columns_to_cast->empty()) + if (!columns_to_cast.empty()) { applyHybridCastsToQueryTree( query_tree_to_modify, replacement_table_expression, metadata_columns, - *columns_to_cast, + columns_to_cast, actual_columns_for_cast, query_context); } @@ -1221,7 +1221,7 @@ void StorageDistributed::read( const bool enable_hybrid_column_casts = settings[Setting::hybrid_table_auto_cast_columns] && (!base_segment_columns.empty() || !segments.empty()); - NameSet columns_to_cast_names; + NameSet columns_to_cast_names; /// Empty when no casts are needed if (enable_hybrid_column_casts) { if (!base_segment_columns.empty()) @@ -1238,7 +1238,6 @@ void StorageDistributed::read( } const bool need_hybrid_casts = !columns_to_cast_names.empty(); - const auto * columns_to_cast_ptr = need_hybrid_casts ? &columns_to_cast_names : nullptr; const auto * base_actual_columns_for_cast = !base_segment_columns.empty() ? &base_segment_columns : &metadata_columns; if (need_hybrid_casts && !settings[Setting::allow_experimental_analyzer]) @@ -1281,7 +1280,7 @@ void StorageDistributed::read( remote_storage_id, remote_table_function_ptr, base_segment_predicate, - columns_to_cast_ptr, + columns_to_cast_names, metadata_columns, *base_actual_columns_for_cast); Block block = *InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); @@ -1311,7 +1310,7 @@ void StorageDistributed::read( segment.storage_id ? *segment.storage_id : StorageID::createEmpty(), segment.storage_id ? nullptr : segment.table_function_ast, segment.predicate_ast, - columns_to_cast_ptr, + columns_to_cast_names, metadata_columns, *segment_actual_columns); From ac8ad7d53e685d7ad9b351788bd8943353621bec Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 16:33:49 +0100 Subject: [PATCH 19/26] review comments --- src/Storages/StorageDistributed.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 26064c85f02a..60491c58d285 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -258,7 +258,10 @@ void applyHybridCastsToQueryTree( auto expected_column_opt = hybrid_schema.tryGetPhysical(column_name); if (!expected_column_opt) - return; + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Column '{}' expected in Hybrid schema while applying casts, but it is missing", + column_name); auto actual_column_opt = segment_columns.tryGetPhysical(column_name); const auto & source_column = actual_column_opt ? *actual_column_opt : *expected_column_opt; @@ -1249,7 +1252,8 @@ void StorageDistributed::read( return segment.storage_id->getNameForLogs(); if (segment.table_function_ast) return segment.table_function_ast->formatForLogging(); - return String{""}; + chassert(false, "Hybrid segment is missing both storage_id and table_function_ast"); + return String{""}; }; const bool log_hybrid_casts = need_hybrid_casts && log->is(Poco::Message::PRIO_TRACE); @@ -2712,8 +2716,9 @@ void registerStorageHybrid(StorageFactory & factory) } ColumnsDescription segment_columns; + if (validated_table) - segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); + segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); segment_definitions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); } From 113cfe6b93147ec9c8bc8759ce1931741bdda9b3 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 16:54:11 +0100 Subject: [PATCH 20/26] Prevent creation of the Hybrid tables when some segments does not provide the required columns --- src/Storages/StorageDistributed.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 60491c58d285..8e3d19848917 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -2572,6 +2572,22 @@ void registerStorageHybrid(StorageFactory & factory) if (columns_to_use.empty()) columns_to_use = first_segment_columns; + auto validate_segment_schema = [&](const ColumnsDescription & segment_columns, const String & segment_name) + { + for (const auto & column : columns_to_use.getAllPhysical()) + { + if (!segment_columns.tryGetPhysical(column.name)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Hybrid segment '{}' is missing column '{}' required by Hybrid schema", + segment_name, column.name); + } + } + }; + + validate_segment_schema(first_segment_columns, engine_args[0]->formatForLogging()); + // Execute the table function to get the underlying storage StoragePtr storage = table_function->execute( first_arg, @@ -2648,6 +2664,8 @@ void registerStorageHybrid(StorageFactory & factory) ColumnsDescription segment_columns = additional_table_function->getActualTableStructure(local_context, true); replaceCurrentDatabaseFunction(normalized_table_function_ast, local_context); + validate_segment_schema(segment_columns, normalized_table_function_ast->formatForLogging()); + // It's a table function - store the AST and cached schema for later execution segment_definitions.emplace_back(normalized_table_function_ast, predicate_ast, std::move(segment_columns)); } @@ -2720,6 +2738,8 @@ void registerStorageHybrid(StorageFactory & factory) if (validated_table) segment_columns = validated_table->getInMemoryMetadataPtr()->getColumns(); + validate_segment_schema(segment_columns, storage_id.getNameForLogs()); + segment_definitions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); } catch (const Exception & e) From eda930535ca29fdb34bce0b25b8d789110a1bb2b Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 19:57:59 +0100 Subject: [PATCH 21/26] moving the rewrite to a analyzer pass, seem to work much better --- .../engines/table-engines/special/hybrid.md | 2 +- src/Analyzer/Passes/HybridCastsPass.cpp | 168 +++++++++++++ src/Analyzer/Passes/HybridCastsPass.h | 20 ++ src/Analyzer/QueryTreePassManager.cpp | 3 + src/Storages/StorageDistributed.cpp | 226 ++++-------------- src/Storages/StorageDistributed.h | 20 +- tests/queries/0_stateless/03643_hybrid.sql | 13 +- 7 files changed, 251 insertions(+), 201 deletions(-) create mode 100644 src/Analyzer/Passes/HybridCastsPass.cpp create mode 100644 src/Analyzer/Passes/HybridCastsPass.h diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index f7ed0de2172b..b886bd5a736d 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -56,7 +56,7 @@ You must pass at least two arguments – the first table function and its predic - `predicate_n` must be an expression that can be evaluated on the table columns. The engine adds it to the segment's query with an additional `AND`, so expressions like `event_date >= '2025-09-01'` or `id BETWEEN 10 AND 15` are typical. - The query planner picks the same processing stage for every segment as it does for the base `Distributed` plan, so remote aggregation, ORDER BY pushdown, `skip_unused_shards`, and the legacy/analyzer execution modes behave the same way. - `INSERT` statements are forwarded to the first table function only. If you need multi-destination writes, use explicit `INSERT` statements into the respective sources. -- Align schemas across the segments. ClickHouse builds a common header; if the physical types differ you may need to add casts on one side or in the query, just as you would when reading from heterogeneous replicas. +- Align schemas across the segments. ClickHouse builds a common header and rejects creation if any segment misses a column defined in the Hybrid schema. If the physical types differ you may need to add casts on one side or in the query, just as you would when reading from heterogeneous replicas. ## Example: local cluster plus S3 historical tier diff --git a/src/Analyzer/Passes/HybridCastsPass.cpp b/src/Analyzer/Passes/HybridCastsPass.cpp new file mode 100644 index 000000000000..7da89e1034cf --- /dev/null +++ b/src/Analyzer/Passes/HybridCastsPass.cpp @@ -0,0 +1,168 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace Setting +{ + extern const SettingsBool hybrid_table_auto_cast_columns; +} + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +struct HybridCastTask +{ + QueryTreeNodePtr table_expression; + ColumnsDescription cast_schema; +}; + +// Visitor replaces all usages of the column with CAST(column, type) in the query tree. +// +// It normalizes headers coming from different segments when table structure in some segments +// differs from the Hybrid table definition. For example column X is UInt32 in the Hybrid table, +// but Int64 in an additional segment. +// +// Without these casts ConvertingActions may fail to reconcile mismatched headers when casts are impossible +// (e.g. AggregateFunction states carry hashed data tied to their argument type and cannot be recast), for example: +// "Conversion from AggregateFunction(uniq, Decimal(38, 0)) to AggregateFunction(uniq, UInt64) is not supported" +// (CANNOT_CONVERT_TYPE). +// +// Per-segment casts are not reliable because WithMergeState strips aliases, so merged pipelines +// from different segments would return different headers (with or without CAST), leading to errors +// like "Cannot find column `max(value)` in source stream, there are only columns: [max(_CAST(value, 'UInt64'))]" +// (THERE_IS_NO_COLUMN). +class HybridCastVisitor : public InDepthQueryTreeVisitor +{ +public: + HybridCastVisitor( + const std::unordered_map & cast_map_, + ContextPtr context_) + : cast_map(cast_map_) + , context(std::move(context_)) + {} + + bool shouldTraverseTopToBottom() const { return false; } + + static bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) + { + auto child_type = child->getNodeType(); + return !(child_type == QueryTreeNodeType::QUERY || child_type == QueryTreeNodeType::UNION); + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source = column_node->getColumnSourceOrNull(); + if (!column_source) + return; + + auto it = cast_map.find(column_source.get()); + if (it == cast_map.end()) + return; + + const auto & column_name = column_node->getColumnName(); + auto expected_column_opt = it->second.tryGetPhysical(column_name); + if (!expected_column_opt) + return; + + auto column_clone = std::static_pointer_cast(column_node->clone()); + column_clone->setColumnType(expected_column_opt->type); + + auto cast_node = buildCastFunction(column_clone, expected_column_opt->type, context); + const auto & alias = node->getAlias(); + if (!alias.empty()) + cast_node->setAlias(alias); + else + cast_node->setAlias(expected_column_opt->name); + + node = cast_node; + } + +private: + const std::unordered_map & cast_map; + ContextPtr context; +}; + + +} // namespace + +void collectHybridTables(const QueryTreeNodePtr & join_tree, std::unordered_map & cast_map) +{ + if (!join_tree) + return; + if (const auto * table = join_tree->as()) + { + const auto * storage = table->getStorage().get(); + if (const auto * distributed = typeid_cast(storage)) + { + ColumnsDescription to_cast = distributed->getColumnsToCast(); + if (!to_cast.empty()) + cast_map.emplace(join_tree.get(), std::move(to_cast)); + } + return; + } + if (const auto * func = join_tree->as()) + { + for (auto & child : func->getArguments().getNodes()) + collectHybridTables(child, cast_map); + return; + } + if (const auto * query = join_tree->as()) + { + collectHybridTables(query->getJoinTree(), cast_map); + } + if (const auto * union_node = join_tree->as()) + { + for (auto & child : union_node->getQueries().getNodes()) + collectHybridTables(child, cast_map); + } +} + +void HybridCastsPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) +{ + const auto & settings = context->getSettingsRef(); + if (!settings[Setting::hybrid_table_auto_cast_columns]) + return; + + auto * query = query_tree_node->as(); + if (!query) + return; + + std::unordered_map cast_map; + collectHybridTables(query->getJoinTree(), cast_map); + if (cast_map.empty()) + return; + + HybridCastVisitor visitor(cast_map, context); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/HybridCastsPass.h b/src/Analyzer/Passes/HybridCastsPass.h new file mode 100644 index 000000000000..e0696d668ba1 --- /dev/null +++ b/src/Analyzer/Passes/HybridCastsPass.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Adds CASTs for Hybrid segments when physical types differ from the Hybrid schema +/// and reorders the SELECT list to match the schema order (needed because planner +/// later aligns remote headers by position). +class HybridCastsPass : public IQueryTreePass +{ +public: + String getName() override { return "HybridCastsPass"; } + String getDescription() override { return "Inject casts for Hybrid columns to match schema types"; } + void run(QueryTreeNodePtr & query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a818ad348020..1d795869c147 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include namespace DB @@ -266,6 +267,8 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 8e3d19848917..57a4db30ece9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -118,7 +118,6 @@ #include #include #include -#include #include #include @@ -171,130 +170,7 @@ void replaceCurrentDatabaseFunction(ASTPtr & ast, const ContextPtr & context) replaceCurrentDatabaseFunction(child, context); } -NameSet collectColumnsToCast( - const ColumnsDescription & metadata_columns, - const ColumnsDescription & actual_columns) -{ - NameSet names; - if (metadata_columns.empty() || actual_columns.empty()) - return names; - - for (const auto & expected_column : metadata_columns.getAllPhysical()) - { - auto actual_column_opt = actual_columns.tryGetPhysical(expected_column.name); - if (!actual_column_opt) - continue; - if (!actual_column_opt->type->equals(*expected_column.type)) - names.emplace(expected_column.name); - } - - return names; -} - -void applyHybridCastsToQueryTree( - QueryTreeNodePtr & query_tree, - const QueryTreeNodePtr & replacement_table_expression, - const ColumnsDescription & metadata_columns, - const NameSet & columns_to_cast, - const ColumnsDescription & actual_columns, - const ContextPtr & context) -{ - if (columns_to_cast.empty()) - return; - - // Visitor replaces all usages of the column with CAST(column, type) in the query tree. - // - // It normalizes headers coming from different segments when table structure in some segments - // differs from the Hybrid table definition. For example column X is UInt32 in the Hybrid table, - // but Int64 in an additional segment. - // - // Without these casts ConvertingActions may fail to reconcile mismatched headers when casts are impossible - // (e.g. AggregateFunction states carry hashed data tied to their argument type and cannot be recast), for example: - // "Conversion from AggregateFunction(uniq, Decimal(38, 0)) to AggregateFunction(uniq, UInt64) is not supported" - // (CANNOT_CONVERT_TYPE). - // - // Per-segment casts are not reliable because WithMergeState strips aliases, so merged pipelines - // from different segments would return different headers (with or without CAST), leading to errors - // like "Cannot find column `max(value)` in source stream, there are only columns: [max(_CAST(value, 'UInt64'))]" - // (THERE_IS_NO_COLUMN). - class HybridCastVisitor : public InDepthQueryTreeVisitor - { - public: - HybridCastVisitor( - const QueryTreeNodePtr & table_expression_, - const ColumnsDescription & metadata_columns_, - const NameSet & columns_to_cast_, - const ColumnsDescription & actual_columns_, - ContextPtr context_) - : table_expression(table_expression_) - , hybrid_schema(metadata_columns_) - , columns_to_cast(columns_to_cast_) - , segment_columns(actual_columns_) - , context(std::move(context_)) - {} - - bool shouldTraverseTopToBottom() const { return false; } - - static bool needChildVisit(QueryTreeNodePtr & /*parent*/, QueryTreeNodePtr & child) - { - auto child_type = child->getNodeType(); - return !(child_type == QueryTreeNodeType::QUERY || child_type == QueryTreeNodeType::UNION); - } - - void visitImpl(QueryTreeNodePtr & node) - { - auto * column_node = node->as(); - if (!column_node) - return; - - auto column_source = column_node->getColumnSourceOrNull(); - if (column_source != table_expression) - return; - - const auto & column_name = column_node->getColumnName(); - if (!columns_to_cast.contains(column_name)) - return; - - auto expected_column_opt = hybrid_schema.tryGetPhysical(column_name); - if (!expected_column_opt) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Column '{}' expected in Hybrid schema while applying casts, but it is missing", - column_name); - - auto actual_column_opt = segment_columns.tryGetPhysical(column_name); - const auto & source_column = actual_column_opt ? *actual_column_opt : *expected_column_opt; - - auto column_clone = std::static_pointer_cast(column_node->clone()); - column_clone->setColumnType(source_column.type); - - auto cast_node = buildCastFunction(column_clone, expected_column_opt->type, context); - const auto & alias = node->getAlias(); - if (!alias.empty()) - cast_node->setAlias(alias); - else - cast_node->setAlias(column_name); - - node = cast_node; - } - - private: - QueryTreeNodePtr table_expression; - const ColumnsDescription & hybrid_schema; - const NameSet & columns_to_cast; - const ColumnsDescription & segment_columns; - ContextPtr context; - }; - - HybridCastVisitor visitor( - replacement_table_expression, - metadata_columns, - columns_to_cast, - actual_columns, - context); - visitor.visit(query_tree); -} } namespace Setting @@ -328,7 +204,6 @@ namespace Setting extern const SettingsBool prefer_global_in_and_join; extern const SettingsBool enable_global_with_statement; extern const SettingsBool allow_experimental_hybrid_table; - extern const SettingsBool hybrid_table_auto_cast_columns; } namespace DistributedSetting @@ -1075,10 +950,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, const ASTPtr & remote_table_function, - const ASTPtr & additional_filter, - const NameSet & columns_to_cast, - const ColumnsDescription & metadata_columns, - const ColumnsDescription & actual_columns_for_cast) + const ASTPtr & additional_filter) { auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -1167,17 +1039,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, : std::move(filter); } - if (!columns_to_cast.empty()) - { - applyHybridCastsToQueryTree( - query_tree_to_modify, - replacement_table_expression, - metadata_columns, - columns_to_cast, - actual_columns_for_cast, - query_context); - } - ReplaseAliasColumnsVisitor replase_alias_columns_visitor; replase_alias_columns_visitor.visit(query_tree_to_modify); @@ -1219,32 +1080,6 @@ void StorageDistributed::read( const auto & settings = local_context->getSettingsRef(); auto metadata_ptr = getInMemoryMetadataPtr(); - const auto & metadata_columns = metadata_ptr->getColumns(); - - const bool enable_hybrid_column_casts = settings[Setting::hybrid_table_auto_cast_columns] - && (!base_segment_columns.empty() || !segments.empty()); - - NameSet columns_to_cast_names; /// Empty when no casts are needed - if (enable_hybrid_column_casts) - { - if (!base_segment_columns.empty()) - { - auto names = collectColumnsToCast(metadata_columns, base_segment_columns); - columns_to_cast_names.insert(names.begin(), names.end()); - } - - for (const auto & segment : segments) - { - auto names = collectColumnsToCast(metadata_columns, segment.actual_columns); - columns_to_cast_names.insert(names.begin(), names.end()); - } - } - - const bool need_hybrid_casts = !columns_to_cast_names.empty(); - const auto * base_actual_columns_for_cast = !base_segment_columns.empty() ? &base_segment_columns : &metadata_columns; - - if (need_hybrid_casts && !settings[Setting::allow_experimental_analyzer]) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Setting 'hybrid_table_auto_cast_columns' is supported only with allow_experimental_analyzer=1"); auto describe_segment_target = [&](const HybridSegment & segment) -> String { @@ -1256,10 +1091,11 @@ void StorageDistributed::read( return String{""}; }; - const bool log_hybrid_casts = need_hybrid_casts && log->is(Poco::Message::PRIO_TRACE); + const bool log_hybrid_query_rewrites = (!segments.empty() || base_segment_predicate); + auto log_rewritten_query = [&](const String & target, const ASTPtr & ast) { - if (!log_hybrid_casts || !ast) + if (!log_hybrid_query_rewrites || !ast) return; LOG_TRACE(log, "rewriteSelectQuery (target: {}) -> {}", target, ast->formatForLogging()); @@ -1283,10 +1119,7 @@ void StorageDistributed::read( query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr, - base_segment_predicate, - columns_to_cast_names, - metadata_columns, - *base_actual_columns_for_cast); + base_segment_predicate); Block block = *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. @@ -1307,16 +1140,12 @@ void StorageDistributed::read( { // Create a modified query info with the segment predicate SelectQueryInfo additional_query_info = query_info; - const auto * segment_actual_columns = !segment.actual_columns.empty() ? &segment.actual_columns : &metadata_columns; auto additional_query_tree = buildQueryTreeDistributed(additional_query_info, query_info.initial_storage_snapshot ? query_info.initial_storage_snapshot : storage_snapshot, segment.storage_id ? *segment.storage_id : StorageID::createEmpty(), segment.storage_id ? nullptr : segment.table_function_ast, - segment.predicate_ast, - columns_to_cast_names, - metadata_columns, - *segment_actual_columns); + segment.predicate_ast); additional_query_info.query = queryNodeToDistributedSelectQuery(additional_query_tree); additional_query_info.query_tree = std::move(additional_query_tree); @@ -2397,21 +2226,31 @@ void StorageDistributed::delayInsertOrThrowIfNeeded() const } } -void StorageDistributed::setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_) +void StorageDistributed::setHybridLayout(std::vector segments_) { - /// Hybrid keeps a snapshot of segment schemas captured during CREATE/ATTACH to avoid re-reading (possibly remote) headers on every query. - /// A TTL-based cache was considered but deemed overkill for this experimental feature. - /// Subsequent segment DDL changes are not auto-detected; reattach/recreate the Hybrid table to refresh. - base_segment_columns = std::move(base_segment_columns_); segments = std::move(segments_); log = getLogger("Hybrid (" + getStorageID().table_name + ")"); + cached_columns_to_cast = getColumnsToCast(); + if (!cached_columns_to_cast.empty() && log->is(Poco::Message::PRIO_DEBUG)) + { + std::vector cols; + for (const auto & col : cached_columns_to_cast.getAllPhysical()) + cols.push_back(col.name + " -> " + col.type->getName()); + LOG_DEBUG(log, "Hybrid auto-cast will apply to: {}", fmt::join(cols, ", ")); + } + auto virtuals = createVirtuals(); // or _segment_index? virtuals.addEphemeral("_table_index", std::make_shared(), "Index of the table function in Hybrid (0 for main table, 1+ for additional segments)"); setVirtuals(virtuals); } +ColumnsDescription StorageDistributed::getColumnsToCast() const +{ + return cached_columns_to_cast; +} + void registerStorageDistributed(StorageFactory & factory) { factory.registerStorage("Distributed", [](const StorageFactory::Arguments & args) @@ -2572,17 +2411,22 @@ void registerStorageHybrid(StorageFactory & factory) if (columns_to_use.empty()) columns_to_use = first_segment_columns; + NameSet columns_to_cast_names; auto validate_segment_schema = [&](const ColumnsDescription & segment_columns, const String & segment_name) { for (const auto & column : columns_to_use.getAllPhysical()) { - if (!segment_columns.tryGetPhysical(column.name)) + auto found = segment_columns.tryGetPhysical(column.name); + if (!found) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Hybrid segment '{}' is missing column '{}' required by Hybrid schema", segment_name, column.name); } + + if (!found->type->equals(*column.type)) + columns_to_cast_names.emplace(column.name); } }; @@ -2667,7 +2511,7 @@ void registerStorageHybrid(StorageFactory & factory) validate_segment_schema(segment_columns, normalized_table_function_ast->formatForLogging()); // It's a table function - store the AST and cached schema for later execution - segment_definitions.emplace_back(normalized_table_function_ast, predicate_ast, std::move(segment_columns)); + segment_definitions.emplace_back(normalized_table_function_ast, predicate_ast); } else if (const auto * ast_identifier = table_function_ast->as()) { @@ -2740,7 +2584,7 @@ void registerStorageHybrid(StorageFactory & factory) validate_segment_schema(segment_columns, storage_id.getNameForLogs()); - segment_definitions.emplace_back(table_function_ast, predicate_ast, storage_id, std::move(segment_columns)); + segment_definitions.emplace_back(table_function_ast, predicate_ast, storage_id); } catch (const Exception & e) { @@ -2762,7 +2606,17 @@ void registerStorageHybrid(StorageFactory & factory) distributed_storage->renameInMemory({args.table_id.database_name, args.table_id.table_name, args.table_id.uuid}); // Store segment definitions for later use - distributed_storage->setHybridLayout(std::move(first_segment_columns), std::move(segment_definitions)); + distributed_storage->setHybridLayout(std::move(segment_definitions)); + if (!columns_to_cast_names.empty()) + { + NamesAndTypesList cast_cols; + for (const auto & col : columns_to_use.getAllPhysical()) + { + if (columns_to_cast_names.contains(col.name)) + cast_cols.emplace_back(col.name, col.type); + } + distributed_storage->setCachedColumnsToCast(ColumnsDescription(cast_cols)); + } return distributed_storage; }, diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f9dcfe9648ba..f38f25422682 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -60,17 +60,15 @@ class StorageDistributed final : public IStorage, WithContext std::optional storage_id; // For table identifiers instead of table functions ColumnsDescription actual_columns; - HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_, ColumnsDescription actual_columns_) + HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_) : table_function_ast(std::move(table_function_ast_)) , predicate_ast(std::move(predicate_ast_)) - , actual_columns(std::move(actual_columns_)) {} - HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_, StorageID storage_id_, ColumnsDescription actual_columns_) + HybridSegment(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_)) - , actual_columns(std::move(actual_columns_)) {} }; @@ -182,10 +180,12 @@ class StorageDistributed final : public IStorage, WithContext void setBaseSegmentPredicate(ASTPtr predicate) { base_segment_predicate = std::move(predicate); } /// Set segment definitions for Hybrid engine along with cached schema info - void setHybridLayout(ColumnsDescription base_segment_columns_, std::vector segments_); + void setHybridLayout(std::vector segments_); + void setCachedColumnsToCast(ColumnsDescription columns); /// Getter methods for ClusterProxy::executeQuery StorageID getRemoteStorageID() const { return remote_storage; } + ColumnsDescription getColumnsToCast() const; ExpressionActionsPtr getShardingKeyExpression() const { return sharding_key_expr; } const DistributedSettings * getDistributedSettings() const { return distributed_settings.get(); } bool isRemoteFunction() const { return is_remote_function; } @@ -330,7 +330,15 @@ class StorageDistributed final : public IStorage, WithContext /// Additional segments for Hybrid engine std::vector segments; - ColumnsDescription base_segment_columns; + + /// Hybrid build the list of columns which need to be casted once during CREATE/ATTACH + /// those are columns which type differs from the expected at least on one segment. + /// is is used by HybridCastsPass and hybrid_table_auto_cast_columns feature + /// without cache that would require reading the headers of the segments before every query + /// which may trigger extra DESCRIBE call in case of remote queries. + /// Subsequent segment DDL changes are not auto-detected; + /// reattach/recreate the Hybrid table to refresh. + ColumnsDescription cached_columns_to_cast; }; } diff --git a/tests/queries/0_stateless/03643_hybrid.sql b/tests/queries/0_stateless/03643_hybrid.sql index 86ef79259250..ebcd62da9e02 100644 --- a/tests/queries/0_stateless/03643_hybrid.sql +++ b/tests/queries/0_stateless/03643_hybrid.sql @@ -129,10 +129,7 @@ INSERT INTO test_hybrid_segment_partial VALUES (3), (4); CREATE TABLE test_hybrid_missing_column ENGINE = Hybrid( remote('localhost:9000', currentDatabase(), 'test_hybrid_segment_full'), id < 3, remote('localhost:9000', currentDatabase(), 'test_hybrid_segment_partial'), id >= 3 -); - -SELECT id, value FROM test_hybrid_missing_column ORDER BY id SETTINGS enable_analyzer = 0; -- { serverError UNKNOWN_IDENTIFIER } -SELECT id, value FROM test_hybrid_missing_column ORDER BY id SETTINGS enable_analyzer = 1; -- { serverError UNKNOWN_IDENTIFIER } +); -- { serverError BAD_ARGUMENTS } DROP TABLE IF EXISTS test_hybrid_missing_column SYNC; DROP TABLE IF EXISTS test_hybrid_segment_partial SYNC; @@ -306,19 +303,19 @@ SELECT count() FROM test_tiered_watermark_before WHERE id = 17; SELECT 'Read predicate-filtered data with analyzer disabled and no localhost preference'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 0; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0; SELECT 'Read predicate-filtered data with analyzer enabled and no localhost preference'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0; SELECT 'Read predicate-filtered data with analyzer disabled and prefer localhost replica'; SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 1; SELECT 'Read predicate-filtered data with analyzer enabled and prefer localhost replica'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 1; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 1, hybrid_table_auto_cast_columns = 0; -- other combinations of settings work, but give a bit different content in the query_log -- See the problem around is_initial_query described in https://github.com/Altinity/ClickHouse/issues/1077 SELECT 'Check if the subqueries were recorded in query_log'; -SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark', max_threads=1 FORMAT Null; +SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark', max_threads=1 FORMAT Null; SYSTEM FLUSH LOGS; SELECT type, From 21862695bb681822de63c4065d3a58464e45d00c Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 21:29:00 +0100 Subject: [PATCH 22/26] finishing --- .../engines/table-engines/special/hybrid.md | 2 +- src/Analyzer/Passes/HybridCastsPass.cpp | 17 +------ src/Analyzer/Passes/HybridCastsPass.h | 16 ++++++- src/Analyzer/QueryTreePassManager.cpp | 4 +- src/Storages/StorageDistributed.cpp | 46 +++++++++++-------- src/Storages/StorageDistributed.h | 1 - .../0_stateless/03643_hybrid.reference | 34 ++++++++++++-- tests/queries/0_stateless/03643_hybrid.sql | 36 ++++++++++++--- .../0_stateless/03644_hybrid_auto_cast.sql | 4 +- 9 files changed, 106 insertions(+), 54 deletions(-) diff --git a/docs/en/engines/table-engines/special/hybrid.md b/docs/en/engines/table-engines/special/hybrid.md index b886bd5a736d..12df6cd859b8 100644 --- a/docs/en/engines/table-engines/special/hybrid.md +++ b/docs/en/engines/table-engines/special/hybrid.md @@ -32,7 +32,7 @@ SET allow_experimental_hybrid_table = 1; ### Automatic Type Alignment -Hybrid segments can evolve independently, so the same logical column may use different physical types. With the experimental `hybrid_table_auto_cast_columns = 1` (default, requires `allow_experimental_analyzer = 1`), the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. This prevents header mismatches without having to edit each query, and you can opt out by setting the flag to `0` if it causes issues in your setup. +Hybrid segments can evolve independently, so the same logical column may use different physical types. With the experimental `hybrid_table_auto_cast_columns = 1` **(enabled by default and requires `allow_experimental_analyzer = 1`)**, the engine inserts the necessary `CAST` operations into each rewritten query so every shard receives the schema defined by the Hybrid table. You can opt out by setting the flag to `0` if it causes issues. Segment schemas are cached when you create or attach a Hybrid table. If you alter a segment later (for example change a column type), refresh the Hybrid table (detach/attach or recreate it) so the cached headers stay in sync with the new schema; otherwise the auto-cast feature may miss the change and queries can still fail with header/type errors. diff --git a/src/Analyzer/Passes/HybridCastsPass.cpp b/src/Analyzer/Passes/HybridCastsPass.cpp index 7da89e1034cf..a48c25db9c58 100644 --- a/src/Analyzer/Passes/HybridCastsPass.cpp +++ b/src/Analyzer/Passes/HybridCastsPass.cpp @@ -16,7 +16,6 @@ #include #include -#include #include namespace DB @@ -42,20 +41,6 @@ struct HybridCastTask }; // Visitor replaces all usages of the column with CAST(column, type) in the query tree. -// -// It normalizes headers coming from different segments when table structure in some segments -// differs from the Hybrid table definition. For example column X is UInt32 in the Hybrid table, -// but Int64 in an additional segment. -// -// Without these casts ConvertingActions may fail to reconcile mismatched headers when casts are impossible -// (e.g. AggregateFunction states carry hashed data tied to their argument type and cannot be recast), for example: -// "Conversion from AggregateFunction(uniq, Decimal(38, 0)) to AggregateFunction(uniq, UInt64) is not supported" -// (CANNOT_CONVERT_TYPE). -// -// Per-segment casts are not reliable because WithMergeState strips aliases, so merged pipelines -// from different segments would return different headers (with or without CAST), leading to errors -// like "Cannot find column `max(value)` in source stream, there are only columns: [max(_CAST(value, 'UInt64'))]" -// (THERE_IS_NO_COLUMN). class HybridCastVisitor : public InDepthQueryTreeVisitor { public: @@ -125,7 +110,7 @@ void collectHybridTables(const QueryTreeNodePtr & join_tree, std::unordered_map< { ColumnsDescription to_cast = distributed->getColumnsToCast(); if (!to_cast.empty()) - cast_map.emplace(join_tree.get(), std::move(to_cast)); + cast_map.emplace(join_tree.get(), std::move(to_cast)); // repeated table_expression can overwrite } return; } diff --git a/src/Analyzer/Passes/HybridCastsPass.h b/src/Analyzer/Passes/HybridCastsPass.h index e0696d668ba1..6b3159d6e925 100644 --- a/src/Analyzer/Passes/HybridCastsPass.h +++ b/src/Analyzer/Passes/HybridCastsPass.h @@ -7,8 +7,20 @@ namespace DB { /// Adds CASTs for Hybrid segments when physical types differ from the Hybrid schema -/// and reorders the SELECT list to match the schema order (needed because planner -/// later aligns remote headers by position). +/// +/// It normalizes headers coming from different segments when table structure in some segments +/// differs from the Hybrid table definition. For example column X is UInt32 in the Hybrid table, +/// but Int64 in an additional segment. +/// +/// Without these casts ConvertingActions may fail to reconcile mismatched headers when casts are impossible +/// (e.g. AggregateFunction states carry hashed data tied to their argument type and cannot be recast), for example: +/// "Conversion from AggregateFunction(uniq, Decimal(38, 0)) to AggregateFunction(uniq, UInt64) is not supported" +/// (CANNOT_CONVERT_TYPE). +/// +/// Per-segment casts are not reliable because WithMergeState strips aliases, so merged pipelines +/// from different segments would return different headers (with or without CAST), leading to errors +/// like "Cannot find column `max(value)` in source stream, there are only columns: [max(_CAST(value, 'UInt64'))]" +/// (THERE_IS_NO_COLUMN). class HybridCastsPass : public IQueryTreePass { public: diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 1d795869c147..3f94444cdc53 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -267,8 +267,6 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); @@ -312,6 +310,8 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + + manager.addPass(std::make_unique()); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 57a4db30ece9..dc0fda7ece84 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -950,7 +950,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, const ASTPtr & remote_table_function, - const ASTPtr & additional_filter) + const ASTPtr & additional_filter = nullptr) { auto & planner_context = query_info.planner_context; const auto & query_context = planner_context->getQueryContext(); @@ -1091,6 +1091,17 @@ void StorageDistributed::read( return String{""}; }; + auto describe_base_target = [&]() -> String + { + if (remote_table_function_ptr) + return remote_table_function_ptr->formatForLogging(); + if (!remote_database.empty()) + return remote_database + "." + remote_table; + return remote_table; + }; + + String base_target = describe_base_target(); + const bool log_hybrid_query_rewrites = (!segments.empty() || base_segment_predicate); auto log_rewritten_query = [&](const String & target, const ASTPtr & ast) @@ -1101,14 +1112,6 @@ void StorageDistributed::read( LOG_TRACE(log, "rewriteSelectQuery (target: {}) -> {}", target, ast->formatForLogging()); }; - String base_target; - if (remote_table_function_ptr) - base_target = remote_table_function_ptr->formatForLogging(); - else if (!remote_database.empty()) - base_target = remote_database + "." + remote_table; - else - base_target = remote_table; - if (settings[Setting::allow_experimental_analyzer]) { StorageID remote_storage_id = StorageID::createEmpty(); @@ -2231,26 +2234,31 @@ void StorageDistributed::setHybridLayout(std::vector segments_) segments = std::move(segments_); log = getLogger("Hybrid (" + getStorageID().table_name + ")"); - cached_columns_to_cast = getColumnsToCast(); - if (!cached_columns_to_cast.empty() && log->is(Poco::Message::PRIO_DEBUG)) - { - std::vector cols; - for (const auto & col : cached_columns_to_cast.getAllPhysical()) - cols.push_back(col.name + " -> " + col.type->getName()); - LOG_DEBUG(log, "Hybrid auto-cast will apply to: {}", fmt::join(cols, ", ")); - } - auto virtuals = createVirtuals(); // or _segment_index? virtuals.addEphemeral("_table_index", std::make_shared(), "Index of the table function in Hybrid (0 for main table, 1+ for additional segments)"); setVirtuals(virtuals); } +void StorageDistributed::setCachedColumnsToCast(ColumnsDescription columns) +{ + cached_columns_to_cast = std::move(columns); + if (!cached_columns_to_cast.empty() && log) + { + Names columns_with_types; + columns_with_types.reserve(cached_columns_to_cast.getAllPhysical().size()); + for (const auto & col : cached_columns_to_cast.getAllPhysical()) + columns_with_types.emplace_back(col.name + " " + col.type->getName()); + LOG_DEBUG(log, "Hybrid auto-cast will apply to: [{}]", fmt::join(columns_with_types, ", ")); + } +} + ColumnsDescription StorageDistributed::getColumnsToCast() const { return cached_columns_to_cast; } + void registerStorageDistributed(StorageFactory & factory) { factory.registerStorage("Distributed", [](const StorageFactory::Arguments & args) @@ -2421,7 +2429,7 @@ void registerStorageHybrid(StorageFactory & factory) { throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Hybrid segment '{}' is missing column '{}' required by Hybrid schema", + "Hybrid segment {} is missing column '{}' required by Hybrid schema", segment_name, column.name); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f38f25422682..9526fc647fbc 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -58,7 +58,6 @@ class StorageDistributed final : public IStorage, WithContext ASTPtr table_function_ast; ASTPtr predicate_ast; std::optional storage_id; // For table identifiers instead of table functions - ColumnsDescription actual_columns; HybridSegment(ASTPtr table_function_ast_, ASTPtr predicate_ast_) : table_function_ast(std::move(table_function_ast_)) diff --git a/tests/queries/0_stateless/03643_hybrid.reference b/tests/queries/0_stateless/03643_hybrid.reference index aef70a61a949..1954c097b478 100644 --- a/tests/queries/0_stateless/03643_hybrid.reference +++ b/tests/queries/0_stateless/03643_hybrid.reference @@ -174,14 +174,14 @@ Read predicate-filtered data with analyzer enabled and prefer localhost replica 21 Alice 2025-08-15 100 22 Bob 2025-08-20 200 23 Charlie 2025-08-25 300 -Check if the subqueries were recorded in query_log +Check if the subqueries were recorded in query_log (hybrid_table_auto_cast_columns = 0) Row 1: ────── type: QueryFinish is_initial_query2: 1 tbl: ['_table_function.remote','db.test_tiered_watermark'] -qry: SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark', max_threads=1 FORMAT Null; -log_comment: test_tiered_watermark +qry: SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, hybrid_table_auto_cast_columns = 0, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark1', max_threads=1 FORMAT Null; +log_comment: test_tiered_watermark1 Row 2: ────── @@ -189,7 +189,7 @@ type: QueryFinish is_initial_query2: 0 tbl: ['db.test_tiered_watermark_after'] qry: SELECT `__table1`.`id` AS `id`, `__table1`.`name` AS `name`, `__table1`.`date` AS `date`, `__table1`.`value` AS `value` FROM `db`.`test_tiered_watermark_after` AS `__table1` WHERE `__table1`.`date` >= '2025-09-01' ORDER BY `__table1`.`id` DESC -log_comment: test_tiered_watermark +log_comment: test_tiered_watermark1 Row 3: ────── @@ -197,5 +197,29 @@ type: QueryFinish is_initial_query2: 0 tbl: ['db.test_tiered_watermark_before'] qry: SELECT `__table1`.`id` AS `id`, `__table1`.`name` AS `name`, `__table1`.`date` AS `date`, `__table1`.`value` AS `value` FROM `db`.`test_tiered_watermark_before` AS `__table1` WHERE `__table1`.`date` < '2025-09-01' ORDER BY `__table1`.`id` DESC -log_comment: test_tiered_watermark +log_comment: test_tiered_watermark1 +Check if the subqueries were recorded in query_log (hybrid_table_auto_cast_columns = 1) +Row 1: +────── +type: QueryFinish +is_initial_query2: 1 +tbl: ['_table_function.remote','db.test_tiered_watermark'] +qry: SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, hybrid_table_auto_cast_columns = 1, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark2', max_threads=1 FORMAT Null; +log_comment: test_tiered_watermark2 + +Row 2: +────── +type: QueryFinish +is_initial_query2: 0 +tbl: ['db.test_tiered_watermark_after'] +qry: SELECT _CAST(`__table1`.`id`, 'UInt32') AS `id`, _CAST(`__table1`.`name`, 'String') AS `name`, `__table1`.`date` AS `date`, _CAST(`__table1`.`value`, 'UInt32') AS `value` FROM `db`.`test_tiered_watermark_after` AS `__table1` WHERE `__table1`.`date` >= '2025-09-01' ORDER BY `id` DESC +log_comment: test_tiered_watermark2 + +Row 3: +────── +type: QueryFinish +is_initial_query2: 0 +tbl: ['db.test_tiered_watermark_before'] +qry: SELECT _CAST(`__table1`.`id`, 'UInt32') AS `id`, _CAST(`__table1`.`name`, 'String') AS `name`, `__table1`.`date` AS `date`, _CAST(`__table1`.`value`, 'UInt32') AS `value` FROM `db`.`test_tiered_watermark_before` AS `__table1` WHERE `__table1`.`date` < '2025-09-01' ORDER BY _CAST(`__table1`.`id`, 'UInt32') DESC +log_comment: test_tiered_watermark2 Clean up predicate filtering tables diff --git a/tests/queries/0_stateless/03643_hybrid.sql b/tests/queries/0_stateless/03643_hybrid.sql index ebcd62da9e02..851045aafcfa 100644 --- a/tests/queries/0_stateless/03643_hybrid.sql +++ b/tests/queries/0_stateless/03643_hybrid.sql @@ -303,19 +303,19 @@ SELECT count() FROM test_tiered_watermark_before WHERE id = 17; SELECT 'Read predicate-filtered data with analyzer disabled and no localhost preference'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 0; SELECT 'Read predicate-filtered data with analyzer enabled and no localhost preference'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0; SELECT 'Read predicate-filtered data with analyzer disabled and prefer localhost replica'; SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 0, prefer_localhost_replica = 1; SELECT 'Read predicate-filtered data with analyzer enabled and prefer localhost replica'; -SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 1, hybrid_table_auto_cast_columns = 0; +SELECT * FROM test_tiered_watermark ORDER BY id SETTINGS enable_analyzer = 1, prefer_localhost_replica = 1; -- other combinations of settings work, but give a bit different content in the query_log -- See the problem around is_initial_query described in https://github.com/Altinity/ClickHouse/issues/1077 -SELECT 'Check if the subqueries were recorded in query_log'; +SELECT 'Check if the subqueries were recorded in query_log (hybrid_table_auto_cast_columns = 0)'; -SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, prefer_localhost_replica = 0, hybrid_table_auto_cast_columns = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark', max_threads=1 FORMAT Null; +SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, hybrid_table_auto_cast_columns = 0, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark1', max_threads=1 FORMAT Null; SYSTEM FLUSH LOGS; SELECT type, @@ -331,7 +331,31 @@ WHERE FROM system.query_log WHERE event_time > now() - 300 - and log_comment = 'test_tiered_watermark' + and log_comment = 'test_tiered_watermark1' + and current_database = currentDatabase() + and query_id = initial_query_id ) +ORDER BY tbl, event_time_microseconds +FORMAT Vertical; + +SELECT 'Check if the subqueries were recorded in query_log (hybrid_table_auto_cast_columns = 1)'; + +SELECT * FROM test_tiered_watermark ORDER BY id DESC SETTINGS enable_analyzer = 1, hybrid_table_auto_cast_columns = 1, prefer_localhost_replica = 0, log_queries=1, serialize_query_plan=0, log_comment = 'test_tiered_watermark2', max_threads=1 FORMAT Null; +SYSTEM FLUSH LOGS; +SELECT + type, + query_id = initial_query_id AS is_initial_query2, + arraySort(arrayMap(x -> replaceAll(x, currentDatabase(), 'db'), tables)) as tbl, + replaceAll(query, currentDatabase(), 'db') as qry, + log_comment +FROM system.query_log +WHERE + event_time > now() - 300 AND type = 'QueryFinish' AND + initial_query_id IN ( + SELECT initial_query_id + FROM system.query_log + WHERE + event_time > now() - 300 + and log_comment = 'test_tiered_watermark2' and current_database = currentDatabase() and query_id = initial_query_id ) ORDER BY tbl, event_time_microseconds diff --git a/tests/queries/0_stateless/03644_hybrid_auto_cast.sql b/tests/queries/0_stateless/03644_hybrid_auto_cast.sql index eb5447572418..7248dd9a55ef 100644 --- a/tests/queries/0_stateless/03644_hybrid_auto_cast.sql +++ b/tests/queries/0_stateless/03644_hybrid_auto_cast.sql @@ -81,5 +81,5 @@ SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_t SELECT 'hybrid_table_auto_cast_columns = 1, enable_analyzer = 0 (analizer required)'; SET hybrid_table_auto_cast_columns = 1, enable_analyzer = 0; -SELECT max(value) FROM test_tiered_watermark; -- { serverError NOT_IMPLEMENTED } -SELECT sum(if(arrayExists(x -> (x IN (10)), categories), 1, 0)) AS x FROM test_tiered_watermark; -- { serverError NOT_IMPLEMENTED } +SELECT max(value) FROM test_tiered_watermark; -- { serverError CANNOT_CONVERT_TYPE } + From 90cbc5e838c597044acdeed8a0e4043df80ed98b Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 21:52:52 +0100 Subject: [PATCH 23/26] better --- src/Analyzer/Passes/HybridCastsPass.cpp | 71 +++++++++++-------------- src/Storages/StorageDistributed.cpp | 1 + 2 files changed, 33 insertions(+), 39 deletions(-) diff --git a/src/Analyzer/Passes/HybridCastsPass.cpp b/src/Analyzer/Passes/HybridCastsPass.cpp index a48c25db9c58..0acf7cb11b01 100644 --- a/src/Analyzer/Passes/HybridCastsPass.cpp +++ b/src/Analyzer/Passes/HybridCastsPass.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -34,10 +35,33 @@ namespace ErrorCodes namespace { -struct HybridCastTask +/// Collect Hybrid table expressions that require casts to normalize headers across segments. +class HybridCastTablesCollector : public InDepthQueryTreeVisitor { - QueryTreeNodePtr table_expression; - ColumnsDescription cast_schema; +public: + explicit HybridCastTablesCollector(std::unordered_map & cast_map_) + : cast_map(cast_map_) + {} + + static bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr &) { return true; } + + void visitImpl(QueryTreeNodePtr & node) + { + const auto * table = node->as(); + if (!table) + return; + + const auto * storage = table->getStorage().get(); + if (const auto * distributed = typeid_cast(storage)) + { + ColumnsDescription to_cast = distributed->getColumnsToCast(); + if (!to_cast.empty()) + cast_map.emplace(node.get(), std::move(to_cast)); // repeated table_expression can overwrite + } + } + +private: + std::unordered_map & cast_map; }; // Visitor replaces all usages of the column with CAST(column, type) in the query tree. @@ -55,8 +79,9 @@ class HybridCastVisitor : public InDepthQueryTreeVisitor static bool needChildVisit(QueryTreeNodePtr &, QueryTreeNodePtr & child) { - auto child_type = child->getNodeType(); - return !(child_type == QueryTreeNodeType::QUERY || child_type == QueryTreeNodeType::UNION); + /// Traverse all child nodes so casts also apply inside subqueries and UNION branches. + (void)child; + return true; } void visitImpl(QueryTreeNodePtr & node) @@ -79,7 +104,6 @@ class HybridCastVisitor : public InDepthQueryTreeVisitor return; auto column_clone = std::static_pointer_cast(column_node->clone()); - column_clone->setColumnType(expected_column_opt->type); auto cast_node = buildCastFunction(column_clone, expected_column_opt->type, context); const auto & alias = node->getAlias(); @@ -99,38 +123,6 @@ class HybridCastVisitor : public InDepthQueryTreeVisitor } // namespace -void collectHybridTables(const QueryTreeNodePtr & join_tree, std::unordered_map & cast_map) -{ - if (!join_tree) - return; - if (const auto * table = join_tree->as()) - { - const auto * storage = table->getStorage().get(); - if (const auto * distributed = typeid_cast(storage)) - { - ColumnsDescription to_cast = distributed->getColumnsToCast(); - if (!to_cast.empty()) - cast_map.emplace(join_tree.get(), std::move(to_cast)); // repeated table_expression can overwrite - } - return; - } - if (const auto * func = join_tree->as()) - { - for (auto & child : func->getArguments().getNodes()) - collectHybridTables(child, cast_map); - return; - } - if (const auto * query = join_tree->as()) - { - collectHybridTables(query->getJoinTree(), cast_map); - } - if (const auto * union_node = join_tree->as()) - { - for (auto & child : union_node->getQueries().getNodes()) - collectHybridTables(child, cast_map); - } -} - void HybridCastsPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) { const auto & settings = context->getSettingsRef(); @@ -142,7 +134,8 @@ void HybridCastsPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context return; std::unordered_map cast_map; - collectHybridTables(query->getJoinTree(), cast_map); + HybridCastTablesCollector collector(cast_map); + collector.visit(query_tree_node); if (cast_map.empty()) return; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index dc0fda7ece84..d518887b3f6d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -120,6 +120,7 @@ #include #include #include +#include namespace fs = std::filesystem; From 0896a857d187c6a4c83dfdd8ebea279a4bb282de Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 22:04:59 +0100 Subject: [PATCH 24/26] comment --- src/Analyzer/Passes/HybridCastsPass.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Analyzer/Passes/HybridCastsPass.cpp b/src/Analyzer/Passes/HybridCastsPass.cpp index 0acf7cb11b01..f40e7664df50 100644 --- a/src/Analyzer/Passes/HybridCastsPass.cpp +++ b/src/Analyzer/Passes/HybridCastsPass.cpp @@ -36,6 +36,10 @@ namespace { /// Collect Hybrid table expressions that require casts to normalize headers across segments. +/// +/// Hybrid is currently exposed only as an engine (TableNode). If it ever gets a table function +/// wrapper, this visitor must also look at TableFunctionNode and unwrap to the underlying +/// StorageDistributed so cached casts can be picked up there as well. class HybridCastTablesCollector : public InDepthQueryTreeVisitor { public: From 6f8377cdb5682bb0603c7316a036155a7f11091d Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 21 Nov 2025 23:02:51 +0100 Subject: [PATCH 25/26] Get rid of experimatal marker for hybrid_table_auto_cast_columns flag (anyway it works only if experimetal hybrid tables are enabled) --- src/Core/Settings.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 80e3b119f215..30728305319c 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6930,8 +6930,8 @@ Allows creation of tables with the [TimeSeries](../../engines/table-engines/inte Allows creation of tables with the [Hybrid](../../engines/table-engines/special/hybrid.md) table engine. )", EXPERIMENTAL) \ DECLARE(Bool, hybrid_table_auto_cast_columns, true, R"( -Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. Enabled by default; disable it if it causes issues. Segment schemas are cached when the Hybrid table is created or attached; if a segment schema changes later, detach/attach or recreate the Hybrid table so the cached headers stay in sync. -)", EXPERIMENTAL) \ +Automatically cast columns to the schema defined in Hybrid tables when remote segments expose different physical types. Works only with analyzer. Enabled by default, does nothing if (experimental) Hybrid tables are disabled; disable it if it causes issues. Segment schemas are cached when the Hybrid table is created or attached; if a segment schema changes later, detach/attach or recreate the Hybrid table so the cached headers stay in sync. +)", 0) \ DECLARE(Bool, allow_experimental_codecs, false, R"( If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing). )", EXPERIMENTAL) \ From 87f0a4ef9201a56cd4cef910666906d8c3766246 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Sat, 22 Nov 2025 18:11:30 +0100 Subject: [PATCH 26/26] Trigger CI/CD