From e436344365d96cc8f8afade5a6573a78bad0cabc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 16 Nov 2025 17:16:00 +0000 Subject: [PATCH 001/113] Backport #89272 to 25.8: Fix applying query masking rules to ddl statements. --- src/Common/SensitiveDataMasker.cpp | 7 ++++--- src/Common/SensitiveDataMasker.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/executeQuery.cpp | 4 ++-- src/Parsers/IAST.cpp | 2 +- .../configs/overrides.xml | 8 ++++++++ .../integration/test_mask_sensitive_info/test.py | 16 ++++++++++++++++ 7 files changed, 33 insertions(+), 8 deletions(-) diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 66bb556076ea..aa3c8eb6537a 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -216,12 +216,13 @@ size_t SensitiveDataMasker::rulesCount() const } -std::string wipeSensitiveDataAndCutToLength(std::string str, size_t max_length) +std::string wipeSensitiveDataAndCutToLength(std::string str, size_t max_length, bool wipe_sensitive) { std::string res = std::move(str); - if (auto masker = SensitiveDataMasker::getInstance()) - masker->wipeSensitiveData(res); + if (wipe_sensitive) + if (auto masker = SensitiveDataMasker::getInstance()) + masker->wipeSensitiveData(res); size_t length = res.length(); if (max_length && (length > max_length)) diff --git a/src/Common/SensitiveDataMasker.h b/src/Common/SensitiveDataMasker.h index 79b4940e48d2..707dbf7fa835 100644 --- a/src/Common/SensitiveDataMasker.h +++ b/src/Common/SensitiveDataMasker.h @@ -77,6 +77,6 @@ class SensitiveDataMasker /// Wipes sensitive data and cuts to a specified maximum length in one function call. /// If the maximum length is zero then the function doesn't cut to the maximum length. -std::string wipeSensitiveDataAndCutToLength(std::string str, size_t max_length); +std::string wipeSensitiveDataAndCutToLength(std::string str, size_t max_length, bool wipe_sensitive); } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 397b12cdf0c4..7c48a8322f71 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -797,7 +797,7 @@ String serializeQuery(const IAST & query, size_t max_length) { return query.hasSecretParts() ? query.formatForLogging(max_length) - : wipeSensitiveDataAndCutToLength(query.formatWithSecretsOneLine(), max_length); + : wipeSensitiveDataAndCutToLength(query.formatWithSecretsOneLine(), max_length, true); } } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 41a312de9e51..2a64d4883168 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1185,7 +1185,7 @@ static BlockIO executeQueryImpl( } else { - query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length, true); } normalized_query_hash = normalizedQueryHash(query_for_logging, false); @@ -1196,7 +1196,7 @@ static BlockIO executeQueryImpl( if (query.empty()) query.assign(begin, std::min(end - begin, static_cast(max_query_size))); - query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length, true); logQuery(query_for_logging, context, internal, stage); if (!internal) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index a9923f6e00d6..7cb64bd931a2 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -176,7 +176,7 @@ String IAST::formatWithPossiblyHidingSensitiveData( settings.identifier_quoting_rule = identifier_quoting_rule; settings.identifier_quoting_style = identifier_quoting_style; format(buf, settings); - return wipeSensitiveDataAndCutToLength(buf.str(), max_length); + return wipeSensitiveDataAndCutToLength(buf.str(), max_length, !show_secrets); } String IAST::formatForLogging(size_t max_length) const diff --git a/tests/integration/test_mask_sensitive_info/configs/overrides.xml b/tests/integration/test_mask_sensitive_info/configs/overrides.xml index 00e2cb229efb..b02424c1713b 100644 --- a/tests/integration/test_mask_sensitive_info/configs/overrides.xml +++ b/tests/integration/test_mask_sensitive_info/configs/overrides.xml @@ -27,4 +27,12 @@ + + + + Query masking rule + sensetive_data + replaced_data + + diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 453613060b91..434017308027 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -6,6 +6,8 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +from helpers.config_cluster import minio_secret_key +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node = cluster.add_instance( @@ -840,3 +842,17 @@ def test_iceberg_cluster_function(): ], must_not_contain=[password, azure_account_key], ) + + +def test_query_masking_rule_with_ddl(): + table_name = "test_ddl_masking" + node.query(f"DROP TABLE IF EXISTS {table_name}") + node.query( + f"CREATE TABLE {table_name} ON CLUSTER 'test_shard_localhost' (s String, sensetive_data UInt32) ENGINE = MergeTree ORDER BY s" + ) + + assert_eq_with_retry(node, "SELECT count(*) FROM system.distribution_queue", "0\n") + assert "sensetive_data" in node.query( + f"SELECT create_table_query FROM system.tables WHERE table='{table_name}' {show_secrets}" + ) + node.query("DROP TABLE IF EXISTS test_table") From 647d7e61216fa4d6b0a14a7ce0663b2ea16b6d20 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 17 Nov 2025 00:09:19 +0100 Subject: [PATCH 002/113] Fix test --- tests/integration/test_mask_sensitive_info/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 434017308027..f79b672e4fd1 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -853,6 +853,6 @@ def test_query_masking_rule_with_ddl(): assert_eq_with_retry(node, "SELECT count(*) FROM system.distribution_queue", "0\n") assert "sensetive_data" in node.query( - f"SELECT create_table_query FROM system.tables WHERE table='{table_name}' {show_secrets}" + f"SELECT create_table_query FROM system.tables WHERE table='{table_name}' {show_secrets}=1" ) node.query("DROP TABLE IF EXISTS test_table") From 9de21ecb30131b94f722c853d8f1ccbe771360aa Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Nov 2025 19:12:23 +0000 Subject: [PATCH 003/113] Backport #90013 to 25.8: Fix bug in DateTime64 inference --- src/IO/ReadHelpers.h | 3 +-- .../0_stateless/03720_datetime64_bad_inference.reference | 2 ++ tests/queries/0_stateless/03720_datetime64_bad_inference.sql | 4 ++++ 3 files changed, 7 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03720_datetime64_bad_inference.reference create mode 100644 tests/queries/0_stateless/03720_datetime64_bad_inference.sql diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 058799acf457..02caaf26736d 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1131,8 +1131,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } else { - auto ok = readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); - if (!ok && (buf.eof() || *buf.position() != '.')) + if (!readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters)) return ReturnType(false); } } diff --git a/tests/queries/0_stateless/03720_datetime64_bad_inference.reference b/tests/queries/0_stateless/03720_datetime64_bad_inference.reference new file mode 100644 index 000000000000..b0138a319d38 --- /dev/null +++ b/tests/queries/0_stateless/03720_datetime64_bad_inference.reference @@ -0,0 +1,2 @@ +d Nullable(String) +5981 10:01.000 diff --git a/tests/queries/0_stateless/03720_datetime64_bad_inference.sql b/tests/queries/0_stateless/03720_datetime64_bad_inference.sql new file mode 100644 index 000000000000..d9c76a1bc167 --- /dev/null +++ b/tests/queries/0_stateless/03720_datetime64_bad_inference.sql @@ -0,0 +1,4 @@ +set date_time_input_format='basic'; +desc format(JSONEachRow, '{"d" : "5981 10:01.000"}'); +select * from format(JSONEachRow, '{"d" : "5981 10:01.000"}'); + From 3aafc889d50264c41284a05c92a7daa46ff34097 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 25 Nov 2025 07:14:46 +0000 Subject: [PATCH 004/113] Backport #90556 to 25.8: Retry to markReplicasActive when failing to resolve address during local host check. --- src/Interpreters/DDLTask.cpp | 66 +++++++++++++++++++++------------- src/Interpreters/DDLTask.h | 6 ++-- src/Interpreters/DDLWorker.cpp | 38 +++++++------------- 3 files changed, 58 insertions(+), 52 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 1ebbe4a5965d..beebea7dd5ac 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -55,21 +55,8 @@ HostID HostID::fromString(const String & host_port_str) bool HostID::isLocalAddress(UInt16 clickhouse_port) const { - try - { - auto address = DNSResolver::instance().resolveAddress(host_name, port); - return DB::isLocalAddress(address, clickhouse_port); - } - catch (const DB::NetException &) - { - /// Avoid "Host not found" exceptions - return false; - } - catch (const Poco::Net::NetException &) - { - /// Avoid "Host not found" exceptions - return false; - } + auto address = DNSResolver::instance().resolveAddress(host_name, port); + return DB::isLocalAddress(address, clickhouse_port); } bool HostID::isLoopbackHost() const @@ -292,7 +279,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, LoggerPtr log, const if (config_host_name) { - if (!IsSelfHostname(*config_host_name, maybe_secure_port, port)) + if (!isSelfHostname(log, *config_host_name, maybe_secure_port, port)) throw Exception( ErrorCodes::DNS_ERROR, "{} is not a local address. Check parameter 'host_name' in the configuration", @@ -317,7 +304,7 @@ bool DDLTask::findCurrentHostID(ContextPtr global_context, LoggerPtr log, const try { - if (!IsSelfHostID(host, maybe_secure_port, port)) + if (!isSelfHostID(log, host, maybe_secure_port, port)) continue; if (host.isLoopbackHost()) @@ -541,18 +528,47 @@ String DDLTask::getShardID() const return res; } -bool DDLTask::IsSelfHostID(const HostID & checking_host_id, std::optional maybe_self_secure_port, UInt16 self_port) +bool DDLTask::isSelfHostID(LoggerPtr log, const HostID & checking_host_id, std::optional maybe_self_secure_port, UInt16 self_port) { - // If the checking_host_id has a loopback address, it is not considered as the self host_id. - // Because all replicas will try to claim it as their own hosts. - return (maybe_self_secure_port && checking_host_id.isLocalAddress(*maybe_self_secure_port)) - || checking_host_id.isLocalAddress(self_port); + try + { + return (maybe_self_secure_port && checking_host_id.isLocalAddress(*maybe_self_secure_port)) + || checking_host_id.isLocalAddress(self_port); + } + catch (const DB::NetException & e) + { + /// Avoid "Host not found" exceptions + LOG_WARNING(log, "Unable to check if host {} is a local address, exception: {}", checking_host_id.host_name, e.displayText()); + return false; + } + catch (const Poco::Net::NetException & e) + { + /// Avoid "Host not found" exceptions + LOG_WARNING(log, "Unable to check if host {} is a local address, exception: {}", checking_host_id.host_name, e.displayText()); + return false; + } } -bool DDLTask::IsSelfHostname(const String & checking_host_name, std::optional maybe_self_secure_port, UInt16 self_port) +bool DDLTask::isSelfHostname( + LoggerPtr log, const String & checking_host_name, std::optional maybe_self_secure_port, UInt16 self_port) { - return (maybe_self_secure_port && HostID(checking_host_name, *maybe_self_secure_port).isLocalAddress(*maybe_self_secure_port)) - || HostID(checking_host_name, self_port).isLocalAddress(self_port); + try + { + return (maybe_self_secure_port && HostID(checking_host_name, *maybe_self_secure_port).isLocalAddress(*maybe_self_secure_port)) + || HostID(checking_host_name, self_port).isLocalAddress(self_port); + } + catch (const DB::NetException & e) + { + /// Avoid "Host not found" exceptions + LOG_WARNING(log, "Unable to check if host {} is a local address, exception: {}", checking_host_name, e.displayText()); + return false; + } + catch (const Poco::Net::NetException & e) + { + /// Avoid "Host not found" exceptions + LOG_WARNING(log, "Unable to check if host {} is a local address, exception: {}", checking_host_name, e.displayText()); + return false; + } } DatabaseReplicatedTask::DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 609b632d18b5..d0bcb5887bf9 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -158,8 +158,10 @@ struct DDLTask : public DDLTaskBase String getShardID() const override; - static bool IsSelfHostID(const HostID & checking_host_id, std::optional maybe_self_secure_port, UInt16 self_port); - static bool IsSelfHostname(const String & checking_host_name, std::optional maybe_self_secure_port, UInt16 self_port); + static bool + isSelfHostID(LoggerPtr log, const HostID & checking_host_id, std::optional maybe_self_secure_port, UInt16 self_port); + static bool + isSelfHostname(LoggerPtr log, const String & checking_host_name, std::optional maybe_self_secure_port, UInt16 self_port); private: bool tryFindHostInCluster(); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index b1922a694919..bfbc507d2246 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -76,6 +76,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; extern const int NO_ZOOKEEPER; + extern const int INVALID_CONFIG_PARAMETER; } constexpr const char * TASK_PROCESSED_OUT_REASON = "Task has been already processed"; @@ -1305,27 +1306,6 @@ void DDLWorker::markReplicasActive(bool /*reinitialized*/) active_node_holders.clear(); - for (auto it = active_node_holders.begin(); it != active_node_holders.end();) - { - auto & zk = it->second.first; - if (zk->expired()) - { - const auto & host_id = it->first; - String active_path = fs::path(replicas_dir) / host_id / "active"; - LOG_DEBUG(log, "Zookeeper of active_path {} expired, removing the holder", active_path); - - auto & active_node_holder = it->second.second; - if (active_node_holder) - active_node_holder->setAlreadyRemoved(); - it = active_node_holders.erase(it); - } - else - { - ++it; - } - } - - const auto maybe_secure_port = context->getTCPPortSecure(); const auto port = context->getTCPPort(); @@ -1334,13 +1314,10 @@ void DDLWorker::markReplicasActive(bool /*reinitialized*/) NameSet local_host_ids; for (const auto & host_id : host_ids) { - if (active_node_holders.contains(host_id)) - continue; - try { HostID host = HostID::fromString(host_id); - if (DDLTask::IsSelfHostID(host, maybe_secure_port, port)) + if (DDLTask::isSelfHostID(log, host, maybe_secure_port, port)) local_host_ids.emplace(host_id); } catch (const Exception & e) @@ -1394,6 +1371,17 @@ void DDLWorker::markReplicasActive(bool /*reinitialized*/) auto active_node_holder = zkutil::EphemeralNodeHolder::existing(active_path, *active_node_holder_zookeeper); active_node_holders[host_id] = {active_node_holder_zookeeper, active_node_holder}; } + + if (active_node_holders.empty()) + { + for (const auto & it : context->getClusters()) + { + const auto & cluster = it.second; + if (!cluster->getHostIDs().empty()) + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, "There are clusters with host ids but no local host found for this replica."); + } + } } void DDLWorker::cleanupStaleReplicas(Int64 current_time_seconds, const ZooKeeperPtr & zookeeper) From da5b853cb7664a812be0ec532bd6e77825043b92 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 25 Nov 2025 15:04:45 +0000 Subject: [PATCH 005/113] Update autogenerated version to 25.8.12.129 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- .../System/StorageSystemContributors.generated.cpp | 1 + 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 42bc8307e14a..7bef112d31f1 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54512) +SET(VERSION_REVISION 54513) SET(VERSION_MAJOR 25) SET(VERSION_MINOR 8) -SET(VERSION_PATCH 12) -SET(VERSION_GITHASH fa393206741c830da77b8f1bcf18c753161932c8) -SET(VERSION_DESCRIBE v25.8.12.1-lts) -SET(VERSION_STRING 25.8.12.1) +SET(VERSION_PATCH 13) +SET(VERSION_GITHASH 25db09bd0a09eb1576ae0ba56f6e52d9f2c4651e) +SET(VERSION_DESCRIBE v25.8.13.1-lts) +SET(VERSION_STRING 25.8.13.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 91d6a338f074..064736620152 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -962,6 +962,7 @@ const char * auto_contributors[] { "Nicolae Vartolomei", "Niek", "Nihal Z.", + "Nihal Z. Miaji", "Nik", "NikBarykin", "Nikhil Nadig", From c1599d35a708a08567d97b7ac7c45b57770c5929 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 26 Nov 2025 03:36:00 +0000 Subject: [PATCH 006/113] Backport #90816 to 25.8: Fix possible inconsistent state of shared data and dynamic paths in JSON --- src/Columns/ColumnObject.cpp | 45 +++++++++---------- src/Columns/ColumnObject.h | 4 +- .../Serializations/SerializationObject.cpp | 6 +-- .../SerializationVariantElement.cpp | 5 +++ ...ynamic_paths_and_shared_data_bug.reference | 0 ...h_in_dynamic_paths_and_shared_data_bug.sql | 9 ++++ 6 files changed, 38 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.reference create mode 100644 tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.sql diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 2d158c4896bf..4bb5e80de8bd 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -632,7 +632,6 @@ void ColumnObject::doInsertFrom(const IColumn & src, size_t n) /// Finally, insert paths from shared data. insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, std::move(src_dynamic_paths_for_shared_data), n, 1); - validateDynamicPathsSizes(); } #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -667,7 +666,6 @@ void ColumnObject::doInsertRangeFrom(const IColumn & src, size_t start, size_t l /// Finally, insert paths from shared data. insertFromSharedDataAndFillRemainingDynamicPaths(src_object_column, std::move(src_dynamic_paths_for_shared_data), start, length); - validateDynamicPathsSizes(); } void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::ColumnObject & src_object_column, std::vector && src_dynamic_paths_for_shared_data, size_t start, size_t length) @@ -731,32 +729,15 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co /// Deserialize binary value into dynamic column from shared data. if (it->second->size() != current_size) { - if (src_object_column.getDynamicPaths().contains(path)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Path {} is present both in shared data and in dynamic paths at row {}. Dynamic path value type: {}. Shared data path value type: {}", - path, - row, - src_object_column.getDynamicPathsPtrs().at(toString(path))->getTypeNameAt(row), - decodeDataType(src_shared_data_values->getDataAt(i).toString())->getName()); - - for (size_t j = offset; j != end; ++j) - { - if (j != i && src_shared_data_paths->getDataAt(j).toView() == path) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Path {} is duplicated inside shared data at offsets {} and {}. First value type: {}. Second value type: {}", - path, - i, - j, - decodeDataType(src_shared_data_values->getDataAt(i).toString())->getName(), - decodeDataType(src_shared_data_values->getDataAt(j).toString())->getName()); - } - + src_object_column.validateDynamicPathsAndSharedData(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of dynamic path {}: {} != {}", path, it->second->size(), current_size); } deserializeValueFromSharedData(src_shared_data_values, i, *it->second); } + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + { + deserializeValueFromSharedData(src_shared_data_values, i, *dynamic_path_column); + } else { /// Before inserting this path into shared data check if we need to @@ -2071,14 +2052,28 @@ int ColumnObject::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_d return 1; } -void ColumnObject::validateDynamicPathsSizes() const +void ColumnObject::validateDynamicPathsAndSharedData(size_t shared_data_offset) const { + if (dynamic_paths.empty()) + return; + size_t expected_size = shared_data->size(); for (const auto & [path, column] : dynamic_paths) { if (column->size() != expected_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of dynamic path {}: {} != {}", path, column->size(), expected_size); } + + const auto & shared_data_offsets = getSharedDataOffsets(); + const auto [shared_data_paths, _] = getSharedDataPathsAndValues(); + size_t shared_data_paths_start = shared_data_offsets[ssize_t(shared_data_offset) - 1]; + size_t shared_data_paths_end = shared_data_offsets.back(); + for (size_t i = shared_data_paths_start; i != shared_data_paths_end; ++i) + { + auto path = shared_data_paths->getDataAt(i); + if (dynamic_paths.contains(path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is present both in dynamic paths and in shared data", path.toString()); + } } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index c21b77f07b33..eb72e19813fd 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -281,8 +281,8 @@ class ColumnObject final : public COWHelper, ColumnO /// Insert all the data from shared data with specified path to dynamic column. static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); - /// Validate that all dynamic paths have correct sizes. - void validateDynamicPathsSizes() const; + /// Validate that all dynamic paths have correct sizes and that shared data doesn't contain any dynamic paths. + void validateDynamicPathsAndSharedData(size_t shared_data_offset = 0) const; private: class SortedPathsIterator; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index ba0753e89399..6108fc5fc528 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -780,7 +780,6 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( } const auto & column_object = assert_cast(column); - column_object.validateDynamicPathsSizes(); const auto & typed_paths = column_object.getTypedPaths(); if (object_state->serialization_version.value == SerializationVersion::FLATTENED) @@ -1032,12 +1031,13 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } + size_t shared_data_previous_size = shared_data->size(); settings.path.push_back(Substream::ObjectSharedData); object_state->shared_data_serialization->deserializeBinaryBulkWithMultipleStreams(shared_data, rows_offset, limit, settings, object_state->shared_data_state, cache); settings.path.pop_back(); settings.path.pop_back(); - column_object.validateDynamicPathsSizes(); + column_object.validateDynamicPathsAndSharedData(shared_data_previous_size); } void SerializationObject::serializeBinary(const Field & field, WriteBuffer & ostr, const DB::FormatSettings & settings) const @@ -1277,8 +1277,6 @@ void SerializationObject::deserializeBinary(IColumn & col, ReadBuffer & istr, co if (column->size() == prev_size) column->insertDefault(); } - - column_object.validateDynamicPathsSizes(); } SerializationPtr SerializationObject::TypedPathSubcolumnCreator::create(const DB::SerializationPtr & prev, const DataTypePtr &) const diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index ca985f03096e..b3ef19b987f9 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -250,6 +250,11 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, *variant_rows_offset, *variant_limit, nested_settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); + /// We want to keep dynamic structure of the variant during deserialization. + /// Keeping dynamic structure improves performance of insertFrom/insertRangeFrom methods. + if (mutable_column->empty()) + mutable_column->takeDynamicStructureFromColumn(variant_element_state->variant); + /// If there was nothing to deserialize or nothing was actually deserialized when variant_limit > 0, just insert defaults. /// The second case means that we don't have a stream for such sub-column. It may happen during ALTER MODIFY column with Variant extension. if (variant_limit == 0 || variant_element_state->variant->empty()) diff --git a/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.reference b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.sql b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.sql new file mode 100644 index 000000000000..5385c5923d96 --- /dev/null +++ b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_bug.sql @@ -0,0 +1,9 @@ +drop table if exists test; +drop table if exists test2; +create table test (id UInt64, json JSON) engine=MergeTree order by id; +insert into test select number, '{}' from numbers(100000); +alter table test update json = '{"a" : 42}' where id > 50000 settings mutations_sync=1; +create table test2 (json JSON) engine=MergeTree order by tuple(); +insert into test2 select if(id < 75000, json, '{"a" : 42}'::JSON) from test; +select * from test2 format Null; + From e7b1481d9aa58a9b72fc66db291ee74a6ce04c4f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 26 Nov 2025 10:13:37 +0000 Subject: [PATCH 007/113] Backport #90765 to 25.8: Fix named collections hidden secrets --- .../System/StorageSystemNamedCollections.cpp | 9 ++++++ .../configs/config.d/named_collections.xml | 1 + .../named_collections_with_zookeeper.xml | 2 ++ .../configs/users.d/users.xml | 1 + .../test_named_collections/test.py | 30 +++++++++++++++++++ .../config.d/named_collections_encrypted.xml | 2 ++ ...d_collections_with_zookeeper_encrypted.xml | 2 ++ .../configs/users.d/users.xml | 1 + .../configs/config.d/cluster.xml | 1 + .../configs/users.d/default.xml | 5 ++++ 10 files changed, 54 insertions(+) diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 12618558a177..cf8a38374bd2 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -17,6 +18,11 @@ namespace DB { +namespace Setting +{ + extern const SettingsBool format_display_secrets_in_show_and_select; +} + ColumnsDescription StorageSystemNamedCollections::getColumnsDescription() { return ColumnsDescription @@ -54,6 +60,9 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte auto & key_column = tuple_column.getColumn(0); auto & value_column = tuple_column.getColumn(1); bool access_secrets = access->isGranted(AccessType::SHOW_NAMED_COLLECTIONS_SECRETS); + access_secrets &= access->isGranted(AccessType::displaySecretsInShowAndSelect); + access_secrets &= context->getSettingsRef()[Setting::format_display_secrets_in_show_and_select]; + access_secrets &= context->displaySecretsInShowAndSelect(); size_t size = 0; for (const auto & key : collection->getKeys()) diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections.xml b/tests/integration/test_named_collections/configs/config.d/named_collections.xml index d24fb303b377..a837f0041ad0 100644 --- a/tests/integration/test_named_collections/configs/config.d/named_collections.xml +++ b/tests/integration/test_named_collections/configs/config.d/named_collections.xml @@ -4,4 +4,5 @@ value1 + 1 diff --git a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml index 43d80ee6f693..2f813ee909df 100644 --- a/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml +++ b/tests/integration/test_named_collections/configs/config.d/named_collections_with_zookeeper.xml @@ -10,6 +10,8 @@ + 1 + diff --git a/tests/integration/test_named_collections/configs/users.d/users.xml b/tests/integration/test_named_collections/configs/users.d/users.xml index 7d4f0543ff1a..f1baf979f7a1 100644 --- a/tests/integration/test_named_collections/configs/users.d/users.xml +++ b/tests/integration/test_named_collections/configs/users.d/users.xml @@ -2,6 +2,7 @@ 0 + 1 diff --git a/tests/integration/test_named_collections/test.py b/tests/integration/test_named_collections/test.py index 027c5304f195..9b1244b067ff 100644 --- a/tests/integration/test_named_collections/test.py +++ b/tests/integration/test_named_collections/test.py @@ -134,6 +134,36 @@ def test_default_access(cluster): ).strip() == "value1" ) + + assert ( + node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'", + settings={"format_display_secrets_in_show_and_select": 0} + ).strip() + == "[HIDDEN]" + ) + + replace_in_server_config( + node, "display_secrets_in_show_and_select>1", "display_secrets_in_show_and_select>0" + ) + assert "display_secrets_in_show_and_select>0" in node.exec_in_container( + ["bash", "-c", f"cat /etc/clickhouse-server/config.d/named_collections.xml"] + ) + node.restart_clickhouse() + assert ( + node.query( + "select collection['key1'] from system.named_collections where name = 'collection1'" + ).strip() + == "[HIDDEN]" + ) + + replace_in_server_config( + node, "display_secrets_in_show_and_select>0", "display_secrets_in_show_and_select>1" + ) + assert "display_secrets_in_show_and_select>1" in node.exec_in_container( + ["bash", "-c", f"cat /etc/clickhouse-server/config.d/named_collections.xml"] + ) + replace_in_users_config( node, "show_named_collections_secrets>1", "show_named_collections_secrets>0" ) diff --git a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml index 233e23846cb9..c4bcf60d8bd6 100644 --- a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml +++ b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_encrypted.xml @@ -9,4 +9,6 @@ value1 + + 1 diff --git a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml index d1dd5c297878..b8d13cef922e 100644 --- a/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml +++ b/tests/integration/test_named_collections_encrypted/configs/config.d/named_collections_with_zookeeper_encrypted.xml @@ -28,4 +28,6 @@ true + + 1 diff --git a/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml b/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml index 7d4f0543ff1a..f1baf979f7a1 100644 --- a/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml +++ b/tests/integration/test_named_collections_encrypted/configs/users.d/users.xml @@ -2,6 +2,7 @@ 0 + 1 diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml b/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml index 95336350c1e8..1cd6b9f13f14 100644 --- a/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml +++ b/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml @@ -19,4 +19,5 @@ true + 1 diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml b/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml index 15da914f666b..308f3b976a50 100644 --- a/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml +++ b/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml @@ -1,4 +1,9 @@ + + + 1 + + From 0268e33a219f0cace1c27870edc7d463742492c8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 26 Nov 2025 16:53:09 +0100 Subject: [PATCH 008/113] Update SerializationVariantElement.cpp --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index b3ef19b987f9..c62c1f4608a7 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -253,7 +253,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( /// We want to keep dynamic structure of the variant during deserialization. /// Keeping dynamic structure improves performance of insertFrom/insertRangeFrom methods. if (mutable_column->empty()) - mutable_column->takeDynamicStructureFromColumn(variant_element_state->variant); + mutable_column->takeDynamicStructureFromSourceColumns({variant_element_state->variant}); /// If there was nothing to deserialize or nothing was actually deserialized when variant_limit > 0, just insert defaults. /// The second case means that we don't have a stream for such sub-column. It may happen during ALTER MODIFY column with Variant extension. From ba7d9029f46f2243111aa90f2332035f8954c149 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 26 Nov 2025 18:15:20 +0000 Subject: [PATCH 009/113] Backport #90302 to 25.8: Fix bug in reading subcolumns from JSON in compact parts --- src/DataTypes/Serializations/ISerialization.cpp | 13 ++++++++++--- src/DataTypes/Serializations/ISerialization.h | 2 +- .../Serializations/SerializationArrayOffsets.cpp | 2 +- .../Serializations/SerializationVariant.cpp | 2 +- ...json_compact_part_substreams_cache_bug.reference | 3 +++ ...03722_json_compact_part_substreams_cache_bug.sql | 7 +++++++ 6 files changed, 23 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.reference create mode 100644 tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.sql diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 6cfe5f9c0644..d932033192d2 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -364,7 +364,7 @@ void ISerialization::addElementToSubstreamsCache(ISerialization::SubstreamsCache if (!cache || path.empty()) return; - cache->emplace(getSubcolumnNameForStream(path), std::move(element)); + cache->insert_or_assign(getSubcolumnNameForStream(path), std::move(element)); } ISerialization::ISubstreamsCacheElement * ISerialization::getElementFromSubstreamsCache(ISerialization::SubstreamsCache * cache, const ISerialization::SubstreamPath & path) @@ -614,11 +614,11 @@ bool ISerialization::insertDataFromSubstreamsCacheIfAny(SubstreamsCache * cache, if (!cached_column_with_num_read_rows) return false; - insertDataFromCachedColumn(settings, result_column, cached_column_with_num_read_rows->first, cached_column_with_num_read_rows->second); + insertDataFromCachedColumn(settings, result_column, cached_column_with_num_read_rows->first, cached_column_with_num_read_rows->second, cache); return true; } -void ISerialization::insertDataFromCachedColumn(const ISerialization::DeserializeBinaryBulkSettings & settings, ColumnPtr & result_column, const ColumnPtr & cached_column, size_t num_read_rows) +void ISerialization::insertDataFromCachedColumn(const ISerialization::DeserializeBinaryBulkSettings & settings, ColumnPtr & result_column, const ColumnPtr & cached_column, size_t num_read_rows, SubstreamsCache * cache) { /// Usually substreams cache contains the whole column from currently deserialized block with rows from multiple ranges. /// It's done to avoid extra data copy, in this case we just use this cached column as the result column. @@ -626,9 +626,16 @@ void ISerialization::insertDataFromCachedColumn(const ISerialization::Deserializ /// constructing another column). In this case we need to insert data into resulting column from cached column. /// To determine what case we have we store number of read rows in last range in cache. if ((settings.insert_only_rows_in_current_range_from_substreams_cache) || (result_column != cached_column && !result_column->empty() && cached_column->size() == num_read_rows)) + { result_column->assumeMutable()->insertRangeFrom(*cached_column, cached_column->size() - num_read_rows, num_read_rows); + /// Replace column in the cache with the new column to avoid inserting into it again + /// from currently cached range if this substream will be read again in current range. + addColumnWithNumReadRowsToSubstreamsCache(cache, settings.path, result_column, num_read_rows); + } else + { result_column = cached_column; + } } } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index b48b38243b28..779eb4f6c27b 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -593,7 +593,7 @@ class ISerialization : private boost::noncopyable, public std::enable_shared_fro /// into resulting column and return true, otherwise do nothing and return false. static bool insertDataFromSubstreamsCacheIfAny(SubstreamsCache * cache, const DeserializeBinaryBulkSettings & settings, ColumnPtr & result_column); /// Perform insertion from column found in substreams cache. - static void insertDataFromCachedColumn(const DeserializeBinaryBulkSettings & settings, ColumnPtr & result_column, const ColumnPtr & cached_column, size_t num_read_rows); + static void insertDataFromCachedColumn(const DeserializeBinaryBulkSettings & settings, ColumnPtr & result_column, const ColumnPtr & cached_column, size_t num_read_rows, SubstreamsCache * cache); protected: void addSubstreamAndCallCallback(SubstreamPath & path, const StreamCallback & callback, Substream substream) const; diff --git a/src/DataTypes/Serializations/SerializationArrayOffsets.cpp b/src/DataTypes/Serializations/SerializationArrayOffsets.cpp index 7271ede6df98..9526168dfb2a 100644 --- a/src/DataTypes/Serializations/SerializationArrayOffsets.cpp +++ b/src/DataTypes/Serializations/SerializationArrayOffsets.cpp @@ -26,7 +26,7 @@ void SerializationArrayOffsets::deserializeBinaryBulkWithMultipleStreams( if (rows_offset) column->assumeMutable()->insertRangeFrom(*cached_column, cached_column->size() - num_read_rows, num_read_rows); else - insertDataFromCachedColumn(settings, column, cached_column, num_read_rows); + insertDataFromCachedColumn(settings, column, cached_column, num_read_rows, cache); } else if (ReadBuffer * stream = settings.getter(settings.path)) { diff --git a/src/DataTypes/Serializations/SerializationVariant.cpp b/src/DataTypes/Serializations/SerializationVariant.cpp index 7471cdf10ca8..71e8efcfd5f4 100644 --- a/src/DataTypes/Serializations/SerializationVariant.cpp +++ b/src/DataTypes/Serializations/SerializationVariant.cpp @@ -500,7 +500,7 @@ void SerializationVariant::deserializeBinaryBulkWithMultipleStreams( if (rows_offset) col.getLocalDiscriminatorsPtr()->assumeMutable()->insertRangeFrom(*cached_column, cached_column->size() - num_read_rows, num_read_rows); else - insertDataFromCachedColumn(settings, col.getLocalDiscriminatorsPtr(), cached_column, num_read_rows); + insertDataFromCachedColumn(settings, col.getLocalDiscriminatorsPtr(), cached_column, num_read_rows, cache); num_read_discriminators = num_read_rows; variant_state = checkAndGetState(state); diff --git a/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.reference b/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.reference new file mode 100644 index 000000000000..533dba4b83ca --- /dev/null +++ b/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.reference @@ -0,0 +1,3 @@ +str_0 \N {"b":{"d":0,"e":"str_0"}} +str_1 \N {"b":{"d":1,"e":"str_1"}} +str_2 \N {"b":{"d":2,"e":"str_2"}} diff --git a/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.sql b/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.sql new file mode 100644 index 000000000000..af67b1ed8299 --- /dev/null +++ b/tests/queries/0_stateless/03722_json_compact_part_substreams_cache_bug.sql @@ -0,0 +1,7 @@ +SET use_variant_as_common_type = 1; +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (json Nullable(JSON)) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity=1, write_marks_for_substreams_in_compact_parts=1, min_bytes_for_wide_part='100G'; +INSERT INTO t0 SELECT toJSONString(map('a.b.d', CAST(number, 'UInt32'), 'a.b.e', concat('str_', toString(number)))) FROM numbers(3); +SELECT json.a.b.e, json.a.b.e.:Int64, json.^a FROM t0; +DROP TABLE t0; + From 386d4eed28ff05b6c55b00fb9e2b9342a8032042 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 26 Nov 2025 18:17:48 +0000 Subject: [PATCH 010/113] Backport #87582 to 25.8: Make a separate IColumn method to take exact dynamic structure from a column --- src/Columns/ColumnArray.cpp | 5 ++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnBLOB.h | 1 + src/Columns/ColumnCompressed.h | 1 + src/Columns/ColumnDynamic.cpp | 22 +++++++++++++++ src/Columns/ColumnDynamic.h | 1 + src/Columns/ColumnMap.cpp | 6 ++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 5 ++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnObject.cpp | 28 +++++++++++++++++++ src/Columns/ColumnObject.h | 1 + src/Columns/ColumnSparse.cpp | 5 ++++ src/Columns/ColumnSparse.h | 1 + src/Columns/ColumnTuple.cpp | 7 +++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 8 ++++++ src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 2 ++ .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- 20 files changed, 99 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 1413e54b145e..18e9958147b5 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1419,4 +1419,9 @@ void ColumnArray::takeDynamicStructureFromSourceColumns(const Columns & source_c data->takeDynamicStructureFromSourceColumns(nested_source_columns); } +void ColumnArray::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + data->takeDynamicStructureFromColumn(assert_cast(*source_column).getDataPtr()); +} + } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index d84827eb384a..acadc0163304 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -213,6 +213,7 @@ class ColumnArray final : public COWHelper, ColumnArr bool hasDynamicStructure() const override { return getData().hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; bool dynamicStructureEquals(const IColumn & rhs) const override { diff --git a/src/Columns/ColumnBLOB.h b/src/Columns/ColumnBLOB.h index c3f13e8f019f..7075d145d43f 100644 --- a/src/Columns/ColumnBLOB.h +++ b/src/Columns/ColumnBLOB.h @@ -202,6 +202,7 @@ class ColumnBLOB : public COWHelper, ColumnBLOB> bool hasDynamicStructure() const override { throwInapplicable(); } void takeDynamicStructureFromSourceColumns(const Columns &) override { throwInapplicable(); } + void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwInapplicable(); } private: /// Compressed and serialized representation of the wrapped column. diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 63cc32299301..ba998f3ce27c 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -136,6 +136,7 @@ class ColumnCompressed : public COWHelper, Colum bool hasDynamicStructure() const override { throwMustBeDecompressed(); } void takeDynamicStructureFromSourceColumns(const Columns &) override { throwMustBeDecompressed(); } + void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwMustBeDecompressed(); } protected: size_t rows; diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index a00b1a44ba18..7a4cbd6cc1c3 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1407,6 +1407,28 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source variant_col.getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i]); } +void ColumnDynamic::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + if (!empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromColumn should be called only on empty Dynamic column"); + + const auto & source_dynamic = assert_cast(*source_column); + variant_column = source_dynamic.getVariantColumn().cloneEmpty(); + variant_column_ptr = assert_cast(variant_column.get()); + variant_info = source_dynamic.getVariantInfo(); + statistics = source_dynamic.getStatistics(); + /// Reduce max_dynamic_types to the number of selected variants, so there will be no possibility + /// to extend selected variants on inserts into this column. + /// -1 because we don't count shared variant in the limit. + max_dynamic_types = variant_info.variant_names.size() - 1; + + /// Run takeDynamicStructureFromColumn recursively for variants. + const auto & source_variant_column = source_dynamic.getVariantColumn(); + auto & variant_col = getVariantColumn(); + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + variant_col.getVariantByGlobalDiscriminator(i).takeDynamicStructureFromColumn(source_variant_column.getVariantPtrByGlobalDiscriminator(i)); +} + void ColumnDynamic::applyNullMap(const ColumnVector::Container & null_map) { variant_column_ptr->applyNullMap(null_map); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index f46b6a465287..c83a144693f7 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -386,6 +386,7 @@ class ColumnDynamic final : public COWHelper, Colum bool hasDynamicStructure() const override { return true; } bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; const StatisticsPtr & getStatistics() const { return statistics; } void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 7471b7b6a276..d84dccd201c8 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -451,4 +451,10 @@ void ColumnMap::takeDynamicStructureFromSourceColumns(const Columns & source_col nested->takeDynamicStructureFromSourceColumns(nested_source_columns); } +void ColumnMap::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + nested->takeDynamicStructureFromColumn(assert_cast(*source_column).getNestedColumnPtr()); +} + + } diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 79caab2b07a0..6a1e09def754 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -127,6 +127,7 @@ class ColumnMap final : public COWHelper, ColumnMap> bool hasDynamicStructure() const override { return nested->hasDynamicStructure(); } bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; }; } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 010722971a5d..78521897340d 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -1006,6 +1006,11 @@ void ColumnNullable::takeDynamicStructureFromSourceColumns(const Columns & sourc nested_column->takeDynamicStructureFromSourceColumns(nested_source_columns); } +void ColumnNullable::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + nested_column->takeDynamicStructureFromColumn(assert_cast(*source_column).getNestedColumnPtr()); +} + bool ColumnNullable::dynamicStructureEquals(const IColumn & rhs) const { const auto & rhs_nested_column = assert_cast(rhs).getNestedColumn(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index a0b80c03305c..d2bceca3bbe6 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -232,6 +232,7 @@ class ColumnNullable final : public COWHelper, Col bool hasDynamicStructure() const override { return nested_column->hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; bool dynamicStructureEquals(const IColumn & rhs) const override; private: diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 2d158c4896bf..8aadcba215cd 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1833,6 +1833,34 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou } } +void ColumnObject::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + if (!empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromColumn should be called only on empty Object column"); + + const auto & source_object = assert_cast(*source_column); + + for (auto & [path, column] : typed_paths) + column->takeDynamicStructureFromColumn(source_object.typed_paths.at(path)); + + dynamic_paths.clear(); + dynamic_paths_ptrs.clear(); + sorted_dynamic_paths.clear(); + for (const auto & [path, column] : source_object.getDynamicPaths()) + { + auto it = dynamic_paths.emplace(path, ColumnDynamic::create(max_dynamic_types)).first; + it->second->takeDynamicStructureFromColumn(column); + dynamic_paths_ptrs.emplace(path, assert_cast(it->second.get())); + sorted_dynamic_paths.insert(it->first); + } + + /// Set max_dynamic_paths to the number of dynamic paths. + /// It's needed to avoid adding new unexpected dynamic paths during later inserts into this column. + max_dynamic_paths = dynamic_paths.size(); + + statistics = source_object.getStatistics(); +} + size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end) { /// Simple random access iterator over values in ColumnString in specified range. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index c21b77f07b33..c601941ea4c3 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -204,6 +204,7 @@ class ColumnObject final : public COWHelper, ColumnO bool hasDynamicStructure() const override { return true; } bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; const PathToColumnMap & getTypedPaths() const { return typed_paths; } PathToColumnMap & getTypedPaths() { return typed_paths; } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0eaaae97c165..ee08112738a7 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -895,6 +895,11 @@ void ColumnSparse::takeDynamicStructureFromSourceColumns(const Columns & source_ values->takeDynamicStructureFromSourceColumns(values_source_columns); } +void ColumnSparse::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + values->takeDynamicStructureFromColumn(assert_cast(*source_column).getValuesPtr()); +} + ColumnPtr recursiveRemoveSparse(const ColumnPtr & column) { if (!column) diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index f51982bd2f2f..091054cb5e1d 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -172,6 +172,7 @@ class ColumnSparse final : public COWHelper, ColumnS bool hasDynamicStructure() const override { return values->hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; size_t getNumberOfTrailingDefaults() const { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 8da05b044c91..42922dc3c531 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -898,6 +898,13 @@ void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_c columns[i]->takeDynamicStructureFromSourceColumns(nested_source_columns[i]); } +void ColumnTuple::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + const auto & source_elements = assert_cast(*source_column).getColumns(); + for (size_t i = 0; i != columns.size(); ++i) + columns[i]->takeDynamicStructureFromColumn(source_elements[i]); +} + ColumnPtr ColumnTuple::compress(bool force_compression) const { diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 14758ae6bee5..a0d27e53e166 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -149,6 +149,7 @@ class ColumnTuple final : public COWHelper, ColumnTup bool hasDynamicStructure() const override; bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; /// Empty tuple needs a public method to manage its size. void addSize(size_t delta) { column_length += delta; } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 111a501a739f..b8ee7a028dd5 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1771,4 +1771,12 @@ void ColumnVariant::takeDynamicStructureFromSourceColumns(const Columns & source getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i]); } +void ColumnVariant::takeDynamicStructureFromColumn(const ColumnPtr & source_column) +{ + const auto & source_variant = assert_cast(*source_column); + for (size_t i = 0; i != variants.size(); ++i) + getVariantByGlobalDiscriminator(i).takeDynamicStructureFromColumn(source_variant.getVariantPtrByGlobalDiscriminator(i)); +} + + } diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 8af324755333..5555bc64f4fd 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -345,6 +345,7 @@ class ColumnVariant final : public COWHelper, Colum bool hasDynamicStructure() const override; bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 31ae50a72d22..cda959a27676 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -669,6 +669,8 @@ class IColumn : public COW /// For columns with dynamic subcolumns this method takes dynamic structure from source columns /// and creates proper resulting dynamic structure in advance for merge of these source columns. virtual void takeDynamicStructureFromSourceColumns(const std::vector & /*source_columns*/) {} + /// For columns with dynamic subcolumns this method takes the exact dynamic structure from provided column. + virtual void takeDynamicStructureFromColumn(const ColumnPtr & /*source_column*/) {} /** Some columns can contain another columns inside. * So, we have a tree of columns. But not all combinations are possible. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f870b4d1fe46..f53d53f09cb7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -601,7 +601,7 @@ void MergeTreeDataPartWriterOnDisk::initOrAdjustDynamicStructureIfNeeded(Block & /// and insert data into it. Resulting column will have required dynamic structure and the content /// of the column in current block. auto new_column = sample_column.type->createColumn(); - new_column->takeDynamicStructureFromSourceColumns({sample_column.column}); + new_column->takeDynamicStructureFromColumn(sample_column.column); new_column->insertRangeFrom(*column.column, 0, column.column->size()); column.column = std::move(new_column); } From 8929944b1f92d31020aff503aba8989f633c33c8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Nov 2025 00:10:49 +0000 Subject: [PATCH 011/113] Use takeDynamicStructureFromColumn instead of takeDynamicStructureFromSourceColumns --- src/DataTypes/Serializations/SerializationVariantElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index c62c1f4608a7..b3ef19b987f9 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -253,7 +253,7 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( /// We want to keep dynamic structure of the variant during deserialization. /// Keeping dynamic structure improves performance of insertFrom/insertRangeFrom methods. if (mutable_column->empty()) - mutable_column->takeDynamicStructureFromSourceColumns({variant_element_state->variant}); + mutable_column->takeDynamicStructureFromColumn(variant_element_state->variant); /// If there was nothing to deserialize or nothing was actually deserialized when variant_limit > 0, just insert defaults. /// The second case means that we don't have a stream for such sub-column. It may happen during ALTER MODIFY column with Variant extension. From 89ad94035be54a28582cc311f603962261b47625 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 00:32:35 +0000 Subject: [PATCH 012/113] Backport #90892 to 25.8: Fix background flush of Buffer in case of frequent INSERTs --- src/Storages/StorageBuffer.cpp | 36 +++++++++++++++++++++++++++++----- src/Storages/StorageBuffer.h | 2 ++ 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 668866920cb4..80d009bd755f 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -139,6 +139,11 @@ StoragePtr StorageBuffer::getDestinationTable() const } +std::string StorageBuffer::Thresholds::toString() const +{ + return fmt::format("time={}, rows={}, bytes={}", time, rows, bytes); +} + StorageBuffer::StorageBuffer( const StorageID & table_id_, const ColumnsDescription & columns_, @@ -183,6 +188,8 @@ StorageBuffer::StorageBuffer( num_shards, 0, num_shards); } flush_handle = bg_pool.createTask(log->name() + "/Bg", [this]{ backgroundFlush(); }); + + LOG_TRACE(log, "Buffer(flush: ({}), min: ({}), max: ({}))", flush_thresholds.toString(), min_thresholds.toString(), max_thresholds.toString()); } @@ -1108,7 +1115,8 @@ void StorageBuffer::backgroundFlush() void StorageBuffer::reschedule() { time_t min_first_write_time = std::numeric_limits::max(); - time_t rows = 0; + size_t rows = 0; + size_t processed_buffers = 0; for (auto & buffer : buffers) { @@ -1124,6 +1132,7 @@ void StorageBuffer::reschedule() std::unique_lock lock(buffer.tryLock()); if (lock.owns_lock()) { + ++processed_buffers; if (!buffer.data.empty()) { min_first_write_time = std::min(min_first_write_time, buffer.first_write_time); @@ -1134,15 +1143,32 @@ void StorageBuffer::reschedule() /// will be rescheduled via INSERT if (!rows) + { + LOG_TRACE(log, "Skipping reschedule (processed buffers: {})", processed_buffers); return; + } time_t current_time = time(nullptr); time_t time_passed = current_time - min_first_write_time; - size_t min = std::max(min_thresholds.time - time_passed, 1); - size_t max = std::max(max_thresholds.time - time_passed, 1); - size_t flush = std::max(flush_thresholds.time - time_passed, 1); - flush_handle->scheduleAfter(std::min({min, max, flush}) * 1000); + size_t min = std::max(min_thresholds.time - time_passed, 0); + size_t max = std::max(max_thresholds.time - time_passed, 0); + size_t reschedule_sec = 0; + if (flush_thresholds.time) + { + size_t flush = std::max(flush_thresholds.time - time_passed, 0); + reschedule_sec = std::min({min, max, flush}); + } + else + { + reschedule_sec = std::min({min, max}); + } + /// Schedule flush in background immediately, otherwise in case of frequent INSERTs we will never schedule the background flush + if (reschedule_sec == 0) + flush_handle->schedule(); + else + flush_handle->scheduleAfter(reschedule_sec * 1000); + LOG_TRACE(log, "Reschedule in {} sec (processed buffers: {}, rows in processed buffers: {}, time passed: {})", reschedule_sec, processed_buffers, rows, time_passed); } void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index d68fb1ee1a95..77c227b3a357 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -52,6 +52,8 @@ friend class BufferSink; time_t time = 0; /// The number of seconds from the insertion of the first row into the block. size_t rows = 0; /// The number of rows in the block. size_t bytes = 0; /// The number of (uncompressed) bytes in the block. + + std::string toString() const; }; /** num_shards - the level of internal parallelism (the number of independent buffers) From e689ac5a5f3bf62bf6f8e460fe2f005f699fff01 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 02:43:47 +0000 Subject: [PATCH 013/113] Backport #90860 to 25.8: Fix ALTER MODIFY QUERY with dictGet() and dictionary name in Common Subexpresseion Elimination for Shared Catalog --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 20 ++++++++++++++++++- ...er_modify_query_dict_name_in_cse.reference | 0 ...40_alter_modify_query_dict_name_in_cse.sql | 10 ++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.reference create mode 100644 tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.sql diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp index 38c9ef89adc4..d9a2ae248873 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.cpp +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -116,7 +116,6 @@ void ApplyWithSubqueryVisitor::visit(ASTTableExpression & table, const Data & da void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data) { /// Special CTE case, where the right argument of IN is alias (ASTIdentifier) from WITH clause. - if (checkFunctionIsInOrGlobalInOperator(func)) { auto & ast = func.arguments->children.at(1); @@ -150,6 +149,25 @@ void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data) } } } + /// Rewrite dictionary name in dictGet*() + else if (functionIsDictGet(func.name)) + { + auto & dict_name_arg = func.arguments->children.at(0); + if (const auto * identifier = dict_name_arg->as(); identifier && identifier->isShort()) + { + LOG_TRACE(getLogger("visitor"), "identifier = {}", identifier->formatForLogging()); + auto name = identifier->shortName(); + + auto literal_it = data.literals.find(name); + if (literal_it != data.literals.end()) + { + auto old_alias = dict_name_arg->tryGetAlias(); + dict_name_arg = literal_it->second->clone(); + /// Always reset the alias name, otherwise the aliases will not match after AddDefaultDatabaseVisitor + dict_name_arg->setAlias(old_alias); + } + } + } } } diff --git a/tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.reference b/tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.sql b/tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.sql new file mode 100644 index 000000000000..568d8850bbf9 --- /dev/null +++ b/tests/queries/0_stateless/03740_alter_modify_query_dict_name_in_cse.sql @@ -0,0 +1,10 @@ +drop table if exists mv; +drop table if exists dst; +drop table if exists src; +drop dictionary if exists dict; + +create table src (key Int) engine=MergeTree order by (); +create table dst (key Int) engine=MergeTree order by (); +create dictionary dict (key Int, value Int) primary key key layout(direct) source(clickhouse(query 'select 0 key, 0 value')); +create materialized view mv to dst as select * from src; +alter table mv modify query with 'dict' as dict_name select dictGetInt32(dict_name, 'value', key) from src; From 0d6f28de95e7cd72f667adab5fd27824cca9129c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 13:21:38 +0000 Subject: [PATCH 014/113] Backport #90412 to 25.8: Fix renaming and modifying the same Nested column in one ALTER --- src/Storages/AlterCommands.cpp | 1 + .../03727_rename_nested_and_modify_in_one_later.reference | 0 .../03727_rename_nested_and_modify_in_one_later.sql | 5 +++++ 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.reference create mode 100644 tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e7b53c887eb5..14b6fd6134ab 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1697,6 +1697,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (from_nested_table_name != to_nested_table_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot rename column from one nested name to another"); + all_columns.rename(command.column_name, command.rename_to); } else if (!from_nested && !to_nested) { diff --git a/tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.reference b/tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.sql b/tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.sql new file mode 100644 index 000000000000..926fce19f92e --- /dev/null +++ b/tests/queries/0_stateless/03727_rename_nested_and_modify_in_one_later.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (`c0.c1` Array(Int)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t0 VALUES ([1]); +ALTER TABLE t0 (RENAME COLUMN `c0.c1` TO `c0.c2`), (MODIFY COLUMN `c0.c1` MODIFY SETTING max_compress_block_size = 1); -- {serverError NOT_FOUND_COLUMN_IN_BLOCK} +DROP TABLE t0; From d04cb989e1caea2e7d8099cd04c8c3c327bf4bde Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 16:13:39 +0000 Subject: [PATCH 015/113] Backport #90951 to 25.8: Revert "Fix(Hive): Fix thread-safety race condition in static extractor" --- src/Storages/HivePartitioningUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HivePartitioningUtils.cpp b/src/Storages/HivePartitioningUtils.cpp index 27a59aae044c..2dbb792ef84d 100644 --- a/src/Storages/HivePartitioningUtils.cpp +++ b/src/Storages/HivePartitioningUtils.cpp @@ -34,7 +34,7 @@ static auto makeExtractor() HivePartitioningKeysAndValues parseHivePartitioningKeysAndValues(const String & path) { - thread_local auto extractor = makeExtractor(); + static auto extractor = makeExtractor(); HivePartitioningKeysAndValues key_values; From f72477bad05a85f613c42d755db82e6636f8c093 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 19:11:56 +0000 Subject: [PATCH 016/113] Backport #89090 to 25.8: remove try/catch from `~PooledConnection` --- src/Common/HTTPConnectionPool.cpp | 59 ++++++++++++++----------------- 1 file changed, 26 insertions(+), 33 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a6db50c05455..33a85b3a7c09 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -438,45 +438,38 @@ class EndpointConnectionPool : public std::enable_shared_from_this(response_stream)) { - if (auto * fixed_steam = dynamic_cast(response_stream)) - { - response_stream_completed = fixed_steam->isComplete(); - } - else if (auto * chunked_steam = dynamic_cast(response_stream)) - { - response_stream_completed = chunked_steam->isComplete(); - } - else if (auto * http_stream = dynamic_cast(response_stream)) - { - response_stream_completed = http_stream->isComplete(); - } - else - { - response_stream_completed = false; - } + response_stream_completed = fixed_steam->isComplete(); } - response_stream = nullptr; - Session::setSendDataHooks(); - Session::setReceiveDataHooks(); - Session::setSendThrottler(); - Session::setReceiveThrottler(); + else if (auto * chunked_steam = dynamic_cast(response_stream)) + { + response_stream_completed = chunked_steam->isComplete(); + } + else if (auto * http_stream = dynamic_cast(response_stream)) + { + response_stream_completed = http_stream->isComplete(); + } + else + { + response_stream_completed = false; + } + } + response_stream = nullptr; + Session::setSendDataHooks(); + Session::setReceiveDataHooks(); + Session::setSendThrottler(); + Session::setReceiveThrottler(); - group->atConnectionDestroy(); + group->atConnectionDestroy(); - if (!isExpired) - if (auto lock = pool.lock()) - lock->atConnectionDestroy(*this); + if (!isExpired) + if (auto lock = pool.lock()) + lock->atConnectionDestroy(*this); - CurrentMetrics::sub(metrics.active_count); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + CurrentMetrics::sub(metrics.active_count); } private: From 3a68871dd0d7dbca38a774722f4f7bc9e2c3ec13 Mon Sep 17 00:00:00 2001 From: rienath Date: Thu, 27 Nov 2025 19:20:52 +0000 Subject: [PATCH 017/113] Change connection retries from 10 to 500 --- src/IO/S3/PocoHTTPClient.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index db8f8ea415aa..42c515897994 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -46,7 +46,7 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration { struct RetryStrategy { - unsigned int max_retries = 10; + unsigned int max_retries = 500; unsigned int initial_delay_ms = 25; unsigned int max_delay_ms = 5000; double jitter_factor = 0; From a00d957dc92c58b2d15822e4530a8da353d61b1e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 20:13:13 +0000 Subject: [PATCH 018/113] Backport #90505 to 25.8: Support output of LowCardinality(Bool/Date32) in Arrow format --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 4 +++- .../03728_arrow_low_cardinality_over_bool.reference | 1 + .../0_stateless/03728_arrow_low_cardinality_over_bool.sh | 8 ++++++++ ..._arrow_low_cardinality_over_bool_and_date32.reference | 2 ++ .../03728_arrow_low_cardinality_over_bool_and_date32.sh | 9 +++++++++ 5 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.reference create mode 100755 tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.sh create mode 100644 tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.reference create mode 100755 tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.sh diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 6d76a105e96d..402ada7a491e 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -41,6 +41,7 @@ M(Float64, arrow::DoubleBuilder) #define FOR_ARROW_TYPES(M) \ + M(BOOL, arrow::BooleanType) \ M(UINT8, arrow::UInt8Type) \ M(INT8, arrow::Int8Type) \ M(UINT16, arrow::UInt16Type) \ @@ -53,7 +54,8 @@ M(DOUBLE, arrow::DoubleType) \ M(BINARY, arrow::BinaryType) \ M(STRING, arrow::StringType) \ - M(FIXED_SIZE_BINARY, arrow::FixedSizeBinaryType) + M(FIXED_SIZE_BINARY, arrow::FixedSizeBinaryType) \ + M(DATE32, arrow::Date32Type) namespace DB { diff --git a/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.reference b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.reference new file mode 100644 index 000000000000..27ba77ddaf61 --- /dev/null +++ b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.reference @@ -0,0 +1 @@ +true diff --git a/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.sh b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.sh new file mode 100755 index 000000000000..2926a8053168 --- /dev/null +++ b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select true::LowCardinality(Bool) as a format Arrow settings allow_suspicious_low_cardinality_types = 1, output_format_arrow_low_cardinality_as_dictionary = 1" | $CLICKHOUSE_LOCAL -q "select * from table" --input-format=Arrow diff --git a/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.reference b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.reference new file mode 100644 index 000000000000..2841005f5171 --- /dev/null +++ b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.reference @@ -0,0 +1,2 @@ +true +2020-01-01 diff --git a/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.sh b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.sh new file mode 100755 index 000000000000..82ac7a1e8370 --- /dev/null +++ b/tests/queries/0_stateless/03728_arrow_low_cardinality_over_bool_and_date32.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select true::LowCardinality(Bool) as a format Arrow settings allow_suspicious_low_cardinality_types = 1, output_format_arrow_low_cardinality_as_dictionary = 1" | $CLICKHOUSE_LOCAL -q "select * from table" --input-format=Arrow +$CLICKHOUSE_LOCAL -q "select '2020-01-01'::LowCardinality(Date32) as a format Arrow settings allow_suspicious_low_cardinality_types = 1, output_format_arrow_low_cardinality_as_dictionary = 1" | $CLICKHOUSE_LOCAL -q "select * from table" --input-format=Arrow From daf4247ff47ec0093469e64884fa995776c7070b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 21:12:13 +0000 Subject: [PATCH 019/113] Backport #90907 to 25.8: Fix high memory usage during reading JSON/Dynamic/Variant columns --- src/Columns/ColumnDynamic.h | 5 +++++ src/Columns/ColumnObject.cpp | 9 +++++++++ src/Columns/ColumnObject.h | 1 + src/Columns/ColumnVariant.cpp | 11 +++++++++++ src/Columns/ColumnVariant.h | 1 + 5 files changed, 27 insertions(+) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index c83a144693f7..f4e5d6989284 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -282,6 +282,11 @@ class ColumnDynamic final : public COWHelper, Colum /// Prepare only variants but not discriminators and offsets. void prepareVariantsForSquashing(const Columns & source_columns, size_t factor); + void shrinkToFit() override + { + variant_column_ptr->shrinkToFit(); + } + void ensureOwnership() override { variant_column_ptr->ensureOwnership(); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index ea1cabd0ac68..03f031d4e562 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1328,6 +1328,15 @@ size_t ColumnObject::capacity() const return shared_data->capacity(); } +void ColumnObject::shrinkToFit() +{ + for (auto & [_, column] : typed_paths) + column->shrinkToFit(); + for (auto & [_, column] : dynamic_paths_ptrs) + column->shrinkToFit(); + shared_data->shrinkToFit(); +} + void ColumnObject::ensureOwnership() { for (auto & [_, column] : typed_paths) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 1197872e0de7..35190711908b 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -176,6 +176,7 @@ class ColumnObject final : public COWHelper, ColumnO void reserve(size_t n) override; size_t capacity() const override; void prepareForSquashing(const std::vector & source_columns, size_t factor) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index b8ee7a028dd5..6cb706f13295 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1392,8 +1392,19 @@ size_t ColumnVariant::capacity() const return local_discriminators->capacity(); } +void ColumnVariant::shrinkToFit() +{ + offsets->shrinkToFit(); + local_discriminators->shrinkToFit(); + const size_t num_variants = variants.size(); + for (size_t i = 0; i < num_variants; ++i) + getVariantByLocalDiscriminator(i).shrinkToFit(); +} + void ColumnVariant::ensureOwnership() { + offsets->ensureOwnership(); + local_discriminators->ensureOwnership(); const size_t num_variants = variants.size(); for (size_t i = 0; i < num_variants; ++i) getVariantByLocalDiscriminator(i).ensureOwnership(); diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 5555bc64f4fd..8d91fb018c22 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -248,6 +248,7 @@ class ColumnVariant final : public COWHelper, Colum void reserve(size_t n) override; size_t capacity() const override; void prepareForSquashing(const Columns & source_columns, size_t factor) override; + void shrinkToFit() override; void ensureOwnership() override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; From 284f5133b54daf43fc2c6eb1e0a554cceaffde3b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Nov 2025 21:14:39 +0000 Subject: [PATCH 020/113] Backport #90430 to 25.8: Fix parsing JSON/Dynamic/Variant values from HTTP parameters --- .../ReplaceQueryParameterVisitor.cpp | 38 +++++++++++++++++-- ...amic_variant_from_http_parameter.reference | 4 ++ ...son_dynamic_variant_from_http_parameter.sh | 13 +++++++ 3 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.reference create mode 100755 tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.sh diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 240da7a9af65..51d05824927c 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -81,6 +82,28 @@ const String & ReplaceQueryParameterVisitor::getParamValue(const String & name) return search->second; } +namespace +{ + +/// Return true if we cannot use cast from Field for this type and need to use cast from String +bool needCastFromString(const DataTypePtr & type) +{ + if (type->getCustomSerialization()) + return true; + + bool result = false; + auto check = [&](const IDataType & t) + { + result |= isVariant(t) || isDynamic(t) || isObject(t); + }; + + check(*type); + type->forEachChild(check); + return result; +} + +} + void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) { const auto & ast_param = ast->as(); @@ -115,12 +138,19 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) value, type_name, ast_param.name, read_buffer.count(), value.size(), value.substr(0, read_buffer.count())); Field literal; - /// If data type has custom serialization, we should use CAST from String, - /// because CAST from field may not work correctly (for example for type IPv6). - if (data_type->getCustomSerialization()) - literal = value; + + /// For some data types we should use CAST from String, + /// because CAST from field may not work correctly (for example for type IPv6, JSON, Dynamic, etc). + if (needCastFromString(data_type)) + { + WriteBufferFromOwnString value_buf; + serialization->serializeText(temp_column, 0, value_buf, format_settings); + literal = value_buf.str(); + } else + { literal = temp_column[0]; + } /// If it's a String, substitute it in the form of a string literal without CAST /// to enable substitutions in simple queries that don't support expressions diff --git a/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.reference b/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.reference new file mode 100644 index 000000000000..0563b74b7756 --- /dev/null +++ b/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.reference @@ -0,0 +1,4 @@ +{"a":1} {"a":1} 1 {'a':'Int64'} {'a':'Int64'} +1 1 1 String String +1.1 1.1 1 Float32 Float32 +'Hello' String diff --git a/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.sh b/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.sh new file mode 100755 index 000000000000..871444c0b027 --- /dev/null +++ b/tests/queries/0_stateless/03728_parsing_json_dynamic_variant_from_http_parameter.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-parallel-replicas +# no-parallel-replicas: the query from query_log errors due to missing columns. + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}¶m_x=%7B%22a%22%3A1%7D" -d "SELECT {x:Json} as j1, '{\"a\":1}'::Json as j2, j1 = j2 as equals, JSONAllPathsWithTypes(j1), JSONAllPathsWithTypes(j2)"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}¶m_x=1" -d "SELECT {x:Dynamic} as d1, 1::Dynamic as d2, d1 = d2 as equals, dynamicType(d1), dynamicType(d2)"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}¶m_x=1.1" -d "SELECT {x:Variant(Float32, String)} as v1, 1.1::Variant(Float32, String) as v2, v1 = v2 as equals, variantType(v1), variantType(v2)"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}¶m_x=%5C%27Hello%5C%27" -d "SELECT {x:Dynamic} as d, dynamicType(d) format Raw"; From 3c97191d0bbbe0779de616fcdebf11a2ca9556f9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Nov 2025 01:40:24 +0000 Subject: [PATCH 021/113] Backport #91014 to 25.8: Fix one more bug in JSON when path can be both in shared data and dynamic paths --- src/DataTypes/DataTypeObject.cpp | 1 + src/DataTypes/Serializations/SerializationSubObject.cpp | 3 +++ ..._dynamic_paths_and_shared_data_compact_part_bug.reference | 0 ...ath_in_dynamic_paths_and_shared_data_compact_part_bug.sql | 5 +++++ 4 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.reference create mode 100644 tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 4ff113c0481e..923ee858e1e2 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -360,6 +360,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu if (path.starts_with(prefix)) result_dynamic_paths.emplace_back(path.substr(prefix.size()), column); } + result_object_column.setMaxDynamicPaths(result_dynamic_paths.size()); result_object_column.setDynamicPaths(result_dynamic_paths); const auto & shared_data_offsets = object_column.getSharedDataOffsets(); diff --git a/src/DataTypes/Serializations/SerializationSubObject.cpp b/src/DataTypes/Serializations/SerializationSubObject.cpp index 61611490d6be..8c855a61f733 100644 --- a/src/DataTypes/Serializations/SerializationSubObject.cpp +++ b/src/DataTypes/Serializations/SerializationSubObject.cpp @@ -187,7 +187,10 @@ void SerializationSubObject::deserializeBinaryBulkWithMultipleStreams( auto & column_object = assert_cast(*mutable_column); /// If it's a new object column, set dynamic paths and statistics. if (column_object.empty()) + { + column_object.setMaxDynamicPaths(sub_object_state->dynamic_sub_paths.size()); column_object.setDynamicPaths(sub_object_state->dynamic_sub_paths); + } auto & typed_paths = column_object.getTypedPaths(); auto & dynamic_paths = column_object.getDynamicPaths(); diff --git a/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.reference b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.sql b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.sql new file mode 100644 index 000000000000..aa33cc217b5a --- /dev/null +++ b/tests/queries/0_stateless/03732_json_duplicated_path_in_dynamic_paths_and_shared_data_compact_part_bug.sql @@ -0,0 +1,5 @@ +drop table if exists test; +create table test (id UInt64, json JSON(max_dynamic_paths=1)) engine=MergeTree order by tuple() settings min_bytes_for_wide_part='100G', write_marks_for_substreams_in_compact_parts=0; +insert into test select number, '{"a" : 42, "b" : {"c" : 42}}' from numbers(100000); +select json.^b from test order by id format Null; +drop table test; From 6639ee280d75864d021e86cbbb9537a1ea608097 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Nov 2025 01:42:08 +0000 Subject: [PATCH 022/113] Backport #90947 to 25.8: Fix infinite loop in AzureObjectStorage::listObjects --- .../AzureBlobStorage/AzureObjectStorage.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 83ca6950da69..de6c777b7aa2 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -184,7 +184,6 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); - blob_list_response = client_ptr->ListBlobs(options); const auto & blobs_list = blob_list_response.Blobs; for (const auto & blob : blobs_list) @@ -200,13 +199,8 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith {}})); } - if (max_keys) - { - ssize_t keys_left = static_cast(max_keys) - children.size(); - if (keys_left <= 0) - break; - options.PageSizeHint = keys_left; - } + if (max_keys && children.size() >= max_keys) + break; } } From e3e62979255972df3cbc1d63e56f8e0565a3e99f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Nov 2025 07:12:44 +0000 Subject: [PATCH 023/113] Backport #91022 to 25.8: Fix std::out_of_range in ApplyWithSubqueryVisitor with dictGet --- src/Interpreters/ApplyWithSubqueryVisitor.cpp | 3 +-- ...41_dict_get_in_cte_with_no_arguments_old_analyzer.reference | 0 .../03741_dict_get_in_cte_with_no_arguments_old_analyzer.sql | 2 ++ 3 files changed, 3 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.reference create mode 100644 tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.sql diff --git a/src/Interpreters/ApplyWithSubqueryVisitor.cpp b/src/Interpreters/ApplyWithSubqueryVisitor.cpp index d9a2ae248873..8e370a5bc208 100644 --- a/src/Interpreters/ApplyWithSubqueryVisitor.cpp +++ b/src/Interpreters/ApplyWithSubqueryVisitor.cpp @@ -150,12 +150,11 @@ void ApplyWithSubqueryVisitor::visit(ASTFunction & func, const Data & data) } } /// Rewrite dictionary name in dictGet*() - else if (functionIsDictGet(func.name)) + else if (functionIsDictGet(func.name) && !func.arguments->children.empty()) { auto & dict_name_arg = func.arguments->children.at(0); if (const auto * identifier = dict_name_arg->as(); identifier && identifier->isShort()) { - LOG_TRACE(getLogger("visitor"), "identifier = {}", identifier->formatForLogging()); auto name = identifier->shortName(); auto literal_it = data.literals.find(name); diff --git a/tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.reference b/tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.sql b/tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.sql new file mode 100644 index 000000000000..3e2fb2443d8a --- /dev/null +++ b/tests/queries/0_stateless/03741_dict_get_in_cte_with_no_arguments_old_analyzer.sql @@ -0,0 +1,2 @@ +SELECT ( SELECT dictGet() ) settings enable_analyzer=0; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + From a3cac081f831956286f865b4adc2a977b574d347 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Nov 2025 16:13:29 +0000 Subject: [PATCH 024/113] Backport #91000 to 25.8: Fix excessive Buffer flushes (burns CPU and produce tons of logs) --- src/Storages/StorageBuffer.cpp | 18 +++++---- src/Storages/StorageBuffer.h | 4 +- .../03735_excessive_buffer_flush.reference | 5 +++ .../03735_excessive_buffer_flush.sql | 40 +++++++++++++++++++ 4 files changed, 59 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/03735_excessive_buffer_flush.reference create mode 100644 tests/queries/0_stateless/03735_excessive_buffer_flush.sql diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 80d009bd755f..21a5e11f52df 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -95,7 +95,6 @@ namespace ErrorCodes extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; } - std::unique_lock StorageBuffer::Buffer::lockForReading() const { return lockImpl(/* read= */true); @@ -738,7 +737,7 @@ class BufferSink : public SinkToStorage, WithContext insertIntoBuffer(block, *least_busy_buffer, metadata_snapshot->metadata_version); least_busy_lock.unlock(); - storage.reschedule(); + storage.reschedule(0); } private: StorageBuffer & storage; @@ -1109,10 +1108,10 @@ void StorageBuffer::backgroundFlush() tryLogCurrentException(__PRETTY_FUNCTION__); } - reschedule(); + reschedule(BACKGROUND_RESCHEDULE_MIN_DELAY); } -void StorageBuffer::reschedule() +void StorageBuffer::reschedule(size_t min_delay) { time_t min_first_write_time = std::numeric_limits::max(); size_t rows = 0; @@ -1151,12 +1150,17 @@ void StorageBuffer::reschedule() time_t current_time = time(nullptr); time_t time_passed = current_time - min_first_write_time; - size_t min = std::max(min_thresholds.time - time_passed, 0); - size_t max = std::max(max_thresholds.time - time_passed, 0); + /// checkThresholdsImpl() uses strict comparison (> not >=), so we need to add offset to original time values + static constexpr time_t THRESHOLD_COMPARISON_OFFSET = 1; + + /// For minimal threshold min_delay is ignored, since otherwise it will be triggered too frequently, once it is reached + /// (while the Buffer cannot be flushed due to other min thresholds). + size_t min = std::max(min_thresholds.time + THRESHOLD_COMPARISON_OFFSET - time_passed, 1); + size_t max = std::max(max_thresholds.time + THRESHOLD_COMPARISON_OFFSET - time_passed, min_delay); size_t reschedule_sec = 0; if (flush_thresholds.time) { - size_t flush = std::max(flush_thresholds.time - time_passed, 0); + size_t flush = std::max(flush_thresholds.time + THRESHOLD_COMPARISON_OFFSET - time_passed, min_delay); reschedule_sec = std::min({min, max, flush}); } else diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 77c227b3a357..d8de9b92922c 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -184,12 +184,14 @@ friend class BufferSink; void writeBlockToDestination(const Block & block, StoragePtr table); void backgroundFlush(); - void reschedule(); + void reschedule(size_t min_delay); StoragePtr getDestinationTable() const; BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder flush_handle; + + static constexpr size_t BACKGROUND_RESCHEDULE_MIN_DELAY = 1; }; } diff --git a/tests/queries/0_stateless/03735_excessive_buffer_flush.reference b/tests/queries/0_stateless/03735_excessive_buffer_flush.reference new file mode 100644 index 000000000000..cdd48f6eb519 --- /dev/null +++ b/tests/queries/0_stateless/03735_excessive_buffer_flush.reference @@ -0,0 +1,5 @@ +buffer_flush_by_flush_time 10 +buffer_flush_by_max 10 +buffer_flush_by_min 10 +empty_buffer 10 +empty_buffer_zero_time 10 diff --git a/tests/queries/0_stateless/03735_excessive_buffer_flush.sql b/tests/queries/0_stateless/03735_excessive_buffer_flush.sql new file mode 100644 index 000000000000..4b32e912ba65 --- /dev/null +++ b/tests/queries/0_stateless/03735_excessive_buffer_flush.sql @@ -0,0 +1,40 @@ +-- Note, this test uses sleep, but, it should not affect it's flakiness + +set function_sleep_max_microseconds_per_block=5e9; + +drop table if exists data; +create table data (key Int) engine=Null; + +drop table if exists empty_buffer; +create table empty_buffer (key Int) engine=Buffer(currentDatabase(), data, 2, 2, 4, 100_000, 1_000_000, 10e9, 10e9, 3); +select sleep(5) format Null; +optimize table empty_buffer; +drop table empty_buffer; + +drop table if exists empty_buffer_zero_time; +create table empty_buffer_zero_time (key Int) engine=Buffer(currentDatabase(), data, 2, 0, 0, 100_000, 1_000_000, 10e9, 10e9, 0); +select sleep(1) format Null; +optimize table empty_buffer_zero_time; +drop table empty_buffer_zero_time; + +drop table if exists buffer_flush_by_min; +create table buffer_flush_by_min (key Int) engine=Buffer(currentDatabase(), data, 2, 2, 4, 100_000, 1_000_000, 0, 10e9, 3); +insert into buffer_flush_by_min select * from numbers(100_000 + 1); +select sleep(5) format Null; +drop table buffer_flush_by_min; + +drop table if exists buffer_flush_by_max; +create table buffer_flush_by_max (key Int) engine=Buffer(currentDatabase(), data, 2, 2, 4, 100_000, 1_000_000, 0, 10e9); +insert into buffer_flush_by_max select * from numbers(1); +select sleep(5) format Null; +drop table buffer_flush_by_max; + +drop table if exists buffer_flush_by_flush_time; +create table buffer_flush_by_flush_time (key Int) engine=Buffer(currentDatabase(), data, 2, 2, 4, 100_000, 1_000_000, 10e9, 10e9, 3); +insert into buffer_flush_by_flush_time values (1); +select sleep(5) format Null; +drop table buffer_flush_by_flush_time; + +system flush logs text_log; +-- to avoid flakiness we only check that number of logs < 10, instead of some strict values +select extractAll(logger_name, 'StorageBuffer \\([^.]+\\.([^)]+)\\)')[1] as table_name, max2(count(), 10) from system.text_log where logger_name LIKE format('%StorageBuffer ({}.%', currentDatabase()) group by 1 order by 1; From ac3bc697e18ff8175ed676acb9783f9e7fea2fcb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Nov 2025 20:12:23 +0000 Subject: [PATCH 025/113] Backport #91094 to 25.8: Fix CI (due to tmpfs permissions changes in runc 1.3.3) --- ci/defs/job_configs.py | 5 +++-- tests/ci/stress_check.py | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/ci/defs/job_configs.py b/ci/defs/job_configs.py index 8cf465351790..64f88b14ad09 100644 --- a/ci/defs/job_configs.py +++ b/ci/defs/job_configs.py @@ -28,7 +28,7 @@ command='python3 ./ci/jobs/functional_tests.py --options "{PARAMETER}"', # some tests can be flaky due to very slow disks - use tmpfs for temporary ClickHouse files # --cap-add=SYS_PTRACE and --privileged for gdb in docker - run_in_docker=f"clickhouse/stateless-test+--memory={LIMITED_MEM}+--cap-add=SYS_PTRACE+--privileged+--security-opt seccomp=unconfined+--tmpfs /tmp/clickhouse+--volume=./ci/tmp/var/lib/clickhouse:/var/lib/clickhouse+--volume=./ci/tmp/etc/clickhouse-client:/etc/clickhouse-client+--volume=./ci/tmp/etc/clickhouse-server:/etc/clickhouse-server+--volume=./ci/tmp/etc/clickhouse-server1:/etc/clickhouse-server1+--volume=./ci/tmp/etc/clickhouse-server2:/etc/clickhouse-server2+--volume=./ci/tmp/var/log:/var/log", + run_in_docker=f"clickhouse/stateless-test+--memory={LIMITED_MEM}+--cap-add=SYS_PTRACE+--privileged+--security-opt seccomp=unconfined+--tmpfs /tmp/clickhouse:mode=1777+--volume=./ci/tmp/var/lib/clickhouse:/var/lib/clickhouse+--volume=./ci/tmp/etc/clickhouse-client:/etc/clickhouse-client+--volume=./ci/tmp/etc/clickhouse-server:/etc/clickhouse-server+--volume=./ci/tmp/etc/clickhouse-server1:/etc/clickhouse-server1+--volume=./ci/tmp/etc/clickhouse-server2:/etc/clickhouse-server2+--volume=./ci/tmp/var/log:/var/log", digest_config=Job.CacheDigestConfig( include_paths=[ "./ci/jobs/functional_tests.py", @@ -39,6 +39,7 @@ "./tests/config", "./tests/*.txt", "./ci/docker/stateless-test", + "./ci/defs/job_configs.py", ], ), result_name_for_cidb="Tests", @@ -352,7 +353,7 @@ class JobConfigs: runs_on=RunnerLabels.FUNC_TESTER_ARM, command="python3 ./ci/jobs/functional_tests.py --options BugfixValidation", # some tests can be flaky due to very slow disks - use tmpfs for temporary ClickHouse files - run_in_docker="clickhouse/stateless-test+--network=host+--security-opt seccomp=unconfined+--tmpfs /tmp/clickhouse", + run_in_docker="clickhouse/stateless-test+--network=host+--security-opt seccomp=unconfined+--tmpfs /tmp/clickhouse:mode=1777", digest_config=Job.CacheDigestConfig( include_paths=[ "./ci/jobs/functional_tests.py", diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 73707c488c8b..f77433ca7297 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -72,7 +72,7 @@ def get_run_command( "--privileged " # a static link, don't use S3_URL or S3_DOWNLOAD "-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets' " - "--tmpfs /tmp/clickhouse " + "--tmpfs /tmp/clickhouse:mode=1777 " f"--volume={build_path}:/package_folder " f"--volume={result_path}:/test_output " f"--volume={repo_tests_path}/..:/repo " From bb8b7f8a24f17e9158645f1ab2ff07e84684d86a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Dec 2025 09:17:31 +0000 Subject: [PATCH 026/113] Backport #91159 to 25.8: Improve DDLDependencyVisitor --- src/Databases/DDLDependencyVisitor.cpp | 44 ++++++++++++-------------- 1 file changed, 21 insertions(+), 23 deletions(-) diff --git a/src/Databases/DDLDependencyVisitor.cpp b/src/Databases/DDLDependencyVisitor.cpp index 9e6cae8bf172..7a3382d73276 100644 --- a/src/Databases/DDLDependencyVisitor.cpp +++ b/src/Databases/DDLDependencyVisitor.cpp @@ -378,34 +378,32 @@ namespace const auto & arg = args[arg_idx]; - if (evaluate) + if (const auto * id = arg->as()) + return id->name(); + + if (const auto * literal = arg->as()) { - try - { - /// We're just searching for dependencies here, it's not safe to execute subqueries now. - /// Use copy of the global_context and set current database, because expressions can contain currentDatabase() function. - ContextMutablePtr global_context_copy = Context::createCopy(global_context); - global_context_copy->setCurrentDatabase(current_database); - auto evaluated = evaluateConstantExpressionOrIdentifierAsLiteral(arg, global_context_copy); - const auto * literal = evaluated->as(); - if (!literal || (literal->value.getType() != Field::Types::String)) - return {}; + if (literal->value.getType() == Field::Types::String) return literal->value.safeGet(); - } - catch (...) - { + } + + if (!evaluate) + return {}; + + try + { + /// We're just searching for dependencies here, it's not safe to execute subqueries now. + /// Use copy of the global_context and set current database, because expressions can contain currentDatabase() function. + ContextMutablePtr global_context_copy = Context::createCopy(global_context); + global_context_copy->setCurrentDatabase(current_database); + auto evaluated = evaluateConstantExpressionOrIdentifierAsLiteral(arg, global_context_copy); + const auto * literal = evaluated->as(); + if (!literal || (literal->value.getType() != Field::Types::String)) return {}; - } + return literal->value.safeGet(); } - else + catch (...) { - if (const auto * id = arg->as()) - return id->name(); - if (const auto * literal = arg->as()) - { - if (literal->value.getType() == Field::Types::String) - return literal->value.safeGet(); - } return {}; } } From 7e7f17c0a72c9159978609e21ea103c8c1999be7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Dec 2025 17:14:28 +0000 Subject: [PATCH 027/113] Backport #90922 to 25.8: Fix `isGrantedWildcard` for partial revoke. --- src/Access/AccessRights.cpp | 3 +- src/Access/tests/gtest_access_rights_ops.cpp | 469 +++++++++++++++++++ 2 files changed, 471 insertions(+), 1 deletion(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index f2b60053e402..29bf46479cf9 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -499,7 +499,8 @@ struct AccessRights::Node /// / \ /// "" (leaf, USAGE) "bar" (SELECT) const auto & [node, _] = tryGetLeafOrPrefix(name, /* return_parent_node= */ true); - return node.flags.contains(flags_to_check); + /// Check min_flags_with_children because wildcard allows to grant for all children. + return node.min_flags_with_children.contains(flags_to_check); } const auto & [node, final] = tryGetLeafOrPrefix(name); diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 01190e8ec7ff..43006f0f8b26 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -546,3 +546,472 @@ TEST(AccessRights, Filter) res = root.getFilters("URL"); ASSERT_EQ(res.size(), 0); } + +TEST(AccessRights, ParialRevokeWithGrantOption) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.revoke(AccessType::SELECT, "default", "zookeeper"); + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "default", "zoo")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "default", "zoo")); + + root = {}; + root.grantWithGrantOption(AccessType::SELECT); + root.revoke(AccessType::SELECT, "default", "zookeeper"); + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "default", "zoo")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "default", "zoo")); +} + +TEST(AccessRights, WildcardGrantEdgeCases) +{ + AccessRights root; + root.grantWildcard(AccessType::SELECT, "a"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "abc")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "a")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "bcd")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON a*.*"); + + root = {}; + root.grantWildcard(AccessType::SELECT, "test"); + root.grant(AccessType::INSERT, "test"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "testing")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "test")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "testing")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON test*.*, GRANT INSERT ON test.*"); + + root = {}; + root.grantWildcard(AccessType::SELECT, "prod"); + root.grantWildcard(AccessType::INSERT, "production"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "production")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "product")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "prod")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "product")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "production")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "production_v2")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "db", "tbl"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl_backup")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl123")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "other")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db2", "tbl")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "db", "tbl", "col"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl", "col")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl", "col_id")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl", "column")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "tbl", "other")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "tbl")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "sys"); + root.grantWildcard(AccessType::INSERT, "sys", "log"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "sys", "any_table", "any_col")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "system", "tables")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "sys", "log_events")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "sys", "data")); +} + +TEST(AccessRights, PartialRevokeWithWildcard) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.revokeWildcard(AccessType::SELECT, "system"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "production")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system_logs")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON *.*, REVOKE SELECT ON system*.*"); + + root = {}; + root.grantWildcard(AccessType::SELECT, "prod"); + root.revoke(AccessType::SELECT, "production"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "product")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "production")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON prod*.*, REVOKE SELECT ON production.*"); + + root = {}; + root.grantWildcard(AccessType::SELECT, "test"); + root.revokeWildcard(AccessType::SELECT, "testing"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "test_env")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "testing")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "testing_v2")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON test*.*, REVOKE SELECT ON testing*.*"); + + root = {}; + root.grantWildcard(AccessType::SELECT, "db"); + root.revoke(AccessType::SELECT, "db", "secret"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "public")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db123", "any")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "secret")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON db*.*, REVOKE SELECT ON db.secret"); + + root = {}; + root.grant(AccessType::SELECT); + root.revokeWildcard(AccessType::SELECT, "secret"); + root.revokeWildcard(AccessType::SELECT, "private"); + root.revoke(AccessType::SELECT, "system"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "public")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "secret")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "secret_data")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "private")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "private_logs")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "system_public")); + + root = {}; + root.grant(AccessType::SELECT); + root.revokeWildcard(AccessType::SELECT, "internal"); + root.grant(AccessType::SELECT, "internal_public"); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "internal")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "internal_data")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "internal_public")); + ASSERT_EQ(root.toString(), "GRANT SELECT ON *.*, REVOKE SELECT ON internal*.*, GRANT SELECT ON internal_public.*"); +} + +TEST(AccessRights, WildcardGrantOptionInteractions) +{ + AccessRights root; + root.grantWildcardWithGrantOption(AccessType::SELECT, "db"); + root.revokeGrantOption(AccessType::SELECT, "db", "sensitive"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "sensitive")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db", "public")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "db", "sensitive")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db123", "any")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "prod"); + root.grantWithGrantOption(AccessType::SELECT, "production"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "production")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "prod")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "production")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "product")); + + root = {}; + root.grantWildcardWithGrantOption(AccessType::SELECT, "test"); + root.revokeWildcardGrantOption(AccessType::SELECT, "testing"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "testing")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "test")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "testing")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "testing_v2")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "test_env")); + + root = {}; + root.grantWildcardWithGrantOption(AccessType::INSERT, "analytics"); + ASSERT_TRUE(root.hasGrantOptionWildcard(AccessType::INSERT, "analytics")); + ASSERT_TRUE(root.hasGrantOptionWildcard(AccessType::INSERT, "analytics_v2")); + ASSERT_FALSE(root.hasGrantOptionWildcard(AccessType::INSERT, "other")); + + root = {}; + root.grantWildcardWithGrantOption(AccessType::SELECT, "db"); + root.revokeWildcardGrantOption(AccessType::SELECT, "db", "tbl"); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db", "other")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db123", "any")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "db", "tbl")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "db", "tbl_backup")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "tbl")); +} + +TEST(AccessRights, UnionWithPartialRevokes) +{ + AccessRights lhs; + AccessRights rhs; + + lhs.grant(AccessType::SELECT); + lhs.revoke(AccessType::SELECT, "secret"); + rhs.grant(AccessType::SELECT, "secret", "public_table"); + lhs.makeUnion(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "secret", "public_table")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "secret")); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + lhs.revokeWildcard(AccessType::SELECT, "sys"); + rhs.grantWildcard(AccessType::SELECT, "system"); + lhs.makeUnion(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "system")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "system_logs")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "sys")); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "prod"); + rhs.grantWildcard(AccessType::INSERT, "prod"); + lhs.makeUnion(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "production")); + ASSERT_TRUE(lhs.isGranted(AccessType::INSERT, "production")); + ASSERT_EQ(lhs.toString(), "GRANT SELECT, INSERT ON prod*.*"); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "test"); + rhs.grantWildcard(AccessType::SELECT, "testing"); + lhs.makeUnion(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "testing")); + ASSERT_EQ(lhs.toString(), "GRANT SELECT ON test*.*"); +} + +TEST(AccessRights, IntersectionWithPartialRevokes) +{ + AccessRights lhs; + AccessRights rhs; + + lhs.grant(AccessType::SELECT); + lhs.revoke(AccessType::SELECT, "secret"); + rhs.grant(AccessType::SELECT); + rhs.revoke(AccessType::SELECT, "private"); + lhs.makeIntersection(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "secret")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "private")); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "test"); + rhs.grant(AccessType::SELECT, "test"); + rhs.grant(AccessType::SELECT, "testing"); + lhs.makeIntersection(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "testing")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "test_env")); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "prod"); + rhs.grantWildcard(AccessType::SELECT, "production"); + lhs.makeIntersection(rhs); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "prod")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "product")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "production")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "production_v2")); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + rhs.grant(AccessType::SELECT); + rhs.revokeWildcard(AccessType::SELECT, "sys"); + lhs.makeIntersection(rhs); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "sys")); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "system")); +} + +TEST(AccessRights, DifferenceWithPartialRevokes) +{ + AccessRights lhs; + AccessRights rhs; + + lhs.grant(AccessType::SELECT); + rhs.grant(AccessType::SELECT); + rhs.revoke(AccessType::SELECT, "secret"); + lhs.makeDifference(rhs); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "secret")); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "test"); + rhs.grant(AccessType::SELECT, "test"); + lhs.makeDifference(rhs); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "testing")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "test_env")); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + rhs.grant(AccessType::SELECT); + rhs.revokeWildcard(AccessType::SELECT, "internal"); + lhs.makeDifference(rhs); + ASSERT_FALSE(lhs.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "internal")); + ASSERT_TRUE(lhs.isGranted(AccessType::SELECT, "internal_data")); +} + +TEST(AccessRights, ContainsWithWildcardsAndPartialRevokes) +{ + AccessRights lhs; + AccessRights rhs; + + lhs.grantWildcard(AccessType::SELECT, "test"); + rhs.grant(AccessType::SELECT, "testing"); + ASSERT_TRUE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "test"); + rhs.grant(AccessType::SELECT, "prod"); + ASSERT_FALSE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + lhs.revoke(AccessType::SELECT, "secret"); + rhs.grant(AccessType::SELECT, "default"); + ASSERT_TRUE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + lhs.revoke(AccessType::SELECT, "secret"); + rhs.grant(AccessType::SELECT, "secret"); + ASSERT_FALSE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + lhs.revokeWildcard(AccessType::SELECT, "sys"); + rhs.grant(AccessType::SELECT, "default"); + rhs.grant(AccessType::SELECT, "prod"); + ASSERT_TRUE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grant(AccessType::SELECT); + lhs.revokeWildcard(AccessType::SELECT, "sys"); + rhs.grant(AccessType::SELECT, "system"); + ASSERT_FALSE(lhs.contains(rhs)); + + lhs = {}; + rhs = {}; + lhs.grantWildcard(AccessType::SELECT, "testing"); + rhs.grantWildcard(AccessType::SELECT, "test"); + ASSERT_FALSE(lhs.contains(rhs)); +} + +TEST(AccessRights, ColumnLevelWildcardOperations) +{ + AccessRights root; + root.grant(AccessType::SELECT, "db", "users"); + root.revokeWildcard(AccessType::SELECT, "db", "users", "secret"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "users", "name")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "users", "email")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "users", "secret")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "users", "secret_key")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "users")); + + root = {}; + root.grant(AccessType::SELECT, "db", "table"); + root.revokeWildcard(AccessType::SELECT, "db", "table", "private"); + root.revokeWildcard(AccessType::SELECT, "db", "table", "secret"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "public_col")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "table", "private_data")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "table", "secret_key")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "db", "table", "col"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "col")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "col_id")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "column")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "table", "other")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "table")); + + root = {}; + root.grantWildcardWithGrantOption(AccessType::SELECT, "db", "table", "visible"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "visible")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "table", "visible_data")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db", "table", "visible")); + ASSERT_TRUE(root.hasGrantOption(AccessType::SELECT, "db", "table", "visible_data")); + ASSERT_FALSE(root.hasGrantOption(AccessType::SELECT, "db", "table", "hidden")); +} + +TEST(AccessRights, ComplexWildcardScenarios) +{ + AccessRights root; + root.grant(AccessType::ALL); + root.revokeWildcard(AccessType::SELECT, "internal"); + root.grantWildcard(AccessType::SELECT, "internal_public"); + root.revoke(AccessType::SELECT, "internal_public_secret"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "default")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "internal")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "internal_data")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "internal_public")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "internal_public_api")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "internal_public_secret")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "prod"); + root.revokeWildcard(AccessType::SELECT, "prod", "secret"); + root.grant(AccessType::SELECT, "prod", "secret_allowed"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "production", "users")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "prod", "secret_data")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod", "secret_allowed")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod", "public")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "analytics"); + root.grantWildcard(AccessType::INSERT, "analytics_write"); + root.revoke(AccessType::SELECT, "analytics_read_only", "sensitive"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "analytics", "any")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "analytics_read_only", "normal")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "analytics_read_only", "sensitive")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "analytics_write", "data")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "analytics", "data")); + + root = {}; + root.grantWildcard(AccessType::SELECT, ""); + root.revokeWildcard(AccessType::SELECT, "a"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "b")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "z")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "a")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "abc")); + + root = {}; + root.grant(AccessType::SELECT); + root.revoke(AccessType::SELECT); + root.grantWildcard(AccessType::SELECT, "allowed"); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "default")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "allowed")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "allowed_db")); +} + +TEST(AccessRights, PartialRevokePropagation) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.revoke(AccessType::SELECT, "db1", "secret"); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db1", "secret")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db1", "public")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db2", "secret")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db2")); + + root = {}; + root.grant(AccessType::SELECT); + root.revokeWildcard(AccessType::SELECT, "temp"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "permanent")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "temp")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "temp1")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "temporary")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "team")); + + root = {}; + root.grant(AccessType::SELECT, "db"); + root.revoke(AccessType::SELECT, "db", "t1", "col1"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "t1", "col2")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "db", "t2")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "t1", "col1")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "db", "t1")); + + root = {}; + root.grant(AccessType::SELECT); + root.grant(AccessType::INSERT); + root.revokeWildcard(AccessType::SELECT, "readonly"); + root.revokeWildcard(AccessType::INSERT, "writeonly"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "normal")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "normal")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "readonly")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "readonly")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "writeonly")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "writeonly")); +} From 3c4fdbe746ff0c456c85124eb7b9b14da49963f5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Dec 2025 20:13:32 +0000 Subject: [PATCH 028/113] Backport #91208 to 25.8: Respect prefixes deserialization thread pool setting --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85e1e56d0241..3d0570d0acb2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -89,6 +90,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool columns_and_secondary_indices_sizes_lazy_calculation; } +namespace Setting +{ + extern const SettingsBool merge_tree_use_prefixes_deserialization_thread_pool; +} + namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; @@ -2655,7 +2661,8 @@ ColumnPtr IMergeTreeDataPart::getColumnSample(const NameAndTypePair & column) co settings.can_read_part_without_marks = true; /// Use prefixes deserialization thread pool to read prefixes faster. /// In JSON type there might be hundreds of small files that needs to be read. - settings.use_prefixes_deserialization_thread_pool = true; + /// Set this to value from storage settings rather than true by default as this could cause starvation on huge JSON. + settings.use_prefixes_deserialization_thread_pool = storage.getContext()->getSettingsRef()[Setting::merge_tree_use_prefixes_deserialization_thread_pool]; auto alter_conversions = std::make_shared(); auto part_info = std::make_shared(shared_from_this(), alter_conversions); From eb8b97e05c946eebae6cf98034cd30329bb63a05 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Dec 2025 20:17:20 +0000 Subject: [PATCH 029/113] Backport #91151 to 25.8: Fix merging JSON columns in Summing/Aggregating/Coalescing MergeTree --- .../Algorithms/AggregatingSortedAlgorithm.cpp | 7 ++-- .../Algorithms/SummingSortedAlgorithm.cpp | 8 +++-- ...ating_merge_tree_with_json_merge.reference | 9 +++++ ...aggregating_merge_tree_with_json_merge.sql | 34 +++++++++++++++++++ 4 files changed, 53 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.reference create mode 100644 tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.sql diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index f9c5d1c5e815..958f3303570b 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -152,9 +152,10 @@ void AggregatingSortedAlgorithm::AggregatingMergedData::initialize(const DB::Blo for (const auto & desc : def.columns_to_simple_aggregate) { - const auto & type = desc.nested_type ? desc.nested_type - : desc.real_type; - columns[desc.column_number] = type->createColumn(); + /// Remove LowCardinality from columns if needed. It's important to use columns initialized in + /// MergedData::initialize to keep correct dynamic structure of some columns (like JSON/Dynamic). + if (desc.nested_type) + columns[desc.column_number] = recursiveRemoveLowCardinality(std::move(columns[desc.column_number]))->assumeMutable(); } initAggregateDescription(); diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index aa9b0525931d..468e05e53835 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -613,8 +613,12 @@ void SummingSortedAlgorithm::SummingMergedData::initialize(const DB::Block & hea } else { - const auto & type = desc.nested_type ? desc.nested_type : desc.real_type; - new_columns.emplace_back(type->createColumn()); + /// Remove LowCardinality from columns if needed. It's important to use columns initialized in + /// MergedData::initialize to keep correct dynamic structure of some columns (like JSON/Dynamic). + if (desc.nested_type) + new_columns.emplace_back(recursiveRemoveLowCardinality(std::move(columns[desc.column_numbers[0]]))->assumeMutable()); + else + new_columns.emplace_back(std::move(columns[desc.column_numbers[0]])); } } diff --git a/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.reference b/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.reference new file mode 100644 index 000000000000..2b76d346bc7f --- /dev/null +++ b/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.reference @@ -0,0 +1,9 @@ +Dynamic paths +a +Shared data parhs +Dynamic paths +a +Shared data parhs +Dynamic paths +a +Shared data parhs diff --git a/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.sql b/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.sql new file mode 100644 index 000000000000..94a2d3aa9106 --- /dev/null +++ b/tests/queries/0_stateless/03743_summing_and_aggregating_merge_tree_with_json_merge.sql @@ -0,0 +1,34 @@ +set mutations_sync=1; + +drop table if exists test; +create table test (id UInt64, json JSON) engine=SummingMergeTree order by id settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, index_granularity=32, merge_max_block_size=32; +insert into test select number, '{}' from numbers(100); +alter table test update json = '{"a" : 42}' where id > 90; +optimize table test final; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) from test; +select 'Shared data parhs'; +select distinct arrayJoin(JSONSharedDataPaths(json)) from test; +drop table test; + +create table test (id UInt64, json JSON) engine=AggregatingMergeTree order by id settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, index_granularity=32, merge_max_block_size=32; +insert into test select number, '{}' from numbers(100); +alter table test update json = '{"a" : 42}' where id > 90; +optimize table test final; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) from test; +select 'Shared data parhs'; +select distinct arrayJoin(JSONSharedDataPaths(json)) from test; +drop table test; + +create table test (id UInt64, json JSON) engine=CoalescingMergeTree order by id settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, index_granularity=32, merge_max_block_size=32; +insert into test select number, '{}' from numbers(100); +alter table test update json = '{"a" : 42}' where id > 90; +optimize table test final; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(json)) from test; +select 'Shared data parhs'; +select distinct arrayJoin(JSONSharedDataPaths(json)) from test; +drop table test; + + From 33bea7a992f686ad200ebd5261216ca1969df1a5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Dec 2025 12:18:28 +0000 Subject: [PATCH 030/113] Backport #90909 to 25.8: Fix base58Decode function --- src/Functions/FunctionBase58Conversion.h | 11 ++++++++--- .../0_stateless/03733_base58_decode_bug.reference | 0 tests/queries/0_stateless/03733_base58_decode_bug.sql | 1 + 3 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03733_base58_decode_bug.reference create mode 100644 tests/queries/0_stateless/03733_base58_decode_bug.sql diff --git a/src/Functions/FunctionBase58Conversion.h b/src/Functions/FunctionBase58Conversion.h index fe0d32aa1f7f..3a5c5e1a718c 100644 --- a/src/Functions/FunctionBase58Conversion.h +++ b/src/Functions/FunctionBase58Conversion.h @@ -29,9 +29,14 @@ struct Base58DecodeTraits template static size_t getBufferSize(Col const & src_column) { - auto const string_length = src_column.getChars().size(); - /// decoded size is at most length of encoded (every 8 bytes becomes at most 6 bytes) - return (string_length * 6 + 7) / 8; + /// According to the RFC https://datatracker.ietf.org/doc/html/draft-msporny-base58-03 + /// base58 doesn't have a clean bitsequence-to-character mapping like base32 or base64. + /// Instead, it uses division by 58 and modulo operations on big integers. + /// In addition all the leading zeros are converted to "1"s as is. + /// Thus, if we decode the can have at most same amount of bytes as a result. + /// Example: + /// "11111" (5 chars) -> b'\x00\x00\x00\x00\x00' (5 bytes) + return src_column.getChars().size(); } static std::optional perform(std::string_view src, UInt8 * dst) diff --git a/tests/queries/0_stateless/03733_base58_decode_bug.reference b/tests/queries/0_stateless/03733_base58_decode_bug.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03733_base58_decode_bug.sql b/tests/queries/0_stateless/03733_base58_decode_bug.sql new file mode 100644 index 000000000000..b484dc74c6c2 --- /dev/null +++ b/tests/queries/0_stateless/03733_base58_decode_bug.sql @@ -0,0 +1 @@ +SELECT base58Decode(materialize('11111111111111')) FROM numbers(100000) FORMAT Null; From ad6d8441b5e9b0880915437a2acd8f1dc4f74ac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Dec 2025 14:18:36 +0000 Subject: [PATCH 031/113] Backport #90880 to 25.8: Add setting for compatibility serialziation of string --- .../AggregateFunctionGroupArray.cpp | 3 +- .../AggregateFunctionGroupArrayIntersect.cpp | 6 +- .../AggregateFunctionTopK.cpp | 3 +- .../Combinators/AggregateFunctionDistinct.h | 8 +- src/AggregateFunctions/KeyHolderHelpers.h | 6 +- src/Columns/ColumnAggregateFunction.cpp | 5 +- src/Columns/ColumnAggregateFunction.h | 4 +- src/Columns/ColumnArray.cpp | 50 ++------ src/Columns/ColumnArray.h | 10 +- src/Columns/ColumnBLOB.h | 6 +- src/Columns/ColumnCompressed.h | 6 +- src/Columns/ColumnConst.h | 20 ++-- src/Columns/ColumnDecimal.cpp | 2 +- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnDynamic.cpp | 4 +- src/Columns/ColumnDynamic.h | 6 +- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnFixedString.h | 2 +- src/Columns/ColumnFunction.h | 4 +- src/Columns/ColumnLazy.cpp | 2 +- src/Columns/ColumnLazy.h | 2 +- src/Columns/ColumnLowCardinality.cpp | 28 ++--- src/Columns/ColumnLowCardinality.h | 10 +- src/Columns/ColumnMap.cpp | 26 ++--- src/Columns/ColumnMap.h | 10 +- src/Columns/ColumnNullable.cpp | 48 ++------ src/Columns/ColumnNullable.h | 10 +- src/Columns/ColumnObject.cpp | 34 +----- src/Columns/ColumnObject.h | 8 +- src/Columns/ColumnSparse.cpp | 26 ++--- src/Columns/ColumnSparse.h | 10 +- src/Columns/ColumnString.cpp | 83 ++++++-------- src/Columns/ColumnString.h | 14 +-- src/Columns/ColumnTuple.cpp | 52 ++------- src/Columns/ColumnTuple.h | 10 +- src/Columns/ColumnUnique.h | 35 +++--- src/Columns/ColumnVariant.cpp | 52 ++------- src/Columns/ColumnVariant.h | 10 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.cpp | 63 +++++----- src/Columns/IColumn.h | 67 ++++++----- src/Columns/IColumnDummy.cpp | 4 +- src/Columns/IColumnDummy.h | 4 +- src/Columns/IColumnUnique.h | 4 +- src/Columns/tests/gtest_column_dynamic.cpp | 84 ++++++-------- src/Columns/tests/gtest_column_object.cpp | 18 +-- src/Columns/tests/gtest_column_unique.cpp | 8 +- src/Common/ColumnsHashing.h | 39 +++++-- src/Common/ColumnsHashingImpl.h | 1 + src/Core/Settings.cpp | 3 + src/Core/SettingsChangesHistory.cpp | 3 +- src/Dictionaries/DictionaryHelpers.cpp | 2 +- src/Dictionaries/DictionaryHelpers.h | 4 +- src/Dictionaries/SSDCacheDictionaryStorage.h | 2 +- src/Functions/array/arrayIntersect.cpp | 6 +- src/Interpreters/AggregationCommon.h | 4 +- src/Interpreters/AggregationMethod.cpp | 34 +++++- src/Interpreters/AggregationMethod.h | 22 ++-- src/Interpreters/Aggregator.cpp | 21 +++- src/Interpreters/Aggregator.h | 8 +- src/Interpreters/InterpreterSelectQuery.cpp | 9 +- src/Planner/Planner.cpp | 7 +- src/Processors/QueryPlan/AggregatingStep.cpp | 5 +- .../QueryPlan/MergingAggregatedStep.cpp | 4 +- .../QueryPlanSerializationSettings.cpp | 4 +- .../TTL/TTLAggregationAlgorithm.cpp | 3 +- .../Transforms/MergingAggregatedTransform.cpp | 3 +- .../__init__.py | 0 .../test.py | 108 ++++++++++++++++++ 70 files changed, 575 insertions(+), 592 deletions(-) create mode 100644 tests/integration/test_string_aggregation_compatibility/__init__.py create mode 100644 tests/integration/test_string_aggregation_compatibility/test.py diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 2371477c084a..6e0d144efb4a 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -457,7 +457,8 @@ struct GroupArrayNodeGeneral : public GroupArrayNodeBase static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) { const char * begin = arena->alignedAlloc(sizeof(Node), alignof(Node)); - StringRef value = column.serializeAggregationStateValueIntoArena(row_num, *arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto value = column.serializeValueIntoArena(row_num, *arena, begin, &settings); Node * node = reinterpret_cast(const_cast(begin)); node->size = value.size; diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 6305add32200..de5225225648 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -236,7 +236,8 @@ class AggregateFunctionGroupArrayIntersectGeneric final else { const char * begin = nullptr; - StringRef serialized = data_column->serializeAggregationStateValueIntoArena(offset + i, *arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin, &settings); chassert(serialized.data != nullptr); set.emplace(SerializedKeyHolder{serialized, *arena}, it, inserted); } @@ -256,7 +257,8 @@ class AggregateFunctionGroupArrayIntersectGeneric final else { const char * begin = nullptr; - StringRef serialized = data_column->serializeAggregationStateValueIntoArena(offset + i, *arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto serialized = data_column->serializeValueIntoArena(offset + i, *arena, begin, &settings); chassert(serialized.data != nullptr); it = set.find(serialized); diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 0500466d0be8..0cd48eca0a1d 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -310,7 +310,8 @@ class AggregateFunctionTopKGeneric final else { const char * begin = nullptr; - StringRef str_serialized = columns[0]->serializeAggregationStateValueIntoArena(row_num, *arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto str_serialized = columns[0]->serializeValueIntoArena(row_num, *arena, begin, &settings); if constexpr (is_weighted) set.insert(str_serialized, columns[1]->getUInt(row_num)); else diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h index 99507db6b12b..eedf13a7281e 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h @@ -116,9 +116,10 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi StringRef value(begin, 0); for (size_t i = 0; i < columns_num; ++i) { - auto cur_ref = columns[i]->serializeAggregationStateValueIntoArena(row_num, *arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto cur_ref = columns[i]->serializeValueIntoArena(row_num, *arena, begin, &settings); value.data = cur_ref.data - value.size; - value.size += cur_ref.size; + value.size += cur_ref.size } Set::LookupResult it; @@ -140,8 +141,9 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi history.emplace(ArenaKeyHolder{value, *arena}, it, inserted); ReadBufferFromString in({it->getValue().data, it->getValue().size}); /// Multiple columns are serialized one by one + auto settings = IColumn::SerializationSettings::createForAggregationState(); for (auto & column : argument_columns) - column->deserializeAndInsertAggregationStateValueFromArena(in); + column->deserializeAndInsertFromArena(in, &settings); } } } diff --git a/src/AggregateFunctions/KeyHolderHelpers.h b/src/AggregateFunctions/KeyHolderHelpers.h index d1a78e855c0b..2cf0dd318ef1 100644 --- a/src/AggregateFunctions/KeyHolderHelpers.h +++ b/src/AggregateFunctions/KeyHolderHelpers.h @@ -23,7 +23,8 @@ static auto getKeyHolder(const IColumn & column, size_t row_num, Arena & arena) else { const char * begin = nullptr; - StringRef serialized = column.serializeAggregationStateValueIntoArena(row_num, arena, begin); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + auto serialized = column.serializeValueIntoArena(row_num, arena, begin, &settings); assert(serialized.data != nullptr); return SerializedKeyHolder{serialized, arena}; } @@ -37,7 +38,8 @@ static void deserializeAndInsert(StringRef str, IColumn & data_to) else { ReadBufferFromString in({str.data, str.size}); - data_to.deserializeAndInsertAggregationStateValueFromArena(in); + auto settings = IColumn::SerializationSettings::createForAggregationState(); + data_to.deserializeAndInsertFromArena(in, &settings); if (!in.eof()) { throw Exception(ErrorCodes::INCORRECT_DATA, "Extra bytes ({}) found after deserializing aggregation state", in.available()); diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 461a391501ca..70deb3d6ce92 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -601,7 +601,8 @@ void ColumnAggregateFunction::insertDefault() pushBackAndCreateState(data, arena, func.get()); } -StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const +StringRef ColumnAggregateFunction::serializeValueIntoArena( + size_t n, Arena & arena, const char *& begin, const IColumn::SerializationSettings *) const { WriteBufferFromArena out(arena, begin); func->serialize(data[n], out, version); @@ -609,7 +610,7 @@ StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & are return out.complete(); } -void ColumnAggregateFunction::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnAggregateFunction::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { ensureOwnership(); diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index ab17001d677b..e64ba35f112c 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -171,9 +171,9 @@ class ColumnAggregateFunction final : public COWHelper ColumnArray::getSerializedValueSize(size_t n) const +std::optional ColumnArray::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { const auto & offsets_data = getOffsets(); @@ -287,7 +266,7 @@ std::optional ColumnArray::getSerializedValueSize(size_t n) const size_t res = sizeof(offsets_data[0]); for (; pos < end; ++pos) { - auto element_size = getData().getSerializedValueSize(pos); + auto element_size = getData().getSerializedValueSize(pos, settings); if (!element_size) return std::nullopt; res += *element_size; @@ -297,24 +276,13 @@ std::optional ColumnArray::getSerializedValueSize(size_t n) const } -void ColumnArray::deserializeAndInsertFromArena(ReadBuffer & in) -{ - size_t array_size; - readBinaryLittleEndian(array_size, in); - - for (size_t i = 0; i < array_size; ++i) - getData().deserializeAndInsertFromArena(in); - - getOffsets().push_back(getOffsets().back() + array_size); -} - -void ColumnArray::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnArray::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { size_t array_size; readBinaryLittleEndian(array_size, in); for (size_t i = 0; i < array_size; ++i) - getData().deserializeAndInsertAggregationStateValueFromArena(in); + getData().deserializeAndInsertFromArena(in, settings); getOffsets().push_back(getOffsets().back() + array_size); } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index acadc0163304..9cf091f9c409 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -78,12 +78,10 @@ class ColumnArray final : public COWHelper, ColumnArr StringRef getDataAt(size_t n) const override; bool isDefaultAt(size_t n) const override; void insertData(const char * pos, size_t length) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnBLOB.h b/src/Columns/ColumnBLOB.h index 7075d145d43f..c994c8bfa575 100644 --- a/src/Columns/ColumnBLOB.h +++ b/src/Columns/ColumnBLOB.h @@ -161,9 +161,9 @@ class ColumnBLOB : public COWHelper, ColumnBLOB> void insertData(const char *, size_t) override { throwInapplicable(); } void insertDefault() override { throwInapplicable(); } void popBack(size_t) override { throwInapplicable(); } - StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override { throwInapplicable(); } - char * serializeValueIntoMemory(size_t, char *) const override { throwInapplicable(); } - void deserializeAndInsertFromArena(ReadBuffer &) override { throwInapplicable(); } + StringRef serializeValueIntoArena(size_t, Arena &, char const *&, const IColumn::SerializationSettings *) const override { throwInapplicable(); } + char * serializeValueIntoMemory(size_t, char *, const IColumn::SerializationSettings *) const override { throwInapplicable(); } + void deserializeAndInsertFromArena(ReadBuffer &, const IColumn::SerializationSettings *) override { throwInapplicable(); } void skipSerializedInArena(ReadBuffer &) const override { throwInapplicable(); } void updateHashWithValue(size_t, SipHash &) const override { throwInapplicable(); } WeakHash32 getWeakHash32() const override { throwInapplicable(); } diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index ba998f3ce27c..3e14d6dd00b7 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -97,9 +97,9 @@ class ColumnCompressed : public COWHelper, Colum void insertData(const char *, size_t) override { throwMustBeDecompressed(); } void insertDefault() override { throwMustBeDecompressed(); } void popBack(size_t) override { throwMustBeDecompressed(); } - StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override { throwMustBeDecompressed(); } - char * serializeValueIntoMemory(size_t, char *) const override { throwMustBeDecompressed(); } - void deserializeAndInsertFromArena(ReadBuffer &) override { throwMustBeDecompressed(); } + StringRef serializeValueIntoArena(size_t, Arena &, char const *&, const IColumn::SerializationSettings *) const override { throwMustBeDecompressed(); } + char * serializeValueIntoMemory(size_t, char *, const IColumn::SerializationSettings *) const override { throwMustBeDecompressed(); } + void deserializeAndInsertFromArena(ReadBuffer &, const IColumn::SerializationSettings *) override { throwMustBeDecompressed(); } void skipSerializedInArena(ReadBuffer &) const override { throwMustBeDecompressed(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeDecompressed(); } WeakHash32 getWeakHash32() const override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 95c696d586db..f8aed534e64c 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -174,26 +174,20 @@ class ColumnConst final : public COWHelper, ColumnCon s -= n; } - StringRef serializeValueIntoArena(size_t, Arena & arena, char const *& begin) const override + StringRef + serializeValueIntoArena(size_t, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override { - return data->serializeValueIntoArena(0, arena, begin); + return data->serializeValueIntoArena(0, arena, begin, settings); } - char * serializeValueIntoMemory(size_t, char * memory) const override + char * serializeValueIntoMemory(size_t, char * memory, const IColumn::SerializationSettings * settings) const override { - return data->serializeValueIntoMemory(0, memory); + return data->serializeValueIntoMemory(0, memory, settings); } - void deserializeAndInsertFromArena(ReadBuffer & in) override + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override { - data->deserializeAndInsertFromArena(in); - data->popBack(1); - ++s; - } - - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override - { - data->deserializeAndInsertAggregationStateValueFromArena(in); + data->deserializeAndInsertFromArena(in, settings); data->popBack(1); ++s; } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index cb37ca914a29..5542e229d0da 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -78,7 +78,7 @@ Float64 ColumnDecimal::getFloat64(size_t n) const } template -void ColumnDecimal::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnDecimal::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { T dec; readBinaryLittleEndian(dec, in); diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 72943a696d51..0660b4f90681 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -103,7 +103,7 @@ class ColumnDecimal final : public COWHelper, Col Float64 getFloat64(size_t n) const final; - void deserializeAndInsertFromArena(ReadBuffer & in) override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 7a4cbd6cc1c3..4ad7cee1641a 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -727,7 +727,7 @@ void ColumnDynamic::serializeValueIntoSharedVariant( shared_variant.getOffsets().push_back(chars.size()); } -StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const +StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin, const IColumn::SerializationSettings *) const { /// We cannot use Variant serialization here as it serializes discriminator + value, /// but Dynamic doesn't have fixed mapping discriminator <-> variant type @@ -771,7 +771,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const return res; } -void ColumnDynamic::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnDynamic::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { auto & variant_col = getVariantColumn(); UInt8 null_bit; diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index f4e5d6989284..38375b65f1ea 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -193,10 +193,10 @@ class ColumnDynamic final : public COWHelper, Colum variant_column_ptr->popBack(n); } - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; - std::optional getSerializedValueSize(size_t) const override { return std::nullopt; } + std::optional getSerializedValueSize(size_t, const IColumn::SerializationSettings *) const override { return std::nullopt; } void updateHashWithValue(size_t n, SipHash & hash) const override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 0954922542a6..68a9d0bb7488 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -119,7 +119,7 @@ void ColumnFixedString::insertData(const char * pos, size_t length) memset(chars.data() + old_size + length, 0, n - length); } -void ColumnFixedString::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnFixedString::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { size_t old_size = chars.size(); chars.resize(old_size + n); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 795bed3dd2d6..eda8171912b4 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -137,7 +137,7 @@ class ColumnFixedString final : public COWHelper, Col throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot insert into {}", getName()); } - StringRef serializeValueIntoArena(size_t, Arena &, char const *&) const override + StringRef serializeValueIntoArena(size_t, Arena &, char const *&, const IColumn::SerializationSettings *) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot serialize from {}", getName()); } - void deserializeAndInsertFromArena(ReadBuffer &) override + void deserializeAndInsertFromArena(ReadBuffer &, const IColumn::SerializationSettings *) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot deserialize to {}", getName()); } diff --git a/src/Columns/ColumnLazy.cpp b/src/Columns/ColumnLazy.cpp index d01e63ac05c9..10e6b2270ce7 100644 --- a/src/Columns/ColumnLazy.cpp +++ b/src/Columns/ColumnLazy.cpp @@ -152,7 +152,7 @@ void ColumnLazy::popBack(size_t) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method popBack is not supported for {}", getName()); } -void ColumnLazy::deserializeAndInsertFromArena(ReadBuffer &) +void ColumnLazy::deserializeAndInsertFromArena(ReadBuffer &, const IColumn::SerializationSettings *) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method deserializeAndInsertFromArena is not supported for {}", getName()); } diff --git a/src/Columns/ColumnLazy.h b/src/Columns/ColumnLazy.h index 2a4826c264b7..b9cf6564b72c 100644 --- a/src/Columns/ColumnLazy.h +++ b/src/Columns/ColumnLazy.h @@ -91,7 +91,7 @@ class ColumnLazy final : public COWHelper void insertDefault() override; void popBack(size_t n) override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 964b43652b8a..352e3998c4a2 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -276,22 +276,18 @@ void ColumnLowCardinality::insertData(const char * pos, size_t length) idx.insertPosition(getDictionary().uniqueInsertData(pos, length)); } -StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnLowCardinality::serializeValueIntoArena( + size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { - return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin); + return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin, settings); } -StringRef ColumnLowCardinality::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const +char * ColumnLowCardinality::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { - return getDictionary().serializeAggregationStateValueIntoArena(getIndexes().getUInt(n), arena, begin); + return getDictionary().serializeValueIntoMemory(getIndexes().getUInt(n), memory, settings); } -char * ColumnLowCardinality::serializeValueIntoMemory(size_t n, char * memory) const -{ - return getDictionary().serializeValueIntoMemory(getIndexes().getUInt(n), memory); -} - -void ColumnLowCardinality::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const +void ColumnLowCardinality::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { /// nullable is handled internally. chassert(is_null == nullptr); @@ -305,20 +301,14 @@ void ColumnLowCardinality::collectSerializedValueSizes(PaddedPODArray & throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of sizes: {} doesn't match rows_num: {}. It is a bug", sizes.size(), rows); PaddedPODArray dict_sizes; - getDictionary().collectSerializedValueSizes(dict_sizes, nullptr); + getDictionary().collectSerializedValueSizes(dict_sizes, nullptr, settings); idx.collectSerializedValueSizes(sizes, dict_sizes); } -void ColumnLowCardinality::deserializeAndInsertFromArena(ReadBuffer & in) -{ - compactIfSharedDictionary(); - idx.insertPosition(getDictionary().uniqueDeserializeAndInsertFromArena(in)); -} - -void ColumnLowCardinality::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnLowCardinality::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { compactIfSharedDictionary(); - idx.insertPosition(getDictionary().uniqueDeserializeAndInsertAggregationStateValueFromArena(in)); + idx.insertPosition(getDictionary().uniqueDeserializeAndInsertFromArena(in, settings)); } void ColumnLowCardinality::skipSerializedInArena(ReadBuffer & in) const diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 429c58d8603c..8e8ce8840ed6 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -101,14 +101,12 @@ class ColumnLowCardinality final : public COWHelper & sizes, const UInt8 * is_null) const override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index d84dccd201c8..9472985ddd01 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -150,34 +150,24 @@ void ColumnMap::popBack(size_t n) nested->popBack(n); } -StringRef ColumnMap::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnMap::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { - return nested->serializeValueIntoArena(n, arena, begin); + return nested->serializeValueIntoArena(n, arena, begin, settings); } -StringRef ColumnMap::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const +char * ColumnMap::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { - return nested->serializeAggregationStateValueIntoArena(n, arena, begin); + return nested->serializeValueIntoMemory(n, memory, settings); } -char * ColumnMap::serializeValueIntoMemory(size_t n, char * memory) const +std::optional ColumnMap::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { - return nested->serializeValueIntoMemory(n, memory); + return nested->getSerializedValueSize(n, settings); } -std::optional ColumnMap::getSerializedValueSize(size_t n) const +void ColumnMap::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { - return nested->getSerializedValueSize(n); -} - -void ColumnMap::deserializeAndInsertFromArena(ReadBuffer & in) -{ - nested->deserializeAndInsertFromArena(in); -} - -void ColumnMap::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) -{ - nested->deserializeAndInsertAggregationStateValueFromArena(in); + nested->deserializeAndInsertFromArena(in, settings); } void ColumnMap::skipSerializedInArena(ReadBuffer & in) const diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 6a1e09def754..ea9a5194ac9e 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -56,12 +56,10 @@ class ColumnMap final : public COWHelper, ColumnMap> bool tryInsert(const Field & x) override; void insertDefault() override; void popBack(size_t n) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 78521897340d..109d822d041d 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -166,7 +166,7 @@ void ColumnNullable::insertData(const char * pos, size_t length) } } -StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { const auto & arr = getNullMapData(); @@ -179,32 +179,13 @@ StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char return StringRef(pos, 1); /// Now serialize the nested value. Note that it also uses allocContinue so that the memory range remains contiguous. - auto nested_ref = getNestedColumn().serializeValueIntoArena(n, arena, begin); + auto nested_ref = getNestedColumn().serializeValueIntoArena(n, arena, begin, settings); /// serializeValueIntoArena may reallocate memory. Have to use ptr from nested_ref.data and move it back. return StringRef(nested_ref.data - 1, nested_ref.size + 1); } -StringRef ColumnNullable::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - const auto & arr = getNullMapData(); - - /// First serialize the NULL map byte. - auto * pos = arena.allocContinue(1, begin); - *pos = arr[n]; - - /// If the value is NULL, that's it. - if (arr[n]) - return StringRef(pos, 1); - - /// Now serialize the nested value. Note that it also uses allocContinue so that the memory range remains contiguous. - auto nested_ref = getNestedColumn().serializeAggregationStateValueIntoArena(n, arena, begin); - - /// serializeAggregationStateValueIntoArena may reallocate memory. Have to use ptr from nested_ref.data and move it back. - return StringRef(nested_ref.data - 1, nested_ref.size + 1); -} - -char * ColumnNullable::serializeValueIntoMemory(size_t n, char * memory) const +char * ColumnNullable::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { const auto & arr = getNullMapData(); @@ -214,31 +195,18 @@ char * ColumnNullable::serializeValueIntoMemory(size_t n, char * memory) const if (arr[n]) return memory; - return getNestedColumn().serializeValueIntoMemory(n, memory); + return getNestedColumn().serializeValueIntoMemory(n, memory, settings); } -std::optional ColumnNullable::getSerializedValueSize(size_t n) const +std::optional ColumnNullable::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { - auto nested_size = getNestedColumn().getSerializedValueSize(n); + auto nested_size = getNestedColumn().getSerializedValueSize(n, settings); if (!nested_size) return std::nullopt; return 1 + *nested_size; /// +1 for null mask byte. } -void ColumnNullable::deserializeAndInsertFromArena(ReadBuffer & in) -{ - UInt8 val; - readBinaryLittleEndian(val, in); - - getNullMapData().push_back(val); - - if (val == 0) - getNestedColumn().deserializeAndInsertFromArena(in); - else - getNestedColumn().insertDefault(); -} - -void ColumnNullable::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnNullable::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { UInt8 val; readBinaryLittleEndian(val, in); @@ -246,7 +214,7 @@ void ColumnNullable::deserializeAndInsertAggregationStateValueFromArena(ReadBuff getNullMapData().push_back(val); if (val == 0) - getNestedColumn().deserializeAndInsertAggregationStateValueFromArena(in); + getNestedColumn().deserializeAndInsertFromArena(in, settings); else getNestedColumn().insertDefault(); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index d2bceca3bbe6..e530502def60 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -66,12 +66,10 @@ class ColumnNullable final : public COWHelper, Col StringRef getDataAt(size_t) const override; /// Will insert null value if pos=nullptr void insertData(const char * pos, size_t length) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; #if !defined(DEBUG_OR_SANITIZER_BUILD) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 03f031d4e562..1e931c8d9ce5 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -895,29 +895,13 @@ void ColumnObject::rollback(const ColumnCheckpoint & checkpoint) shared_data->rollback(*object_checkpoint.shared_data); } -StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const +StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin, const IColumn::SerializationSettings * settings) const { StringRef res(begin, 0); /// First serialize values from typed paths in sorted order. They are the same for all instances of this column. for (auto path : sorted_typed_paths) { - auto data_ref = typed_paths.find(path)->second->serializeValueIntoArena(n, arena, begin); - res.data = data_ref.data - res.size; - res.size += data_ref.size; - } - - /// Second, serialize paths and values in binary format from dynamic paths and shared data in sorted by path order. - serializeDynamicPathsAndSharedDataIntoArena(n, arena, begin, res); - return res; -} - -StringRef ColumnObject::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - StringRef res(begin, 0); - /// First serialize values from typed paths in sorted order. They are the same for all instances of this column. - for (auto path : sorted_typed_paths) - { - auto data_ref = typed_paths.find(path)->second->serializeAggregationStateValueIntoArena(n, arena, begin); + auto data_ref = typed_paths.find(path)->second->serializeValueIntoArena(n, arena, begin, settings); res.data = data_ref.data - res.size; res.size += data_ref.size; } @@ -990,21 +974,11 @@ void ColumnObject::serializePathAndValueIntoArena(DB::Arena & arena, const char res.size += sizeof(size_t) + path_size + sizeof(size_t) + value_size; } -void ColumnObject::deserializeAndInsertFromArena(ReadBuffer & in) -{ - /// First deserialize typed paths. They come first. - for (auto path : sorted_typed_paths) - typed_paths.find(path)->second->deserializeAndInsertFromArena(in); - - /// Second deserialize all other paths and values and insert them into dynamic paths or shared data. - deserializeDynamicPathsAndSharedDataFromArena(in); -} - -void ColumnObject::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnObject::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { /// First deserialize typed paths. They come first. for (auto path : sorted_typed_paths) - typed_paths.find(path)->second->deserializeAndInsertAggregationStateValueFromArena(in); + typed_paths.find(path)->second->deserializeAndInsertFromArena(in, settings); /// Second deserialize all other paths and values and insert them into dynamic paths or shared data. deserializeDynamicPathsAndSharedDataFromArena(in); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 35190711908b..6fab2ca6dde5 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -143,12 +143,10 @@ class ColumnObject final : public COWHelper, ColumnO void popBack(size_t n) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; - std::optional getSerializedValueSize(size_t) const override { return std::nullopt; } + std::optional getSerializedValueSize(size_t, const IColumn::SerializationSettings *) const override { return std::nullopt; } void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index ee08112738a7..e963488a3468 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -158,34 +158,24 @@ void ColumnSparse::insertData(const char * pos, size_t length) insertSingleValue([&](IColumn & column) { column.insertData(pos, length); }); } -StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { - return values->serializeValueIntoArena(getValueIndex(n), arena, begin); + return values->serializeValueIntoArena(getValueIndex(n), arena, begin, settings); } -StringRef ColumnSparse::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const +char * ColumnSparse::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { - return values->serializeAggregationStateValueIntoArena(getValueIndex(n), arena, begin); + return values->serializeValueIntoMemory(getValueIndex(n), memory, settings); } -char * ColumnSparse::serializeValueIntoMemory(size_t n, char * memory) const +std::optional ColumnSparse::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { - return values->serializeValueIntoMemory(getValueIndex(n), memory); + return values->getSerializedValueSize(getValueIndex(n), settings); } -std::optional ColumnSparse::getSerializedValueSize(size_t n) const +void ColumnSparse::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { - return values->getSerializedValueSize(getValueIndex(n)); -} - -void ColumnSparse::deserializeAndInsertFromArena(ReadBuffer & in) -{ - insertSingleValue([&](IColumn & column) { column.deserializeAndInsertFromArena(in); }); -} - -void ColumnSparse::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) -{ - insertSingleValue([&](IColumn & column) { column.deserializeAndInsertAggregationStateValueFromArena(in); }); + insertSingleValue([&](IColumn & column) { column.deserializeAndInsertFromArena(in, settings); }); } void ColumnSparse::skipSerializedInArena(ReadBuffer & in) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 091054cb5e1d..30293913a401 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -78,12 +78,10 @@ class ColumnSparse final : public COWHelper, ColumnS /// Will insert null value if pos=nullptr void insertData(const char * pos, size_t length) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; #if !defined(DEBUG_OR_SANITIZER_BUILD) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 11f6c21385fc..77da8370f685 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -226,7 +226,7 @@ void ColumnString::rollback(const ColumnCheckpoint & checkpoint) chars.resize_assume_reserved(assert_cast(checkpoint).nested->size); } -void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const +void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { if (empty()) return; @@ -237,11 +237,12 @@ void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, c else if (sizes.size() != rows) throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of sizes: {} doesn't match rows_num: {}. It is a bug", sizes.size(), rows); + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; if (is_null) { for (size_t i = 0; i < rows; ++i) { - size_t string_size = sizeAt(i); + size_t string_size = sizeAt(i) + serialize_string_with_zero_byte; sizes[i] += 1 + !is_null[i] * (sizeof(string_size) + string_size); } } @@ -249,98 +250,80 @@ void ColumnString::collectSerializedValueSizes(PaddedPODArray & sizes, c { for (size_t i = 0; i < rows; ++i) { - size_t string_size = sizeAt(i); + size_t string_size = sizeAt(i) + serialize_string_with_zero_byte; sizes[i] += sizeof(string_size) + string_size; } } } - -StringRef ColumnString::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +std::optional ColumnString::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { - size_t string_size = sizeAt(n); - size_t offset = offsetAt(n); - - StringRef res; - res.size = sizeof(string_size) + string_size; - char * pos = arena.allocContinue(res.size, begin); - memcpy(pos, &string_size, sizeof(string_size)); - memcpy(pos + sizeof(string_size), &chars[offset], string_size); - res.data = pos; - - return res; + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; + return byteSizeAt(n) + serialize_string_with_zero_byte; } -StringRef ColumnString::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnString::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { - /// Serialize string values with 0 byte at the end for compatibility - /// with old versions where we stored 0 byte at the end of each string value. - size_t string_size_with_zero_byte = sizeAt(n) + 1; + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; + + size_t string_size = sizeAt(n) + serialize_string_with_zero_byte; size_t offset = offsetAt(n); StringRef res; - res.size = sizeof(string_size_with_zero_byte) + string_size_with_zero_byte; + res.size = sizeof(string_size) + string_size; char * pos = arena.allocContinue(res.size, begin); - memcpy(pos, &string_size_with_zero_byte, sizeof(string_size_with_zero_byte)); - memcpy(pos + sizeof(string_size_with_zero_byte), &chars[offset], string_size_with_zero_byte - 1); - /// Add 0 byte at the end. - *(pos + sizeof(string_size_with_zero_byte) + string_size_with_zero_byte - 1) = 0; + memcpy(pos, &string_size, sizeof(string_size)); + memcpy(pos + sizeof(string_size), &chars[offset], string_size - serialize_string_with_zero_byte); + if (serialize_string_with_zero_byte) + *(pos + sizeof(string_size) + string_size - 1) = 0; res.data = pos; return res; } - -ALWAYS_INLINE char * ColumnString::serializeValueIntoMemory(size_t n, char * memory) const +ALWAYS_INLINE char * ColumnString::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { - size_t string_size = sizeAt(n); + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; + size_t string_size = sizeAt(n) + serialize_string_with_zero_byte; size_t offset = offsetAt(n); memcpy(memory, &string_size, sizeof(string_size)); memory += sizeof(string_size); - memcpy(memory, &chars[offset], string_size); + memcpy(memory, &chars[offset], string_size - serialize_string_with_zero_byte); + if (serialize_string_with_zero_byte) + *(memory + string_size - 1) = 0; return memory + string_size; } -void ColumnString::batchSerializeValueIntoMemory(std::vector & memories) const +void ColumnString::batchSerializeValueIntoMemory(std::vector & memories, const IColumn::SerializationSettings * settings) const { chassert(memories.size() == size()); + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; for (size_t i = 0; i < memories.size(); ++i) { - size_t string_size = sizeAt(i); + size_t string_size = sizeAt(i) + serialize_string_with_zero_byte; size_t offset = offsetAt(i); memcpy(memories[i], &string_size, sizeof(string_size)); memories[i] += sizeof(string_size); - memcpy(memories[i], &chars[offset], string_size); + memcpy(memories[i], &chars[offset], string_size - serialize_string_with_zero_byte); + if (serialize_string_with_zero_byte) + *(memories[i] + string_size - 1) = 0; memories[i] += string_size; } } -void ColumnString::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnString::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { size_t string_size; readBinaryLittleEndian(string_size, in); + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; const size_t old_size = chars.size(); - const size_t new_size = old_size + string_size; - chars.resize(new_size); - in.readStrict(reinterpret_cast(chars.data() + old_size), string_size); - - offsets.push_back(new_size); -} - -void ColumnString::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) -{ - /// Serialized value contains string values with 0 byte at the end for compatibility. - size_t string_size_with_zero_byte; - readBinaryLittleEndian(string_size_with_zero_byte, in); - - const size_t old_size = chars.size(); - const size_t new_size = old_size + string_size_with_zero_byte - 1; + const size_t new_size = old_size + string_size - serialize_string_with_zero_byte; chars.resize(new_size); - in.readStrict(reinterpret_cast(chars.data() + old_size), string_size_with_zero_byte - 1); - in.ignore(1); /// ignore the 0 byte at the end. + in.readStrict(reinterpret_cast(chars.data() + old_size), string_size - serialize_string_with_zero_byte); + in.ignore(serialize_string_with_zero_byte); offsets.push_back(new_size); } diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 807f75de0b1f..bcd27b3c1359 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -204,16 +204,16 @@ class ColumnString final : public COWHelper, ColumnS void updateCheckpoint(ColumnCheckpoint & checkpoint) const override; void rollback(const ColumnCheckpoint & checkpoint) override; - void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - ALWAYS_INLINE char * serializeValueIntoMemory(size_t n, char * memory) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; - void batchSerializeValueIntoMemory(std::vector & memories) const override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + ALWAYS_INLINE char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + void batchSerializeValueIntoMemory(std::vector & memories, const IColumn::SerializationSettings * settings) const override; + + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 42922dc3c531..b5cd32a0dc37 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include @@ -308,7 +309,7 @@ void ColumnTuple::rollback(const ColumnCheckpoint & checkpoint) columns[i]->rollback(*checkpoints[i]); } -StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { if (columns.empty()) { @@ -321,7 +322,7 @@ StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char con StringRef res(begin, 0); for (const auto & column : columns) { - auto value_ref = column->serializeValueIntoArena(n, arena, begin); + auto value_ref = column->serializeValueIntoArena(n, arena, begin, settings); res.data = value_ref.data - res.size; res.size += value_ref.size; } @@ -329,28 +330,7 @@ StringRef ColumnTuple::serializeValueIntoArena(size_t n, Arena & arena, char con return res; } -StringRef ColumnTuple::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - if (columns.empty()) - { - /// Has to put one useless byte into Arena, because serialization into zero number of bytes is ambiguous. - char * res = arena.allocContinue(1, begin); - *res = 0; - return { res, 1 }; - } - - StringRef res(begin, 0); - for (const auto & column : columns) - { - auto value_ref = column->serializeAggregationStateValueIntoArena(n, arena, begin); - res.data = value_ref.data - res.size; - res.size += value_ref.size; - } - - return res; -} - -char * ColumnTuple::serializeValueIntoMemory(size_t n, char * memory) const +char * ColumnTuple::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { if (columns.empty()) { @@ -359,12 +339,12 @@ char * ColumnTuple::serializeValueIntoMemory(size_t n, char * memory) const } for (const auto & column : columns) - memory = column->serializeValueIntoMemory(n, memory); + memory = column->serializeValueIntoMemory(n, memory, settings); return memory; } -std::optional ColumnTuple::getSerializedValueSize(size_t n) const +std::optional ColumnTuple::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { if (columns.empty()) return 1; @@ -372,7 +352,7 @@ std::optional ColumnTuple::getSerializedValueSize(size_t n) const size_t res = 0; for (const auto & column : columns) { - auto element_size = column->getSerializedValueSize(n); + auto element_size = column->getSerializedValueSize(n, settings); if (!element_size) return std::nullopt; res += *element_size; @@ -382,21 +362,7 @@ std::optional ColumnTuple::getSerializedValueSize(size_t n) const } -void ColumnTuple::deserializeAndInsertFromArena(ReadBuffer & in) -{ - ++column_length; - - if (columns.empty()) - { - in.ignore(1); - return; - } - - for (auto & column : columns) - column->deserializeAndInsertFromArena(in); -} - -void ColumnTuple::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnTuple::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { ++column_length; @@ -407,7 +373,7 @@ void ColumnTuple::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer } for (auto & column : columns) - column->deserializeAndInsertAggregationStateValueFromArena(in); + column->deserializeAndInsertFromArena(in, settings); } void ColumnTuple::skipSerializedInArena(ReadBuffer & in) const diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index a0d27e53e166..ad4a0da95b60 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -77,12 +77,10 @@ class ColumnTuple final : public COWHelper, ColumnTup void insertDefault() override; void popBack(size_t n) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index c3a533ccaef7..f39092602298 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -66,7 +66,7 @@ class ColumnUnique final : public COWHelpergetFloat32(n); } bool getBool(size_t n) const override { return getNestedColumn()->getBool(n); } bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); } - void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; void skipSerializedInArena(ReadBuffer & in) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; void updateHashWithValue(size_t n, SipHash & hash_func) const override; #if !defined(DEBUG_OR_SANITIZER_BUILD) @@ -444,7 +443,7 @@ size_t ColumnUnique::uniqueInsertData(const char * pos, size_t lengt } template -void ColumnUnique::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const +void ColumnUnique::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { /// nullable is handled internally. chassert(is_null == nullptr); @@ -452,13 +451,18 @@ void ColumnUnique::collectSerializedValueSizes(PaddedPODArraycollectSerializedValueSizes(sizes, assert_cast(*nested_null_mask).getData().data()); + column_holder->collectSerializedValueSizes(sizes, assert_cast(*nested_null_mask).getData().data(), settings); else - column_holder->collectSerializedValueSizes(sizes, nullptr); + column_holder->collectSerializedValueSizes(sizes, nullptr, settings); } template +<<<<<<< HEAD +std::string_view ColumnUnique::serializeValueIntoArena( + size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const +======= StringRef ColumnUnique::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +>>>>>>> backport/25.8/90880 { if (is_nullable) { @@ -471,18 +475,18 @@ StringRef ColumnUnique::serializeValueIntoArena(size_t n, Arena & ar if (n == getNullValueIndex()) return StringRef(pos, s); - auto nested_ref = column_holder->serializeValueIntoArena(n, arena, begin); + auto nested_ref = column_holder->serializeValueIntoArena(n, arena, begin, settings); /// serializeValueIntoArena may reallocate memory. Have to use ptr from nested_ref.data and move it back. return StringRef(nested_ref.data - s, nested_ref.size + s); } - return column_holder->serializeValueIntoArena(n, arena, begin); + return column_holder->serializeValueIntoArena(n, arena, begin, settings); } template -char * ColumnUnique::serializeValueIntoMemory(size_t n, char * memory) const +char * ColumnUnique::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { if (is_nullable) { @@ -494,10 +498,13 @@ char * ColumnUnique::serializeValueIntoMemory(size_t n, char * memor return memory; } - return column_holder->serializeValueIntoMemory(n, memory); + return column_holder->serializeValueIntoMemory(n, memory, settings); } template +<<<<<<< HEAD +size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) +======= StringRef ColumnUnique::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const { if (is_nullable) @@ -522,6 +529,7 @@ StringRef ColumnUnique::serializeAggregationStateValueIntoArena(size template size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(ReadBuffer & in) +>>>>>>> backport/25.8/90880 { if (is_nullable) { @@ -544,12 +552,13 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(ReadBuffer } /// String + bool serialize_string_with_zero_byte = settings && settings->serialize_string_with_zero_byte; size_t string_size; readBinaryLittleEndian(string_size, in); if (in.available() < string_size) throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Not enough data to deserialize string value in ColumnUnique."); - size_t ret = uniqueInsertData(in.position(), string_size); + size_t ret = uniqueInsertData(in.position(), string_size - serialize_string_with_zero_byte); in.ignore(string_size); return ret; } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 6cb706f13295..0fb37f1b32d2 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -786,7 +786,7 @@ void ColumnVariant::rollback(const ColumnCheckpoint & checkpoint) variants[i]->rollback(*checkpoints[i]); } -StringRef ColumnVariant::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef ColumnVariant::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { /// During any serialization/deserialization we should always use global discriminators. Discriminator global_discr = globalDiscriminatorAt(n); @@ -797,50 +797,14 @@ StringRef ColumnVariant::serializeValueIntoArena(size_t n, Arena & arena, char c if (global_discr == NULL_DISCRIMINATOR) return res; - auto value_ref = variants[localDiscriminatorByGlobal(global_discr)]->serializeValueIntoArena(offsetAt(n), arena, begin); + auto value_ref = variants[localDiscriminatorByGlobal(global_discr)]->serializeValueIntoArena(offsetAt(n), arena, begin, settings); res.data = value_ref.data - res.size; res.size += value_ref.size; return res; } -StringRef ColumnVariant::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - /// During any serialization/deserialization we should always use global discriminators. - Discriminator global_discr = globalDiscriminatorAt(n); - char * pos = arena.allocContinue(sizeof(global_discr), begin); - memcpy(pos, &global_discr, sizeof(global_discr)); - StringRef res(pos, sizeof(global_discr)); - - if (global_discr == NULL_DISCRIMINATOR) - return res; - - auto value_ref = variants[localDiscriminatorByGlobal(global_discr)]->serializeAggregationStateValueIntoArena(offsetAt(n), arena, begin); - res.data = value_ref.data - res.size; - res.size += value_ref.size; - - return res; -} - -void ColumnVariant::deserializeAndInsertFromArena(ReadBuffer & in) -{ - /// During any serialization/deserialization we should always use global discriminators. - Discriminator global_discr; - readBinaryLittleEndian(global_discr, in); - - Discriminator local_discr = localDiscriminatorByGlobal(global_discr); - getLocalDiscriminators().push_back(local_discr); - if (local_discr == NULL_DISCRIMINATOR) - { - getOffsets().emplace_back(); - return; - } - - getOffsets().push_back(variants[local_discr]->size()); - variants[local_discr]->deserializeAndInsertFromArena(in); -} - -void ColumnVariant::deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) +void ColumnVariant::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) { /// During any serialization/deserialization we should always use global discriminators. Discriminator global_discr; @@ -855,7 +819,7 @@ void ColumnVariant::deserializeAndInsertAggregationStateValueFromArena(ReadBuffe } getOffsets().push_back(variants[local_discr]->size()); - variants[local_discr]->deserializeAndInsertAggregationStateValueFromArena(in); + variants[local_discr]->deserializeAndInsertFromArena(in, settings); } void ColumnVariant::skipSerializedInArena(ReadBuffer & in) const @@ -869,7 +833,7 @@ void ColumnVariant::skipSerializedInArena(ReadBuffer & in) const variants[localDiscriminatorByGlobal(global_discr)]->skipSerializedInArena(in); } -char * ColumnVariant::serializeValueIntoMemory(size_t n, char * memory) const +char * ColumnVariant::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { Discriminator global_discr = globalDiscriminatorAt(n); memcpy(memory, &global_discr, sizeof(global_discr)); @@ -877,17 +841,17 @@ char * ColumnVariant::serializeValueIntoMemory(size_t n, char * memory) const if (global_discr == NULL_DISCRIMINATOR) return memory; - return variants[localDiscriminatorByGlobal(global_discr)]->serializeValueIntoMemory(offsetAt(n), memory); + return variants[localDiscriminatorByGlobal(global_discr)]->serializeValueIntoMemory(offsetAt(n), memory, settings); } -std::optional ColumnVariant::getSerializedValueSize(size_t n) const +std::optional ColumnVariant::getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const { size_t res = sizeof(Discriminator); Discriminator global_discr = globalDiscriminatorAt(n); if (global_discr == NULL_DISCRIMINATOR) return res; - auto variant_size = variants[localDiscriminatorByGlobal(global_discr)]->getSerializedValueSize(offsetAt(n)); + auto variant_size = variants[localDiscriminatorByGlobal(global_discr)]->getSerializedValueSize(offsetAt(n), settings); if (!variant_size) return std::nullopt; diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 8d91fb018c22..338bdbe45353 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -213,13 +213,11 @@ class ColumnVariant final : public COWHelper, Colum void insertManyDefaults(size_t length) override; void popBack(size_t n) override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; - void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - std::optional getSerializedValueSize(size_t n) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + std::optional getSerializedValueSize(size_t n, const IColumn::SerializationSettings * settings) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; WeakHash32 getWeakHash32() const override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index d03ccca0f5e9..2e0f5b7f1452 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -58,7 +58,7 @@ namespace ErrorCodes } template -void ColumnVector::deserializeAndInsertFromArena(ReadBuffer & in) +void ColumnVector::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { T element; readBinaryLittleEndian(element, in); diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index b5154ee38520..81601c7e1fae 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -106,7 +106,7 @@ class ColumnVector final : public COWHelper, Colum data.resize_assume_reserved(data.size() - n); } - void deserializeAndInsertFromArena(ReadBuffer & in) override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 5c926100e044..b940304b4195 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -136,38 +136,44 @@ bool IColumn::getBool(size_t /*n*/) const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getBool is not supported for {}", getName()); } -StringRef IColumn::serializeValueIntoArena(size_t /* n */, Arena & /* arena */, char const *& /* begin */) const +StringRef IColumn::serializeValueIntoArena(size_t /* n */, Arena & /* arena */, char const *& /* begin */, const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeValueIntoArena is not supported for {}", getName()); } -char * IColumn::serializeValueIntoMemory(size_t /* n */, char * /* memory */) const +char * IColumn::serializeValueIntoMemory(size_t /* n */, char * /* memory */, const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeValueIntoMemory is not supported for {}", getName()); } -void IColumn::batchSerializeValueIntoMemory(std::vector & /* memories */) const +void IColumn::batchSerializeValueIntoMemory(std::vector & /* memories */, const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method batchSerializeValueIntoMemory is not supported for {}", getName()); } -StringRef -IColumn::serializeValueIntoArenaWithNull(size_t /* n */, Arena & /* arena */, char const *& /* begin */, const UInt8 * /* is_null */) const +StringRef IColumn::serializeValueIntoArenaWithNull( + size_t /* n */, + Arena & /* arena */, + char const *& /* begin */, + const UInt8 * /* is_null */, + const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeValueIntoArenaWithNull is not supported for {}", getName()); } -char * IColumn::serializeValueIntoMemoryWithNull(size_t /* n */, char * /* memory */, const UInt8 * /* is_null */) const +char * IColumn::serializeValueIntoMemoryWithNull( + size_t /* n */, char * /* memory */, const UInt8 * /* is_null */, const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serializeValueIntoMemoryWithNull is not supported for {}", getName()); } -void IColumn::batchSerializeValueIntoMemoryWithNull(std::vector & /* memories */, const UInt8 * /* is_null */) const +void IColumn::batchSerializeValueIntoMemoryWithNull( + std::vector & /* memories */, const UInt8 * /* is_null */, const IColumn::SerializationSettings * /* settings */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method batchSerializeValueIntoMemoryWithNull is not supported for {}", getName()); } -void IColumn::collectSerializedValueSizes(PaddedPODArray & /* sizes */, const UInt8 * /* is_null */) const +void IColumn::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const SerializationSettings * settings) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method collectSerializedValueSizes is not supported for {}", getName()); } @@ -547,8 +553,7 @@ void IColumnHelper::fillFromBlocksAndRowNumbers(const DataTypeP } template -StringRef -IColumnHelper::serializeValueIntoArenaWithNull(size_t n, Arena & arena, char const *& begin, const UInt8 * is_null) const +StringRef IColumnHelper::serializeValueIntoArenaWithNull(size_t n, Arena & arena, char const *& begin, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { const auto & self = static_cast(*this); if (is_null) @@ -561,40 +566,41 @@ IColumnHelper::serializeValueIntoArenaWithNull(size_t n, Arena return {memory, 1}; } - auto serialized_value_size = self.getSerializedValueSize(n); + auto serialized_value_size = self.getSerializedValueSize(n, settings); if (serialized_value_size) { size_t total_size = *serialized_value_size + 1 /* null map byte */; memory = arena.allocContinue(total_size, begin); *memory = 0; - self.serializeValueIntoMemory(n, memory + 1); + self.serializeValueIntoMemory(n, memory + 1, settings); return {memory, total_size}; } memory = arena.allocContinue(1, begin); *memory = 0; - auto res = self.serializeValueIntoArena(n, arena, begin); + auto res = self.serializeValueIntoArena(n, arena, begin, settings); return StringRef(res.data - 1, res.size + 1); } - return self.serializeValueIntoArena(n, arena, begin); + return self.serializeValueIntoArena(n, arena, begin, settings); } template -StringRef IColumnHelper::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +StringRef IColumnHelper::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const { if constexpr (!std::is_base_of_v) - return IColumn::serializeValueIntoArena(n, arena, begin); + return IColumn::serializeValueIntoArena(n, arena, begin, settings); const auto & self = static_cast(*this); size_t sz = self.byteSizeAt(n); char * memory = arena.allocContinue(sz, begin); - self.serializeValueIntoMemory(n, memory); + self.serializeValueIntoMemory(n, memory, settings); return {memory, sz}; } template -ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemoryWithNull(size_t n, char * memory, const UInt8 * is_null) const +ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemoryWithNull( + size_t n, char * memory, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { const auto & self = static_cast(*this); if (is_null) @@ -605,18 +611,19 @@ ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemoryWit return memory; } - return self.serializeValueIntoMemory(n, memory); + return self.serializeValueIntoMemory(n, memory, settings); } template -void IColumnHelper::batchSerializeValueIntoMemoryWithNull(std::vector & memories, const UInt8 * is_null) const +void IColumnHelper::batchSerializeValueIntoMemoryWithNull( + std::vector & memories, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { const auto & self = static_cast(*this); chassert(memories.size() == self.size()); if (!is_null) { - self.batchSerializeValueIntoMemory(memories); + self.batchSerializeValueIntoMemory(memories, settings); return; } @@ -626,15 +633,15 @@ void IColumnHelper::batchSerializeValueIntoMemoryWithNull(std:: *memories[i] = is_null[i]; ++memories[i]; if (!is_null[i]) - memories[i] = self.serializeValueIntoMemory(i, memories[i]); + memories[i] = self.serializeValueIntoMemory(i, memories[i], settings); } } template -ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemory(size_t n, char * memory) const +ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const { if constexpr (!std::is_base_of_v) - return IColumn::serializeValueIntoMemory(n, memory); + return IColumn::serializeValueIntoMemory(n, memory, settings); const auto & self = static_cast(*this); auto raw_data = self.getDataAt(n); @@ -643,19 +650,19 @@ ALWAYS_INLINE char * IColumnHelper::serializeValueIntoMemory(si } template -void IColumnHelper::batchSerializeValueIntoMemory(std::vector & memories) const +void IColumnHelper::batchSerializeValueIntoMemory(std::vector & memories, const IColumn::SerializationSettings * settings) const { const auto & self = static_cast(*this); chassert(memories.size() == self.size()); for (size_t i = 0; i < self.size(); ++i) - memories[i] = self.serializeValueIntoMemory(i, memories[i]); + memories[i] = self.serializeValueIntoMemory(i, memories[i], settings); } template -void IColumnHelper::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const +void IColumnHelper::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const { if constexpr (!std::is_base_of_v) - return IColumn::collectSerializedValueSizes(sizes, is_null); + return IColumn::collectSerializedValueSizes(sizes, is_null, settings); const auto & self = static_cast(*this); size_t rows = self.size(); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index cda959a27676..e2c086b9d082 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -266,59 +266,58 @@ class IColumn : public COW */ virtual void popBack(size_t n) = 0; + struct SerializationSettings + { + bool serialize_string_with_zero_byte = false; + + static SerializationSettings createForAggregationState() + { + /// Same aggregation state can be serialized/deserialized by servers with different versions. + /// Add zero byte to the end of the string in aggregation state to keep it compatible with old versions. + return SerializationSettings{.serialize_string_with_zero_byte = true}; + } + }; + /** Serializes n-th element. Serialized element should be placed continuously inside Arena's memory. * Serialized value can be deserialized to reconstruct original object. Is used in aggregation. * The method is similar to getDataAt(), but can work when element's value cannot be mapped to existing continuous memory chunk, * For example, to obtain unambiguous representation of Array of strings, strings data should be interleaved with their sizes. * Parameter begin should be used with Arena::allocContinue. */ - virtual StringRef serializeValueIntoArena(size_t /* n */, Arena & /* arena */, char const *& /* begin */) const; - - /// The same as serializeValueIntoArena but is used to store values inside aggregation states. - /// It's used in generic implementation of some aggregate functions. - /// serializeValueIntoArena is used for in-memory value representations, so it's implementation can be changed. - /// This method must respect compatibility with older versions because aggregation states may be serialized/deserialized - /// by servers with different versions. - virtual StringRef serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const - { - return serializeValueIntoArena(n, arena, begin); - } + virtual StringRef serializeValueIntoArena(size_t /* n */, Arena & /* arena */, char const *& /* begin */, const SerializationSettings * settings) const; /// Same as above but serialize into already allocated continuous memory. /// Return pointer to the end of the serialization data. - virtual char * serializeValueIntoMemory(size_t /* n */, char * /* memory */) const; + virtual char * serializeValueIntoMemory(size_t /* n */, char * /* memory */, const SerializationSettings * settings) const; /// Returns size in bytes required to serialize value into memory using the previous method. /// If size cannot be calculated in advance, return nullopt. In this case serializeValueIntoMemory /// cannot be used and serializeValueIntoArena should be used instead, - virtual std::optional getSerializedValueSize(size_t n) const { return byteSizeAt(n); } + virtual std::optional getSerializedValueSize(size_t n, const SerializationSettings *) const { return byteSizeAt(n); } - virtual void batchSerializeValueIntoMemory(std::vector & /* memories */) const; + virtual void batchSerializeValueIntoMemory(std::vector & /* memories */, const SerializationSettings * settings) const; /// Nullable variant to avoid calling virtualized method inside ColumnNullable. - virtual StringRef - serializeValueIntoArenaWithNull(size_t /* n */, Arena & /* arena */, char const *& /* begin */, const UInt8 * /* is_null */) const; + virtual StringRef serializeValueIntoArenaWithNull( + size_t /* n */, + Arena & /* arena */, + char const *& /* begin */, + const UInt8 * /* is_null */, + const SerializationSettings * settings) const; - virtual char * serializeValueIntoMemoryWithNull(size_t /* n */, char * /* memory */, const UInt8 * /* is_null */) const; + virtual char * serializeValueIntoMemoryWithNull(size_t /* n */, char * /* memory */, const UInt8 * /* is_null */, const SerializationSettings * settings) const; - virtual void batchSerializeValueIntoMemoryWithNull(std::vector & /* memories */, const UInt8 * /* is_null */) const; + virtual void batchSerializeValueIntoMemoryWithNull(std::vector & /* memories */, const UInt8 * /* is_null */, const SerializationSettings * settings) const; /// Calculate all the sizes of serialized data (as in the methods above) in the column and add to `sizes`. /// If `is_null` is not nullptr, also take null byte into account. /// This is currently used to facilitate the allocation of memory for an entire continuous row /// in a single step. For more details, refer to the HashMethodSerialized implementation. - virtual void collectSerializedValueSizes(PaddedPODArray & /* sizes */, const UInt8 * /* is_null */) const; + virtual void collectSerializedValueSizes(PaddedPODArray & /* sizes */, const UInt8 * /* is_null */, const SerializationSettings * settings) const; /// Deserializes a value that was serialized using IColumn::serializeValueIntoArena method. /// Note that it needs to deal with user input - virtual void deserializeAndInsertFromArena(ReadBuffer & in) = 0; - - /// Deserializes a value that was serialized using IColumn::serializeAggregationStateValueIntoArena method. - /// Note that it needs to deal with user input - virtual void deserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) - { - deserializeAndInsertFromArena(in); - } + virtual void deserializeAndInsertFromArena(ReadBuffer & in, const SerializationSettings * settings) = 0; /// Skip previously serialized value that was serialized using IColumn::serializeValueIntoArena method. virtual void skipSerializedInArena(ReadBuffer & in) const = 0; @@ -883,7 +882,7 @@ class IColumnHelper : public Parent void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override; /// Devirtualize byteSizeAt. - void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null) const override; + void collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; /// Devirtualize insertFrom. ColumnPtr updateFrom(const IColumn::Patch & patch) const override; @@ -900,14 +899,14 @@ class IColumnHelper : public Parent void fillFromBlocksAndRowNumbers(const DataTypePtr & type, size_t source_column_index_in_block, const std::vector & columns, const std::vector & row_nums) override; /// Move common implementations into the same translation unit to ensure they are properly inlined. - char * serializeValueIntoMemoryWithNull(size_t n, char * memory, const UInt8 * is_null) const override; - void batchSerializeValueIntoMemoryWithNull(std::vector & memories, const UInt8 * is_null) const override; + char * serializeValueIntoMemoryWithNull(size_t n, char * memory, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; + void batchSerializeValueIntoMemoryWithNull(std::vector & memories, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; - char * serializeValueIntoMemory(size_t n, char * memory) const override; - void batchSerializeValueIntoMemory(std::vector & memories) const override; + char * serializeValueIntoMemory(size_t n, char * memory, const IColumn::SerializationSettings * settings) const override; + void batchSerializeValueIntoMemory(std::vector & memories, const IColumn::SerializationSettings * settings) const override; - StringRef serializeValueIntoArenaWithNull(size_t n, Arena & arena, char const *& begin, const UInt8 * is_null) const override; - StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; + StringRef serializeValueIntoArenaWithNull(size_t n, Arena & arena, char const *& begin, const UInt8 * is_null, const IColumn::SerializationSettings * settings) const override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; }; } diff --git a/src/Columns/IColumnDummy.cpp b/src/Columns/IColumnDummy.cpp index 4532a6a79141..de58c50c0f4e 100644 --- a/src/Columns/IColumnDummy.cpp +++ b/src/Columns/IColumnDummy.cpp @@ -41,7 +41,7 @@ bool IColumnDummy::isDefaultAt(size_t) const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "isDefaultAt is not implemented for {}", getName()); } -StringRef IColumnDummy::serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin) const +StringRef IColumnDummy::serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin, const IColumn::SerializationSettings *) const { /// Has to put one useless byte into Arena, because serialization into zero number of bytes is ambiguous. char * res = arena.allocContinue(1, begin); @@ -49,7 +49,7 @@ StringRef IColumnDummy::serializeValueIntoArena(size_t /*n*/, Arena & arena, cha return { res, 1 }; } -void IColumnDummy::deserializeAndInsertFromArena(ReadBuffer & in) +void IColumnDummy::deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings *) { ++s; in.ignore(1); diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index a66c2223edd5..ad7244e74768 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -55,9 +55,9 @@ class IColumnDummy : public IColumnHelper ++s; } - StringRef serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin) const override; + StringRef serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const override; - void deserializeAndInsertFromArena(ReadBuffer & in) override; + void deserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) override; void skipSerializedInArena(ReadBuffer & in) const override; diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index 5c0d60de8a30..7e30c3c9d882 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -68,7 +68,7 @@ class IColumnUnique : public IColumn virtual size_t getNestedTypeDefaultValueIndex() const = 0; /// removeNullable()->getDefault() value index virtual bool canContainNulls() const = 0; - virtual size_t uniqueDeserializeAndInsertFromArena(ReadBuffer & in) = 0; + virtual size_t uniqueDeserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) = 0; virtual size_t uniqueDeserializeAndInsertAggregationStateValueFromArena(ReadBuffer & in) = 0; /// Returns dictionary hash which is SipHash is applied to each row of nested column. @@ -116,7 +116,7 @@ class IColumnUnique : public IColumn throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method gather is not supported for ColumnUnique."); } - void deserializeAndInsertFromArena(ReadBuffer &) override + void deserializeAndInsertFromArena(ReadBuffer &, const IColumn::SerializationSettings *) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method deserializeAndInsertFromArena is not supported for ColumnUnique."); } diff --git a/src/Columns/tests/gtest_column_dynamic.cpp b/src/Columns/tests/gtest_column_dynamic.cpp index 0679bbd5243c..5ceeb978a070 100644 --- a/src/Columns/tests/gtest_column_dynamic.cpp +++ b/src/Columns/tests/gtest_column_dynamic.cpp @@ -752,16 +752,16 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena1) Arena arena; const char * pos = nullptr; - auto ref1 = column->serializeValueIntoArena(0, arena, pos); - column->serializeValueIntoArena(1, arena, pos); - column->serializeValueIntoArena(2, arena, pos); - column->serializeValueIntoArena(3, arena, pos); + auto ref1 = column->serializeValueIntoArena(0, arena, pos, nullptr); + column->serializeValueIntoArena(1, arena, pos, nullptr); + column->serializeValueIntoArena(2, arena, pos, nullptr); + column->serializeValueIntoArena(3, arena, pos, nullptr); ReadBufferFromString in({ref1.data, arena.usedBytes()}); - column->deserializeAndInsertFromArena(in); - column->deserializeAndInsertFromArena(in); - column->deserializeAndInsertFromArena(in); - column->deserializeAndInsertFromArena(in); + column->deserializeAndInsertFromArena(in, nullptr); + column->deserializeAndInsertFromArena(in, nullptr); + column->deserializeAndInsertFromArena(in, nullptr); + column->deserializeAndInsertFromArena(in, nullptr); ASSERT_EQ((*column)[column->size() - 4], 42); ASSERT_EQ((*column)[column->size() - 3], 42.42); @@ -779,25 +779,17 @@ TEST(ColumnDynamic, SerializeDeserializeFromArena2) Arena arena; const char * pos = nullptr; - auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); - column_from->serializeValueIntoArena(1, arena, pos); - column_from->serializeValueIntoArena(2, arena, pos); - column_from->serializeValueIntoArena(3, arena, pos); + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos, nullptr); + column_from->serializeValueIntoArena(1, arena, pos, nullptr); + column_from->serializeValueIntoArena(2, arena, pos, nullptr); + column_from->serializeValueIntoArena(3, arena, pos, nullptr); auto column_to = ColumnDynamic::create(254); ReadBufferFromString in({ref1.data, arena.usedBytes()}); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - - ASSERT_EQ((*column_to)[column_to->size() - 4], 42); - ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); - ASSERT_EQ((*column_to)[column_to->size() - 2], "str"); - ASSERT_EQ((*column_to)[column_to->size() - 1], Null()); - ASSERT_EQ(column_to->getVariantInfo().variant_type->getName(), "Variant(Float64, Int8, SharedVariant, String)"); - std::vector expected_names = {"Float64", "Int8", "SharedVariant", "String"}; - ASSERT_EQ(column_to->getVariantInfo().variant_names, expected_names); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); std::unordered_map expected_variant_name_to_discriminator = {{"Float64", 0}, {"Int8", 1}, {"SharedVariant", 2}, {"String", 3}}; ASSERT_TRUE(column_to->getVariantInfo().variant_name_to_discriminator == expected_variant_name_to_discriminator); } @@ -812,17 +804,17 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow1) Arena arena; const char * pos = nullptr; - auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); - column_from->serializeValueIntoArena(1, arena, pos); - column_from->serializeValueIntoArena(2, arena, pos); - column_from->serializeValueIntoArena(3, arena, pos); + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos, nullptr); + column_from->serializeValueIntoArena(1, arena, pos, nullptr); + column_from->serializeValueIntoArena(2, arena, pos, nullptr); + column_from->serializeValueIntoArena(3, arena, pos, nullptr); auto column_to = getDynamicWithManyVariants(253); ReadBufferFromString in({ref1.data, arena.usedBytes()}); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); ASSERT_EQ((*column_to)[column_to->size() - 4], 42); ASSERT_EQ((*column_to)[column_to->size() - 3], 42.42); @@ -845,20 +837,20 @@ TEST(ColumnDynamic, SerializeDeserializeFromArenaOverflow2) Arena arena; const char * pos = nullptr; - auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); - column_from->serializeValueIntoArena(1, arena, pos); - column_from->serializeValueIntoArena(2, arena, pos); - column_from->serializeValueIntoArena(3, arena, pos); - column_from->serializeValueIntoArena(4, arena, pos); + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos, nullptr); + column_from->serializeValueIntoArena(1, arena, pos, nullptr); + column_from->serializeValueIntoArena(2, arena, pos, nullptr); + column_from->serializeValueIntoArena(3, arena, pos, nullptr); + column_from->serializeValueIntoArena(4, arena, pos, nullptr); auto column_to = ColumnDynamic::create(2); column_to->insert(Field(42.42)); ReadBufferFromString in({ref1.data, arena.usedBytes()}); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); - column_to->deserializeAndInsertFromArena(in); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); + column_to->deserializeAndInsertFromArena(in, nullptr); ASSERT_EQ((*column_to)[column_to->size() - 5], 42); ASSERT_EQ((*column_to)[column_to->size() - 4], 42.42); @@ -882,10 +874,10 @@ TEST(ColumnDynamic, skipSerializedInArena) Arena arena; const char * pos = nullptr; - auto ref1 = column_from->serializeValueIntoArena(0, arena, pos); - column_from->serializeValueIntoArena(1, arena, pos); - column_from->serializeValueIntoArena(2, arena, pos); - column_from->serializeValueIntoArena(3, arena, pos); + auto ref1 = column_from->serializeValueIntoArena(0, arena, pos, nullptr); + column_from->serializeValueIntoArena(1, arena, pos, nullptr); + column_from->serializeValueIntoArena(2, arena, pos, nullptr); + column_from->serializeValueIntoArena(3, arena, pos, nullptr); auto column_to = ColumnDynamic::create(254); ReadBufferFromString in({ref1.data, arena.usedBytes()}); diff --git a/src/Columns/tests/gtest_column_object.cpp b/src/Columns/tests/gtest_column_object.cpp index fb2ba8c0be6c..2970b44a51fb 100644 --- a/src/Columns/tests/gtest_column_object.cpp +++ b/src/Columns/tests/gtest_column_object.cpp @@ -313,16 +313,16 @@ TEST(ColumnObject, SerializeDeserializerFromArena) Arena arena; const char * pos = nullptr; - auto ref1 = col_object.serializeValueIntoArena(0, arena, pos); - col_object.serializeValueIntoArena(1, arena, pos); - col_object.serializeValueIntoArena(2, arena, pos); + auto ref1 = col_object.serializeValueIntoArena(0, arena, pos, nullptr); + col_object.serializeValueIntoArena(1, arena, pos, nullptr); + col_object.serializeValueIntoArena(2, arena, pos, nullptr); auto col2 = type->createColumn(); auto & col_object2 = assert_cast(*col); ReadBufferFromString in({ref1.data, arena.usedBytes()}); - col_object2.deserializeAndInsertFromArena(in); - col_object2.deserializeAndInsertFromArena(in); - col_object2.deserializeAndInsertFromArena(in); + col_object2.deserializeAndInsertFromArena(in, nullptr); + col_object2.deserializeAndInsertFromArena(in, nullptr); + col_object2.deserializeAndInsertFromArena(in, nullptr); ASSERT_TRUE(in.eof()); ASSERT_EQ(col_object2[0], (Object{{"b.d", Field(42u)}, {"a.b", Array{"Str1", "Str2"}}, {"a.a", Tuple{"Str3", 441u}}, {"a.c", Field("Str4")}, {"a.d", Array{Field(45), Field(46)}}, {"a.e", Field(47)}})); @@ -341,9 +341,9 @@ TEST(ColumnObject, SkipSerializedInArena) Arena arena; const char * pos = nullptr; - auto ref1 = col_object.serializeValueIntoArena(0, arena, pos); - col_object.serializeValueIntoArena(1, arena, pos); - col_object.serializeValueIntoArena(2, arena, pos); + auto ref1 = col_object.serializeValueIntoArena(0, arena, pos, nullptr); + col_object.serializeValueIntoArena(1, arena, pos, nullptr); + col_object.serializeValueIntoArena(2, arena, pos, nullptr); auto col2 = type->createColumn(); ReadBufferFromString in({ref1.data, arena.usedBytes()}); diff --git a/src/Columns/tests/gtest_column_unique.cpp b/src/Columns/tests/gtest_column_unique.cpp index ddebfe076082..6a165adb286c 100644 --- a/src/Columns/tests/gtest_column_unique.cpp +++ b/src/Columns/tests/gtest_column_unique.cpp @@ -118,9 +118,9 @@ void column_unique_unique_deserialize_from_arena_impl(ColumnType & column, const const char * pos = nullptr; for (size_t i = 0; i < num_values; ++i) { - auto ref = column_unique_pattern->serializeValueIntoArena(idx->getUInt(i), arena, pos); + auto ref = column_unique_pattern->serializeValueIntoArena(idx->getUInt(i), arena, pos, nullptr); ReadBufferFromString in({ref.data, ref.size}); - column_unique->uniqueDeserializeAndInsertFromArena(in); + column_unique->uniqueDeserializeAndInsertFromArena(in, nullptr); ASSERT_TRUE(in.eof()) << "Deserialized data has different sizes at position " << i; ASSERT_EQ(column_unique_pattern->getNestedNotNullableColumn()->getDataAt(idx->getUInt(i)), @@ -141,8 +141,8 @@ void column_unique_unique_deserialize_from_arena_impl(ColumnType & column, const const char * pos_lc = nullptr; for (size_t i = 0; i < num_values; ++i) { - auto ref_string = column.serializeValueIntoArena(i, arena_string, pos_string); - auto ref_lc = column_unique->serializeValueIntoArena(idx->getUInt(i), arena_lc, pos_lc); + auto ref_string = column.serializeValueIntoArena(i, arena_string, pos_string, nullptr); + auto ref_lc = column_unique->serializeValueIntoArena(idx->getUInt(i), arena_lc, pos_lc, nullptr); ASSERT_EQ(ref_string, ref_lc) << "Serialized data is different from pattern at position " << i; } } diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 463c6403cc27..6e1f47cac5b0 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB { @@ -331,6 +332,15 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } }; +class HashMethodSerializedContext : public HashMethodContext +{ +public: + explicit HashMethodSerializedContext(const HashMethodContextSettings & settings_) + : settings(settings_) + {} + + HashMethodContextSettings settings; +}; /** Hash by concatenating serialized key values. * The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts. @@ -344,6 +354,11 @@ struct HashMethodSerialized using Self = HashMethodSerialized; using Base = columns_hashing_impl::HashMethodBase; + static HashMethodContextPtr createContext(const HashMethodContextSettings & settings) + { + return std::make_shared(settings); + } + static constexpr bool has_cheap_key_calculation = false; ColumnRawPtrs key_columns; @@ -353,12 +368,22 @@ struct HashMethodSerialized /// Only used if prealloc is true. PaddedPODArray row_sizes; size_t total_size = 0; - PODArray serialized_buffer; + IColumn::SerializationSettings serialization_settings; + PaddedPODArray serialized_buffer; std::vector serialized_keys; - HashMethodSerialized(const ColumnRawPtrs & key_columns_, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + HashMethodSerialized(const ColumnRawPtrs & key_columns_, const Sizes & /*key_sizes*/, const HashMethodContextPtr & context) : key_columns(key_columns_), keys_size(key_columns_.size()) { + const auto * hash_serialized_context = typeid_cast(context.get()); + if (!hash_serialized_context) + { + const auto & cached_val = *context; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid type for HashMethodSerialized context: {}", + demangle(typeid(cached_val).name())); + } + + serialization_settings.serialize_string_with_zero_byte = hash_serialized_context->settings.serialize_string_with_zero_byte; if constexpr (nullable) { null_maps.resize(keys_size, nullptr); @@ -378,7 +403,7 @@ struct HashMethodSerialized /// Calculate serialized value size for each key column in each row. for (size_t i = 0; i < keys_size; ++i) - key_columns[i]->collectSerializedValueSizes(row_sizes, null_maps[i]); + key_columns[i]->collectSerializedValueSizes(row_sizes, null_maps[i], &serialization_settings); for (auto row_size : row_sizes) total_size += row_size; @@ -401,9 +426,9 @@ struct HashMethodSerialized for (size_t i = 0; i < keys_size; ++i) { if constexpr (nullable) - key_columns[i]->batchSerializeValueIntoMemoryWithNull(memories, null_maps[i]); + key_columns[i]->batchSerializeValueIntoMemoryWithNull(memories, null_maps[i], &serialization_settings); else - key_columns[i]->batchSerializeValueIntoMemory(memories); + key_columns[i]->batchSerializeValueIntoMemory(memories, &serialization_settings); } } } @@ -425,13 +450,13 @@ struct HashMethodSerialized size_t sum_size = 0; for (size_t j = 0; j < keys_size; ++j) - sum_size += key_columns[j]->serializeValueIntoArenaWithNull(row, pool, begin, null_maps[j]).size; + sum_size += key_columns[j]->serializeValueIntoArenaWithNull(row, pool, begin, null_maps[j], &serialization_settings).size; return SerializedKeyHolder{{begin, sum_size}, pool}; } return SerializedKeyHolder{ - serializeKeysToPoolContiguous(row, keys_size, key_columns, pool), + serializeKeysToPoolContiguous(row, keys_size, key_columns, pool, &serialization_settings), pool}; } }; diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index 287faed01064..253249d86eb8 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -19,6 +19,7 @@ namespace ColumnsHashing struct HashMethodContextSettings { size_t max_threads; + bool serialize_string_with_zero_byte = false; }; /// Generic context for HashMethod. Context is shared between multiple threads, all methods must be thread-safe. diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 903d984dee49..e37c189d3c2b 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6854,6 +6854,9 @@ Possible values: )", 0) \ DECLARE(Bool, use_roaring_bitmap_iceberg_positional_deletes, false, R"( Use roaring bitmap for iceberg positional deletes. +)", 0) \ + DECLARE(Bool, serialize_string_in_memory_with_zero_byte, true, R"( +Serialize String values during aggregation with zero byte at the end. Enable to keep compatibility when querying cluster of incompatible versions. )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 1ba731c37bbf..eaa78f0a3661 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -94,7 +94,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_ytsaurus_table_function", false, false, "New setting."}, {"allow_experimental_ytsaurus_dictionary_source", false, false, "New setting."}, {"per_part_index_stats", false, false, "New setting."}, - {"allow_experimental_iceberg_compaction", 0, 0, "New setting "}, + {"allow_experimental_iceberg_compaction", 0, 0, "New setting"}, {"delta_lake_snapshot_version", -1, -1, "New setting"}, {"delta_lake_insert_max_bytes_in_data_file", 1_GiB, 1_GiB, "New setting."}, {"delta_lake_insert_max_rows_in_data_file", 100000, 100000, "New setting."}, @@ -109,6 +109,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"enable_lightweight_update", false, true, "Lightweight updates were moved to Beta. Added an alias for setting 'allow_experimental_lightweight_update'."}, {"allow_experimental_lightweight_update", false, true, "Lightweight updates were moved to Beta."}, {"s3_slow_all_threads_after_retryable_error", false, false, "Added an alias for setting `backup_slow_all_threads_after_retryable_s3_error`"}, + {"serialize_string_in_memory_with_zero_byte", true, true, "New setting"}, {"iceberg_metadata_log_level", "none", "none", "New setting."}, }); addSettingsChanges(settings_changes_history, "25.7", diff --git a/src/Dictionaries/DictionaryHelpers.cpp b/src/Dictionaries/DictionaryHelpers.cpp index cd8ce7d4c11a..742558e97469 100644 --- a/src/Dictionaries/DictionaryHelpers.cpp +++ b/src/Dictionaries/DictionaryHelpers.cpp @@ -21,7 +21,7 @@ MutableColumns deserializeColumnsFromKeys( ReadBufferFromString in({key.data, key.size}); for (auto & result_column : result_columns) - result_column->deserializeAndInsertFromArena(in); + result_column->deserializeAndInsertFromArena(in, nullptr); } return result_columns; diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index c249e2c2053d..d48ab9b47b49 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -238,7 +238,7 @@ static inline void deserializeAndInsertIntoColumns( /// NOLINT const auto & column = columns[column_index]; if (fetch_request.shouldFillResultColumnWithIndex(column_index)) - column->deserializeAndInsertFromArena(in); + column->deserializeAndInsertFromArena(in, nullptr); else column->skipSerializedInArena(in); } @@ -478,7 +478,7 @@ class DictionaryKeysExtractor for (const auto & column : key_columns) { - StringRef serialized_data = column->serializeValueIntoArena(current_key_index, *complex_key_arena, block_start); + StringRef serialized_data = column->serializeValueIntoArena(current_key_index, *complex_key_arena, block_start, nullptr); allocated_size_for_columns += serialized_data.size; } diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index 25ea519dc7fc..9c5fa5508f69 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -1157,7 +1157,7 @@ class SSDCacheDictionaryStorage final : public ICacheDictionaryStorage for (size_t column_index = 0; column_index < columns_to_serialize_size; ++column_index) { auto & column = columns[column_index]; - temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_values_pool, block_start); + temporary_column_data[column_index] = column->serializeValueIntoArena(key_index, temporary_values_pool, block_start, nullptr); allocated_size_for_columns += temporary_column_data[column_index].size; } diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 3bf8396bb694..037ee25936d8 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -614,7 +614,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, M else { const char * data = nullptr; - value = &map[columns[arg_num]->serializeValueIntoArena(i, arena, data)]; + value = &map[columns[arg_num]->serializeValueIntoArena(i, arena, data, nullptr)]; } /// Here we count the number of element appearances, but no more than once per array. @@ -712,7 +712,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, M else { const char * data = nullptr; - pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); + pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data, nullptr)); } if (!current_has_nullable) @@ -755,7 +755,7 @@ void FunctionArrayIntersect::insertElement(typename Map::LookupResult & pa else { ReadBufferFromString in({pair->getKey().data, pair->getKey().size}); - result_data.deserializeAndInsertFromArena(in); + result_data.deserializeAndInsertFromArena(in, /*settings=*/nullptr); } if (use_null_map) null_map.push_back(0); diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 99bd2f4b7a93..f4b03364c073 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -226,13 +226,13 @@ static inline T ALWAYS_INLINE packFixed( /** Serialize keys into a continuous chunk of memory. */ static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous( /// NOLINT - size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, Arena & pool) + size_t i, size_t keys_size, const ColumnRawPtrs & key_columns, Arena & pool, const IColumn::SerializationSettings * settings) { const char * begin = nullptr; size_t sum_size = 0; for (size_t j = 0; j < keys_size; ++j) - sum_size += key_columns[j]->serializeValueIntoArena(i, pool, begin).size; + sum_size += key_columns[j]->serializeValueIntoArena(i, pool, begin, settings).size; return {begin, sum_size}; } diff --git a/src/Interpreters/AggregationMethod.cpp b/src/Interpreters/AggregationMethod.cpp index d5fbe360cb97..8106ce3bef65 100644 --- a/src/Interpreters/AggregationMethod.cpp +++ b/src/Interpreters/AggregationMethod.cpp @@ -6,7 +6,10 @@ namespace DB { template void AggregationMethodOneNumber::insertKeyIntoColumns( - const AggregationMethodOneNumber::Key & key, std::vector & key_columns, const Sizes & /*key_sizes*/) + const AggregationMethodOneNumber::Key & key, + std::vector & key_columns, + const Sizes & /*key_sizes*/, + const IColumn::SerializationSettings *) { ColumnFixedSizeHelper * column; if constexpr (nullable) @@ -49,7 +52,12 @@ template struct AggregationMethodOneNumber; template +<<<<<<< HEAD +void AggregationMethodStringNoCache::insertKeyIntoColumns( + std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) +======= void AggregationMethodStringNoCache::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) +>>>>>>> backport/25.8/90880 { if constexpr (nullable) { @@ -68,7 +76,12 @@ template struct AggregationMethodStringNoCache; template +<<<<<<< HEAD +void AggregationMethodFixedString::insertKeyIntoColumns( + std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) +======= void AggregationMethodFixedString::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) +>>>>>>> backport/25.8/90880 { assert_cast(*key_columns[0]).insertData(key.data, key.size); } @@ -78,7 +91,11 @@ template struct AggregationMethodFixedString +<<<<<<< HEAD +void AggregationMethodFixedStringNoCache::insertKeyIntoColumns(std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) +======= void AggregationMethodFixedStringNoCache::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) +>>>>>>> backport/25.8/90880 { if constexpr (nullable) assert_cast(*key_columns[0]).insertData(key.data, key.size); @@ -93,7 +110,10 @@ template struct AggregationMethodFixedStringNoCache void AggregationMethodSingleLowCardinalityColumn::insertKeyIntoColumns( - const Key & key, std::vector & key_columns_low_cardinality, const Sizes & /*key_sizes*/) + const Key & key, + std::vector & key_columns_low_cardinality, + const Sizes & /*key_sizes*/, + const IColumn::SerializationSettings *) { auto * col = assert_cast(key_columns_low_cardinality[0]); @@ -115,7 +135,8 @@ template struct AggregationMethodSingleLowCardinalityColumn -void AggregationMethodKeysFixed::insertKeyIntoColumns(const Key & key, std::vector & key_columns, const Sizes & key_sizes) +void AggregationMethodKeysFixed::insertKeyIntoColumns( + const Key & key, std::vector & key_columns, const Sizes & key_sizes, const IColumn::SerializationSettings *) { size_t keys_size = key_columns.size(); @@ -190,11 +211,16 @@ template struct AggregationMethodKeysFixed +<<<<<<< HEAD +void AggregationMethodSerialized::insertKeyIntoColumns( + std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings) +======= void AggregationMethodSerialized::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) +>>>>>>> backport/25.8/90880 { ReadBufferFromString buf({key.data, key.size}); for (auto & column : key_columns) - column->deserializeAndInsertFromArena(buf); + column->deserializeAndInsertFromArena(buf, settings); } template struct AggregationMethodSerialized; diff --git a/src/Interpreters/AggregationMethod.h b/src/Interpreters/AggregationMethod.h index a4ca3cdc4159..e885c65f9775 100644 --- a/src/Interpreters/AggregationMethod.h +++ b/src/Interpreters/AggregationMethod.h @@ -48,7 +48,7 @@ struct AggregationMethodOneNumber std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } // Insert the key from the hash table into columns. - static void insertKeyIntoColumns(const Key & key, std::vector & key_columns, const Sizes & /*key_sizes*/); + static void insertKeyIntoColumns(const Key & key, std::vector & key_columns, const Sizes & /*key_sizes*/, const IColumn::SerializationSettings * settings); }; /// For the case where there is one string key. @@ -81,7 +81,7 @@ struct AggregationMethodString std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) { static_cast(key_columns[0])->insertData(key.data, key.size); } @@ -117,7 +117,7 @@ struct AggregationMethodStringNoCache std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &); + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings); }; /// For the case where there is one fixed-length string key. @@ -150,7 +150,7 @@ struct AggregationMethodFixedString std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &); + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings); }; /// Same as above but without cache @@ -183,7 +183,7 @@ struct AggregationMethodFixedStringNoCache std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &); + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings); }; /// Single low cardinality column. @@ -214,8 +214,11 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(const Key & key, - std::vector & key_columns_low_cardinality, const Sizes & /*key_sizes*/); + static void insertKeyIntoColumns( + const Key & key, + std::vector & key_columns_low_cardinality, + const Sizes & /*key_sizes*/, + const IColumn::SerializationSettings * settings); }; /// For the case where all keys are of fixed length, and they fit in N (for example, 128) bits. @@ -259,7 +262,8 @@ struct AggregationMethodKeysFixed return State::shuffleKeyColumns(key_columns, key_sizes); } - static void insertKeyIntoColumns(const Key & key, std::vector & key_columns, const Sizes & key_sizes); + static void insertKeyIntoColumns( + const Key & key, std::vector & key_columns, const Sizes & key_sizes, const IColumn::SerializationSettings * settings); }; /** Aggregates by concatenating serialized key values. @@ -298,7 +302,7 @@ struct AggregationMethodSerialized std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } - static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &); + static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings); }; template diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index efdc0f61399a..139b0f6860bb 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -209,7 +209,8 @@ Aggregator::Params::Params( bool only_merge_, // true for projections bool optimize_group_by_constant_keys_, float min_hit_rate_to_use_consecutive_keys_optimization_, - const StatsCollectingParams & stats_collecting_params_) + const StatsCollectingParams & stats_collecting_params_, + bool serialize_string_with_zero_byte_) : keys(keys_) , keys_size(keys.size()) , aggregates(aggregates_) @@ -232,6 +233,7 @@ Aggregator::Params::Params( , optimize_group_by_constant_keys(optimize_group_by_constant_keys_) , min_hit_rate_to_use_consecutive_keys_optimization(min_hit_rate_to_use_consecutive_keys_optimization_) , stats_collecting_params(stats_collecting_params_) + , serialize_string_with_zero_byte(serialize_string_with_zero_byte_) { } @@ -276,7 +278,8 @@ Aggregator::Params::Params( bool overflow_row_, size_t max_threads_, size_t max_block_size_, - float min_hit_rate_to_use_consecutive_keys_optimization_) + float min_hit_rate_to_use_consecutive_keys_optimization_, + bool serialize_string_with_zero_byte_) : keys(keys_) , keys_size(keys.size()) , aggregates(aggregates_) @@ -286,6 +289,7 @@ Aggregator::Params::Params( , max_block_size(max_block_size_) , only_merge(true) , min_hit_rate_to_use_consecutive_keys_optimization(min_hit_rate_to_use_consecutive_keys_optimization_) + , serialize_string_with_zero_byte(serialize_string_with_zero_byte_) { } @@ -538,6 +542,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_) HashMethodContext::Settings cache_settings; cache_settings.max_threads = params.max_threads; + cache_settings.serialize_string_with_zero_byte = params.serialize_string_with_zero_byte; aggregation_state_cache = AggregatedDataVariants::createCache(method_chosen, cache_settings); #if USE_EMBEDDED_COMPILER @@ -2048,7 +2053,9 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are init_out_cols(); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref); + IColumn::SerializationSettings serialization_settings{ + .serialize_string_with_zero_byte = params.serialize_string_with_zero_byte}; + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref, &serialization_settings); if constexpr (is_final) { @@ -2314,7 +2321,9 @@ Aggregator::ConvertToBlockResVariant Aggregator::convertToBlockImplFinal( init_out_cols(); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref); + IColumn::SerializationSettings serialization_settings{ + .serialize_string_with_zero_byte = params.serialize_string_with_zero_byte}; + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref, &serialization_settings); places.emplace_back(mapped); /// Mark the cell as destroyed so it will not be destroyed in destructor. @@ -2387,7 +2396,9 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a init_out_cols(); const auto & key_sizes_ref = shuffled_key_sizes ? *shuffled_key_sizes : key_sizes; - method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref); + IColumn::SerializationSettings serialization_settings{ + .serialize_string_with_zero_byte = params.serialize_string_with_zero_byte}; + method.insertKeyIntoColumns(key, out_cols->raw_key_columns, key_sizes_ref, &serialization_settings); /// reserved, so push_back does not throw exceptions for (size_t i = 0; i < params.aggregates_size; ++i) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index b4ae4943a366..8e50243fe404 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -117,6 +117,8 @@ class Aggregator final const float min_hit_rate_to_use_consecutive_keys_optimization = 0.; StatsCollectingParams stats_collecting_params; + bool serialize_string_with_zero_byte = false; + static size_t getMaxBytesBeforeExternalGroupBy(size_t max_bytes_before_external_group_by, double max_bytes_ratio_before_external_group_by); Params( @@ -139,7 +141,8 @@ class Aggregator final bool only_merge_, // true for projections bool optimize_group_by_constant_keys_, float min_hit_rate_to_use_consecutive_keys_optimization_, - const StatsCollectingParams & stats_collecting_params_); + const StatsCollectingParams & stats_collecting_params_, + bool serialize_string_with_zero_byte_); /// Only parameters that matter during merge. Params( @@ -148,7 +151,8 @@ class Aggregator final bool overflow_row_, size_t max_threads_, size_t max_block_size_, - float min_hit_rate_to_use_consecutive_keys_optimization_); + float min_hit_rate_to_use_consecutive_keys_optimization_, + bool serialize_string_with_zero_byte_); Params cloneWithKeys(const Names & keys_, bool only_merge_ = false) { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e0ffd41fcdd3..679602c038e7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -195,6 +195,7 @@ namespace Setting extern const SettingsUInt64 max_rows_to_transfer; extern const SettingsOverflowMode transfer_overflow_mode; extern const SettingsString implicit_table_at_top_level; + extern const SettingsBool serialize_string_in_memory_with_zero_byte; } namespace ServerSetting @@ -2181,7 +2182,9 @@ static void executeMergeAggregatedImpl( overflow_row, settings[Setting::max_threads], settings[Setting::max_block_size], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization]); + settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], + settings[Setting::serialize_string_in_memory_with_zero_byte]); + auto grouping_sets_params = getAggregatorGroupingSetsParams(aggregation_keys_list, keys); auto merging_aggregated = std::make_unique( @@ -2805,8 +2808,8 @@ static Aggregator::Params getAggregatorParams( /* only_merge */ false, settings[Setting::optimize_group_by_constant_keys], settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - stats_collecting_params - }; + stats_collecting_params, + settings[Setting::serialize_string_in_memory_with_zero_byte]}; } void InterpreterSelectQuery::executeAggregation( diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2bdb558646ee..f5be70fa40e1 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -144,6 +144,7 @@ namespace Setting extern const SettingsUInt64 max_rows_to_transfer; extern const SettingsOverflowMode transfer_overflow_mode; extern const SettingsBool enable_parallel_blocks_marshalling; + extern const SettingsBool serialize_string_in_memory_with_zero_byte; } namespace ServerSetting @@ -517,7 +518,8 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context /* only_merge */ false, settings[Setting::optimize_group_by_constant_keys], settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - stats_collecting_params); + stats_collecting_params, + settings[Setting::serialize_string_in_memory_with_zero_byte]); return aggregator_params; } @@ -627,7 +629,8 @@ void addMergingAggregatedStep(QueryPlan & query_plan, query_analysis_result.aggregate_overflow_row, max_threads, settings[Setting::max_block_size], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization]); + settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], + settings[Setting::serialize_string_in_memory_with_zero_byte]); bool is_remote_storage = false; bool parallel_replicas_from_merge_tree = false; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 840f8d694c46..16b00a3797de 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -51,6 +51,7 @@ namespace QueryPlanSerializationSetting extern const QueryPlanSerializationSettingsUInt64 min_free_disk_space_for_temporary_data; extern const QueryPlanSerializationSettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; extern const QueryPlanSerializationSettingsBool optimize_group_by_constant_keys; + extern const QueryPlanSerializationSettingsBool serialize_string_in_memory_with_zero_byte; } namespace ErrorCodes @@ -867,8 +868,8 @@ std::unique_ptr AggregatingStep::deserialize(Deserialization & c /* only_merge */ false, ctx.settings[QueryPlanSerializationSetting::optimize_group_by_constant_keys], ctx.settings[QueryPlanSerializationSetting::min_hit_rate_to_use_consecutive_keys_optimization], - stats_collecting_params - }; + stats_collecting_params, + ctx.settings[QueryPlanSerializationSetting::serialize_string_in_memory_with_zero_byte]}; SortDescription sort_description_for_merging; diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index f990d5dc5201..57ad612212cf 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -24,6 +24,7 @@ namespace QueryPlanSerializationSetting extern const QueryPlanSerializationSettingsUInt64 max_size_to_preallocate_for_aggregation; extern const QueryPlanSerializationSettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; extern const QueryPlanSerializationSettingsBool distributed_aggregation_memory_efficient; + extern const QueryPlanSerializationSettingsBool serialize_string_in_memory_with_zero_byte; } namespace Setting @@ -309,7 +310,8 @@ std::unique_ptr MergingAggregatedStep::deserialize(Deserializati overflow_row, settings[Setting::max_threads], ctx.settings[QueryPlanSerializationSetting::max_block_size], - ctx.settings[QueryPlanSerializationSetting::min_hit_rate_to_use_consecutive_keys_optimization]); + ctx.settings[QueryPlanSerializationSetting::min_hit_rate_to_use_consecutive_keys_optimization], + ctx.settings[QueryPlanSerializationSetting::serialize_string_in_memory_with_zero_byte]); auto merging_aggregated_step = std::make_unique( ctx.input_headers.front(), diff --git a/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp b/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp index 6e9e7d0beb27..1a9492c4775e 100644 --- a/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp +++ b/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp @@ -80,7 +80,9 @@ namespace DB DECLARE(UInt64, join_output_by_rowlist_perkey_rows_threshold, 5, "The lower limit of per-key average rows in the right table to determine whether to output by row list in hash join.", 0) \ DECLARE(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \ DECLARE(UInt64, join_to_sort_minimum_perkey_rows, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys", 0) \ - DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) + DECLARE(UInt64, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) \ + \ + DECLARE(Bool, serialize_string_in_memory_with_zero_byte, true, "Serialize String values during aggregation with zero byte at the end. Enable to keep compatibility when querying cluster of incompatible versions.", 0) \ // clang-format on diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index bfb79672636e..79f45ff1f255 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -22,6 +22,7 @@ namespace Setting extern const SettingsUInt64 min_count_to_compile_aggregate_expression; extern const SettingsUInt64 min_free_disk_space_for_temporary_data; extern const SettingsBool optimize_group_by_constant_keys; + extern const SettingsBool serialize_string_in_memory_with_zero_byte; } TTLAggregationAlgorithm::TTLAggregationAlgorithm( @@ -65,7 +66,7 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm( /*only_merge=*/false, settings[Setting::optimize_group_by_constant_keys], settings[Setting::min_chunk_bytes_for_parallel_parsing], - /*stats_collecting_params_=*/{}); + settings[Setting::serialize_string_in_memory_with_zero_byte]); aggregator = std::make_unique(header, params); diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index 0508541a2c10..86c1bb44b997 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -90,7 +90,8 @@ MergingAggregatedTransform::MergingAggregatedTransform( params.overflow_row, params.max_threads, params.max_block_size, - params.min_hit_rate_to_use_consecutive_keys_optimization); + params.min_hit_rate_to_use_consecutive_keys_optimization, + params.serialize_string_with_zero_byte); auto transform_params = std::make_shared(std::make_shared(reordering.updateHeader(in_header)), std::move(set_params), final); diff --git a/tests/integration/test_string_aggregation_compatibility/__init__.py b/tests/integration/test_string_aggregation_compatibility/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_string_aggregation_compatibility/test.py b/tests/integration/test_string_aggregation_compatibility/test.py new file mode 100644 index 000000000000..5e916fd7f195 --- /dev/null +++ b/tests/integration/test_string_aggregation_compatibility/test.py @@ -0,0 +1,108 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance("node1") +node2 = cluster.add_instance("node2") +node256 = cluster.add_instance( + "node256", + image="clickhouse/clickhouse-server", + tag="25.6", + with_installed_binary=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_string_aggregation_compatibility(started_cluster): + def create_tables(node, other_node_name): + node.query( + "DROP TABLE IF EXISTS repro1 SYNC; CREATE TABLE repro1 (id UInt64, s1 String, s2 LowCardinality(String)) ENGINE = MergeTree ORDER BY id" + ) + node.query( + "INSERT INTO repro1 SELECT number, 'somestring', 'somestring' FROM numbers(10000)" + ) + node.query( + f"CREATE TABLE IF NOT EXISTS dist_repro1 (id UInt64, s1 String, s2 LowCardinality(String)) AS remote('{other_node_name}', 'default.repro1', 'default', '')" + ) + node.query( + "CREATE TABLE IF NOT EXISTS global_repro1 (id UInt64, s1 String, s2 LowCardinality(String)) ENGINE = Merge('default', '.*repro1')" + ) + + create_tables(node1, other_node_name=node256.name) + create_tables(node256, other_node_name=node1.name) + + def run_query(node, extra_settings={}): + return int( + node.query( + """ + SELECT count() + FROM + ( + SELECT + id, + s2 + FROM global_repro1 + GROUP BY ALL + )""", + settings={"group_by_two_level_threshold": 1} | extra_settings, + ) + ) + + assert run_query(node1) == 10000 + assert run_query(node256) == 10000 + assert ( + run_query(node1, extra_settings={"serialize_string_in_memory_with_zero_byte": 0}) + == 20000 + ) + +def test_string_aggregation_compatibility_setting(started_cluster): + def create_tables(n, other_node_name): + n.query( + "DROP TABLE IF EXISTS repro2 SYNC; CREATE TABLE repro2 (id UInt64, s1 String, s2 LowCardinality(String)) ENGINE = MergeTree ORDER BY id" + ) + n.query( + "INSERT INTO repro2 SELECT number, 'somestring', 'somestring' FROM numbers(10000)" + ) + n.query( + f"CREATE TABLE IF NOT EXISTS dist_repro2 (id UInt64, s1 String, s2 LowCardinality(String)) AS remote('{other_node_name}', 'default.repro2', 'default', '')" + ) + n.query( + "CREATE TABLE IF NOT EXISTS global_repro2 (id UInt64, s1 String, s2 LowCardinality(String)) ENGINE = Merge('default', '.*repro2')" + ) + + create_tables(node1, other_node_name=node2.name) + create_tables(node2, other_node_name=node1.name) + + def run_query(n, extra_settings={}): + return int( + n.query( + """ + SELECT count() + FROM + ( + SELECT + id, + s2 + FROM global_repro2 + GROUP BY ALL + )""", + settings={"group_by_two_level_threshold": 1} | extra_settings, + ) + ) + + assert run_query(node1) == 10000 + assert run_query(node2) == 10000 + assert ( + run_query(node1, extra_settings={"serialize_string_in_memory_with_zero_byte": 0}) + == 10000 + ) From de6c709c0a5f0548ade4fa7deb12e441f94886ff Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Dec 2025 15:13:59 +0000 Subject: [PATCH 032/113] Backport #91152 to 25.8: Fix possible inconsistent dynamic structure during writing in compact parts --- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnBLOB.h | 1 + src/Columns/ColumnCompressed.h | 1 + src/Columns/ColumnDynamic.cpp | 9 +++++++++ src/Columns/ColumnDynamic.h | 1 + src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnObject.cpp | 7 +++++++ src/Columns/ColumnObject.h | 1 + src/Columns/ColumnSparse.h | 1 + src/Columns/ColumnTuple.cpp | 5 +++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVariant.cpp | 6 ++++++ src/Columns/ColumnVariant.h | 1 + src/Columns/IColumn.h | 2 ++ .../MergeTree/MergeTreeDataPartWriterCompact.cpp | 4 ++++ ...03745_fix_dynamic_structure_in_compact_part.reference | 0 .../03745_fix_dynamic_structure_in_compact_part.sql | 6 ++++++ 18 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.reference create mode 100644 tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.sql diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index acadc0163304..1e6536df0323 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -214,6 +214,7 @@ class ColumnArray final : public COWHelper, ColumnArr bool hasDynamicStructure() const override { return getData().hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override { data->fixDynamicStructure(); } bool dynamicStructureEquals(const IColumn & rhs) const override { diff --git a/src/Columns/ColumnBLOB.h b/src/Columns/ColumnBLOB.h index 7075d145d43f..88099da80088 100644 --- a/src/Columns/ColumnBLOB.h +++ b/src/Columns/ColumnBLOB.h @@ -203,6 +203,7 @@ class ColumnBLOB : public COWHelper, ColumnBLOB> bool hasDynamicStructure() const override { throwInapplicable(); } void takeDynamicStructureFromSourceColumns(const Columns &) override { throwInapplicable(); } void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwInapplicable(); } + void fixDynamicStructure() override { throwInapplicable(); } private: /// Compressed and serialized representation of the wrapped column. diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index ba998f3ce27c..bc5870e721a8 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -137,6 +137,7 @@ class ColumnCompressed : public COWHelper, Colum bool hasDynamicStructure() const override { throwMustBeDecompressed(); } void takeDynamicStructureFromSourceColumns(const Columns &) override { throwMustBeDecompressed(); } void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwMustBeDecompressed(); } + void fixDynamicStructure() override { throwMustBeDecompressed(); } protected: size_t rows; diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 7a4cbd6cc1c3..bceb09a9944e 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1429,6 +1429,15 @@ void ColumnDynamic::takeDynamicStructureFromColumn(const ColumnPtr & source_colu variant_col.getVariantByGlobalDiscriminator(i).takeDynamicStructureFromColumn(source_variant_column.getVariantPtrByGlobalDiscriminator(i)); } +void ColumnDynamic::fixDynamicStructure() +{ + /// Reduce max_dynamic_types to the number of selected variants, so there will be no possibility + /// to extend selected variants on inserts into this column. + /// -1 because we don't count shared variant in the limit. + max_dynamic_types = variant_info.variant_names.size() - 1; + getVariantColumn().fixDynamicStructure(); +} + void ColumnDynamic::applyNullMap(const ColumnVector::Container & null_map) { variant_column_ptr->applyNullMap(null_map); diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index f4e5d6989284..4d79c2d723d2 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -392,6 +392,7 @@ class ColumnDynamic final : public COWHelper, Colum bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override; const StatisticsPtr & getStatistics() const { return statistics; } void setStatistics(const StatisticsPtr & statistics_) { statistics = statistics_; } diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 6a1e09def754..785f21d83783 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -128,6 +128,7 @@ class ColumnMap final : public COWHelper, ColumnMap> bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override { nested->fixDynamicStructure(); } }; } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index d2bceca3bbe6..74e154ee4a1c 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -233,6 +233,7 @@ class ColumnNullable final : public COWHelper, Col bool hasDynamicStructure() const override { return nested_column->hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override { nested_column->fixDynamicStructure(); } bool dynamicStructureEquals(const IColumn & rhs) const override; private: diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 03f031d4e562..62ebcbc54e63 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1851,6 +1851,13 @@ void ColumnObject::takeDynamicStructureFromColumn(const ColumnPtr & source_colum statistics = source_object.getStatistics(); } +void ColumnObject::fixDynamicStructure() +{ + /// Set max_dynamic_paths to the number of dynamic paths. + /// It's needed to avoid adding new unexpected dynamic paths during later inserts into this column. + max_dynamic_paths = dynamic_paths.size(); +} + size_t ColumnObject::findPathLowerBoundInSharedData(StringRef path, const ColumnString & shared_data_paths, size_t start, size_t end) { /// Simple random access iterator over values in ColumnString in specified range. diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 35190711908b..08adafb136e0 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -206,6 +206,7 @@ class ColumnObject final : public COWHelper, ColumnO bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override; const PathToColumnMap & getTypedPaths() const { return typed_paths; } PathToColumnMap & getTypedPaths() { return typed_paths; } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 091054cb5e1d..fc8467c6a8a8 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -173,6 +173,7 @@ class ColumnSparse final : public COWHelper, ColumnS bool hasDynamicStructure() const override { return values->hasDynamicStructure(); } void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override { values->fixDynamicStructure(); } size_t getNumberOfTrailingDefaults() const { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 42922dc3c531..a82d4fa9c7c3 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -905,6 +905,11 @@ void ColumnTuple::takeDynamicStructureFromColumn(const ColumnPtr & source_column columns[i]->takeDynamicStructureFromColumn(source_elements[i]); } +void ColumnTuple::fixDynamicStructure() +{ + for (auto & column : columns) + column->fixDynamicStructure(); +} ColumnPtr ColumnTuple::compress(bool force_compression) const { diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index a0d27e53e166..83c22fca6245 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -150,6 +150,7 @@ class ColumnTuple final : public COWHelper, ColumnTup bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override; /// Empty tuple needs a public method to manage its size. void addSize(size_t delta) { column_length += delta; } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 6cb706f13295..bcf4a9fbcd95 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1789,5 +1789,11 @@ void ColumnVariant::takeDynamicStructureFromColumn(const ColumnPtr & source_colu getVariantByGlobalDiscriminator(i).takeDynamicStructureFromColumn(source_variant.getVariantPtrByGlobalDiscriminator(i)); } +void ColumnVariant::fixDynamicStructure() +{ + for (auto & variant : variants) + variant->fixDynamicStructure(); +} + } diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 8d91fb018c22..49708d277468 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -347,6 +347,7 @@ class ColumnVariant final : public COWHelper, Colum bool dynamicStructureEquals(const IColumn & rhs) const override; void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; + void fixDynamicStructure() override; private: void insertFromImpl(const IColumn & src_, size_t n, const std::vector * global_discriminators_mapping); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index cda959a27676..9e76deca3dec 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -671,6 +671,8 @@ class IColumn : public COW virtual void takeDynamicStructureFromSourceColumns(const std::vector & /*source_columns*/) {} /// For columns with dynamic subcolumns this method takes the exact dynamic structure from provided column. virtual void takeDynamicStructureFromColumn(const ColumnPtr & /*source_column*/) {} + /// For columns with dynamic subcolumns fix current dynamic structure so later inserts into this column won't change it. + virtual void fixDynamicStructure() {} /** Some columns can contain another columns inside. * So, we have a tree of columns. But not all combinations are possible. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 73af75859859..3a406fe1219f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -503,7 +503,11 @@ void MergeTreeDataPartWriterCompact::ColumnsBuffer::add(MutableColumns && column else { for (size_t i = 0; i < columns.size(); ++i) + { + /// Fix dynamic structure so it won't changed after insertion of new rows. + accumulated_columns[i]->fixDynamicStructure(); accumulated_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + } } } diff --git a/tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.reference b/tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.sql b/tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.sql new file mode 100644 index 000000000000..83421e486acc --- /dev/null +++ b/tests/queries/0_stateless/03745_fix_dynamic_structure_in_compact_part.sql @@ -0,0 +1,6 @@ +drop table if exists test; +create table test (id UInt64, json JSON) engine=CoalescingMergeTree order by id settings min_bytes_for_wide_part='100G', merge_max_block_size=33, index_granularity=800; +insert into test select number, '{}' from numbers(10000); +alter table test update json = '{"a" : 42}' where id > 9000 settings mutations_sync=1; +optimize table test final; +drop table test; From 3cf92fa5de6dcd7e9974d25ef386a9a1072af6ab Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Dec 2025 18:16:05 +0000 Subject: [PATCH 033/113] Backport #91304 to 25.8: Fix SYSTEM DROP FILESYSTEM CACHE ON CLUSTER --- src/Parsers/ASTSystemQuery.cpp | 11 ++++++- ...drop_filesystem_cache_on_cluster.reference | 2 ++ ...system_drop_filesystem_cache_on_cluster.sh | 31 +++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.reference create mode 100755 tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.sh diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 9cdd034f2ca3..19054ce7d7e0 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -163,7 +163,13 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti print_keyword("SYSTEM") << " "; print_keyword(typeToString(type)); - if (!cluster.empty()) + + std::unordered_set queries_with_on_cluster_at_end = { + Type::DROP_FILESYSTEM_CACHE, + Type::SYNC_FILESYSTEM_CACHE, + }; + + if (!queries_with_on_cluster_at_end.contains(type) && !cluster.empty()) formatOnCluster(ostr, settings); switch (type) @@ -515,6 +521,9 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti case Type::END: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown SYSTEM command"); } + + if (queries_with_on_cluster_at_end.contains(type) && !cluster.empty()) + formatOnCluster(ostr, settings); } diff --git a/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.reference b/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.reference new file mode 100644 index 000000000000..1444d39d9578 --- /dev/null +++ b/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.reference @@ -0,0 +1,2 @@ +localhost 9000 0 0 0 +localhost 9000 0 0 0 diff --git a/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.sh b/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.sh new file mode 100755 index 000000000000..8f40316ac5d1 --- /dev/null +++ b/tests/queries/0_stateless/03643_system_drop_filesystem_cache_on_cluster.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-object-storage, no-random-settings + +# set -x + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" +$CLICKHOUSE_CLIENT -m --query """ +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); + +INSERT INTO test SELECT 1, 'test'; +""" + +$CLICKHOUSE_CLIENT --query """ +SYSTEM SYNC FILESYSTEM CACHE '$disk_name' ON CLUSTER 'test_shard_localhost'; +""" + +$CLICKHOUSE_CLIENT --query """ +SYSTEM DROP FILESYSTEM CACHE '$disk_name' ON CLUSTER 'test_shard_localhost'; +""" + +$CLICKHOUSE_CLIENT --query """ +DROP TABLE IF EXISTS test; +""" From 778562ba6dbbc3d64f610e0790536d6535b14695 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Dec 2025 18:18:37 +0000 Subject: [PATCH 034/113] Backport #91142 to 25.8: Fix lazy materialization of a column added via alter add column in old parts --- .../Serializations/SerializationArray.cpp | 14 +++++++++++--- src/DataTypes/Serializations/SerializationArray.h | 2 +- .../SerializationObjectSharedData.cpp | 4 +++- ...ation_of_array_after_alter_add_column.reference | 1 + ...rialization_of_array_after_alter_add_column.sql | 7 +++++++ 5 files changed, 23 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.reference create mode 100644 tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.sql diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 4edd64a4334f..74405f8c1864 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -372,7 +372,7 @@ void SerializationArray::serializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } -void SerializationArray::deserializeOffsetsBinaryBulk( +bool SerializationArray::deserializeOffsetsBinaryBulk( ColumnPtr & offsets_column, size_t limit, ISerialization::DeserializeBinaryBulkSettings & settings, @@ -390,8 +390,11 @@ void SerializationArray::deserializeOffsetsBinaryBulk( insertArraySizesToOffsets(offsets_column, cached_column, cached_column->size() - num_read_rows, cached_column->size()); else offsets_column = arraySizesToOffsets(*cached_column); + + return true; } - else if (auto * stream = settings.getter(settings.path)) + + if (auto * stream = settings.getter(settings.path)) { size_t prev_size = offsets_column->size(); @@ -416,7 +419,11 @@ void SerializationArray::deserializeOffsetsBinaryBulk( /// Add array sizes read from current range into the cache. if (cache) addColumnWithNumReadRowsToSubstreamsCache(cache, settings.path, arrayOffsetsToSizes(*offsets_column), offsets_column->size() - prev_size); + + return true; } + + return false; } std::pair SerializationArray::deserializeOffsetsBinaryBulkAndGetNestedOffsetAndLimit( @@ -429,7 +436,8 @@ std::pair SerializationArray::deserializeOffsetsBinaryBulkAndGet const auto & offsets_data = assert_cast(*offsets_column).getData(); size_t prev_last_offset = offsets_data.back(); size_t prev_offset_size = offsets_data.size(); - deserializeOffsetsBinaryBulk(offsets_column, offset + limit, settings, cache); + if (!deserializeOffsetsBinaryBulk(offsets_column, offset + limit, settings, cache)) + return {0, 0}; size_t skipped_nested_rows = 0; diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index badbdee45368..fde4e238631c 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -74,7 +74,7 @@ class SerializationArray final : public SimpleTextSerialization SubstreamsCache * cache) const override; static void serializeOffsetsBinaryBulk(const IColumn & offsets_column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings); - static void deserializeOffsetsBinaryBulk(ColumnPtr & offsets_column, size_t limit, DeserializeBinaryBulkSettings & settings, SubstreamsCache * cache); + static bool deserializeOffsetsBinaryBulk(ColumnPtr & offsets_column, size_t limit, DeserializeBinaryBulkSettings & settings, SubstreamsCache * cache); static std::pair deserializeOffsetsBinaryBulkAndGetNestedOffsetAndLimit(ColumnPtr & offsets_column, size_t offset, size_t limit, DeserializeBinaryBulkSettings & settings, SubstreamsCache * cache); struct SubcolumnCreator : public ISubcolumnCreator diff --git a/src/DataTypes/Serializations/SerializationObjectSharedData.cpp b/src/DataTypes/Serializations/SerializationObjectSharedData.cpp index c43d7ab76b0a..52676c4c09d2 100644 --- a/src/DataTypes/Serializations/SerializationObjectSharedData.cpp +++ b/src/DataTypes/Serializations/SerializationObjectSharedData.cpp @@ -1325,7 +1325,9 @@ void SerializationObjectSharedData::deserializeBinaryBulkWithMultipleStreams( /// Read array sizes. settings.path.push_back(Substream::ObjectSharedDataCopySizes); - SerializationArray::deserializeOffsetsBinaryBulk(offsets_column, rows_offset + limit, settings, cache); + if (!SerializationArray::deserializeOffsetsBinaryBulk(offsets_column, rows_offset + limit, settings, cache)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream for object shared data copy sizes"); + settings.path.pop_back(); /// Read paths indexes. diff --git a/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.reference b/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.reference new file mode 100644 index 000000000000..f70bede2109c --- /dev/null +++ b/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.reference @@ -0,0 +1 @@ +42 [] diff --git a/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.sql b/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.sql new file mode 100644 index 000000000000..27958a300f27 --- /dev/null +++ b/tests/queries/0_stateless/03742_lazy_materialization_of_array_after_alter_add_column.sql @@ -0,0 +1,7 @@ +drop table if exists test_lazy; +create table test_lazy (id UInt64) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1; +insert into test_lazy select * from numbers(100); +alter table test_lazy add column array Array(UInt64) settings mutations_sync=1; +select id, array from test_lazy where id = 42 order by id limit 10 settings query_plan_optimize_lazy_materialization = 1; +drop table test_lazy; + From 691889ba1f9fcb979d00399b393b4d079cc05114 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Dec 2025 00:33:41 +0000 Subject: [PATCH 035/113] Backport #90985 to 25.8: Libssh forward to master --- contrib/libssh | 2 +- contrib/libssh-cmake/CMakeLists.txt | 10 ++- .../libssh-cmake/linux/aarch64-musl/config.h | 6 +- contrib/libssh-cmake/linux/aarch64/config.h | 6 +- .../libssh-cmake/linux/loongarch64/config.h | 6 +- contrib/libssh-cmake/linux/ppc64le/config.h | 6 +- contrib/libssh-cmake/linux/riscv64/config.h | 6 +- contrib/libssh-cmake/linux/s390x/config.h | 6 +- .../libssh-cmake/linux/x86-64-musl/config.h | 6 +- contrib/libssh-cmake/linux/x86-64/config.h | 79 ++++++++++--------- src/Common/SSHWrapper.cpp | 73 +++++++---------- src/Server/SSH/SSHPtyHandler.cpp | 4 +- src/Server/SSH/SSHPtyHandlerFactory.h | 8 +- src/Server/SSH/SSHSession.cpp | 25 +++--- src/Server/SSH/SSHSession.h | 5 +- 15 files changed, 125 insertions(+), 123 deletions(-) diff --git a/contrib/libssh b/contrib/libssh index ed4011b91873..47305a2f7257 160000 --- a/contrib/libssh +++ b/contrib/libssh @@ -1 +1 @@ -Subproject commit ed4011b91873836713576475a98cd132cd834539 +Subproject commit 47305a2f7257b56ca407260a72af85db058d551f diff --git a/contrib/libssh-cmake/CMakeLists.txt b/contrib/libssh-cmake/CMakeLists.txt index 4cc3e2a18312..0e1fae5880af 100644 --- a/contrib/libssh-cmake/CMakeLists.txt +++ b/contrib/libssh-cmake/CMakeLists.txt @@ -35,6 +35,7 @@ set(libssh_SRCS ${LIB_SOURCE_DIR}/src/external/blowfish.c ${LIB_SOURCE_DIR}/src/external/chacha.c ${LIB_SOURCE_DIR}/src/external/poly1305.c + ${LIB_SOURCE_DIR}/src/external/sntrup761.c ${LIB_SOURCE_DIR}/src/getpass.c ${LIB_SOURCE_DIR}/src/init.c ${LIB_SOURCE_DIR}/src/kdf.c @@ -61,6 +62,7 @@ set(libssh_SRCS ${LIB_SOURCE_DIR}/src/string.c ${LIB_SOURCE_DIR}/src/threads.c ${LIB_SOURCE_DIR}/src/token.c + ${LIB_SOURCE_DIR}/src/ttyopts.c ${LIB_SOURCE_DIR}/src/wrapper.c # some files of libssh/src/ are missing - why? @@ -69,15 +71,21 @@ set(libssh_SRCS # files missing - why? # LIBCRYPT specific + ${LIB_SOURCE_DIR}/src/crypto_common.c + ${LIB_SOURCE_DIR}/src/curve25519_crypto.c ${LIB_SOURCE_DIR}/src/dh_crypto.c ${LIB_SOURCE_DIR}/src/ecdh_crypto.c + ${LIB_SOURCE_DIR}/src/getrandom_crypto.c + ${LIB_SOURCE_DIR}/src/gzip.c ${LIB_SOURCE_DIR}/src/libcrypto.c + ${LIB_SOURCE_DIR}/src/md_crypto.c ${LIB_SOURCE_DIR}/src/pki_crypto.c + ${LIB_SOURCE_DIR}/src/pki_context.c + ${LIB_SOURCE_DIR}/src/sntrup761.c ${LIB_SOURCE_DIR}/src/threads/libcrypto.c ${LIB_SOURCE_DIR}/src/bind.c ${LIB_SOURCE_DIR}/src/bind_config.c - ${LIB_SOURCE_DIR}/src/options.c ${LIB_SOURCE_DIR}/src/server.c ) diff --git a/contrib/libssh-cmake/linux/aarch64-musl/config.h b/contrib/libssh-cmake/linux/aarch64-musl/config.h index 15236527fdf8..9cc21c1df4ad 100644 --- a/contrib/libssh-cmake/linux/aarch64-musl/config.h +++ b/contrib/libssh-cmake/linux/aarch64-musl/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/aarch64/config.h b/contrib/libssh-cmake/linux/aarch64/config.h index e65ccb8ba3e6..7e21b1b4f683 100644 --- a/contrib/libssh-cmake/linux/aarch64/config.h +++ b/contrib/libssh-cmake/linux/aarch64/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/loongarch64/config.h b/contrib/libssh-cmake/linux/loongarch64/config.h index aa684ca29a35..3e19e6ab945d 100644 --- a/contrib/libssh-cmake/linux/loongarch64/config.h +++ b/contrib/libssh-cmake/linux/loongarch64/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/ppc64le/config.h b/contrib/libssh-cmake/linux/ppc64le/config.h index c56b1ad03344..701ee00416bb 100644 --- a/contrib/libssh-cmake/linux/ppc64le/config.h +++ b/contrib/libssh-cmake/linux/ppc64le/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/riscv64/config.h b/contrib/libssh-cmake/linux/riscv64/config.h index 33c91bf542b9..ca0868072bdc 100644 --- a/contrib/libssh-cmake/linux/riscv64/config.h +++ b/contrib/libssh-cmake/linux/riscv64/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/s390x/config.h b/contrib/libssh-cmake/linux/s390x/config.h index 289a8cabc8e4..6c284c9d6d4f 100644 --- a/contrib/libssh-cmake/linux/s390x/config.h +++ b/contrib/libssh-cmake/linux/s390x/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/x86-64-musl/config.h b/contrib/libssh-cmake/linux/x86-64-musl/config.h index fd7c2e2b0c12..9b325957c358 100644 --- a/contrib/libssh-cmake/linux/x86-64-musl/config.h +++ b/contrib/libssh-cmake/linux/x86-64-musl/config.h @@ -82,13 +82,13 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 /* Define to 1 if you have DSA */ diff --git a/contrib/libssh-cmake/linux/x86-64/config.h b/contrib/libssh-cmake/linux/x86-64/config.h index 4090c5a45ad1..f6316af195cd 100644 --- a/contrib/libssh-cmake/linux/x86-64/config.h +++ b/contrib/libssh-cmake/linux/x86-64/config.h @@ -82,64 +82,37 @@ /* Define to 1 if you have the header file. */ #define HAVE_PTHREAD_H 1 -/* Define to 1 if you have eliptic curve cryptography in openssl */ +/* Define to 1 if you have elliptic curve cryptography in openssl */ #define HAVE_OPENSSL_ECC 1 -/* Define to 1 if you have eliptic curve cryptography in gcrypt */ +/* Define to 1 if you have elliptic curve cryptography in gcrypt */ /* #undef HAVE_GCRYPT_ECC */ -/* Define to 1 if you have eliptic curve cryptography */ +/* Define to 1 if you have elliptic curve cryptography */ #define HAVE_ECC 1 -/* Define to 1 if you have DSA */ -/* #undef HAVE_DSA */ - -/* Define to 1 if you have gl_flags as a glob_t sturct member */ +/* Define to 1 if you have gl_flags as a glob_t struct member */ #define HAVE_GLOB_GL_FLAGS_MEMBER 1 -/* Define to 1 if you have OpenSSL with Ed25519 support */ -#define HAVE_OPENSSL_ED25519 1 +/* Define to 1 if you have gcrypt with ChaCha20/Poly1305 support */ +/* #undef HAVE_GCRYPT_CHACHA_POLY */ -/* Define to 1 if you have OpenSSL with X25519 support */ -#define HAVE_OPENSSL_X25519 1 +/* Define to 1 if you have gcrypt with curve25519 support */ +/* #undef HAVE_GCRYPT_CURVE25519 */ /*************************** FUNCTIONS ***************************/ -/* Define to 1 if you have the `EVP_aes128_ctr' function. */ -#define HAVE_OPENSSL_EVP_AES_CTR 1 - -/* Define to 1 if you have the `EVP_aes128_cbc' function. */ -#define HAVE_OPENSSL_EVP_AES_CBC 1 - -/* Define to 1 if you have the `EVP_aes128_gcm' function. */ -/* #undef HAVE_OPENSSL_EVP_AES_GCM */ - -/* Define to 1 if you have the `CRYPTO_THREADID_set_callback' function. */ -#define HAVE_OPENSSL_CRYPTO_THREADID_SET_CALLBACK 1 - -/* Define to 1 if you have the `CRYPTO_ctr128_encrypt' function. */ -#define HAVE_OPENSSL_CRYPTO_CTR128_ENCRYPT 1 +/* Define to 1 if you have the `EVP_chacha20' function. */ +#define HAVE_OPENSSL_EVP_CHACHA20 1 -/* Define to 1 if you have the `EVP_CIPHER_CTX_new' function. */ -#define HAVE_OPENSSL_EVP_CIPHER_CTX_NEW 1 - -/* Define to 1 if you have the `EVP_KDF_CTX_new_id' function. */ -/* #undef HAVE_OPENSSL_EVP_KDF_CTX_NEW_ID */ +/* Define to 1 if you have the `EVP_KDF_CTX_new_id' or `EVP_KDF_CTX_new` function. */ +#define HAVE_OPENSSL_EVP_KDF_CTX 1 /* Define to 1 if you have the `FIPS_mode' function. */ #if USE_BORINGSSL #define HAVE_OPENSSL_FIPS_MODE 1 #endif -/* Define to 1 if you have the `EVP_DigestSign' function. */ -#define HAVE_OPENSSL_EVP_DIGESTSIGN 1 - -/* Define to 1 if you have the `EVP_DigestVerify' function. */ -#define HAVE_OPENSSL_EVP_DIGESTVERIFY 1 - -/* Define to 1 if you have the `OPENSSL_ia32cap_loc' function. */ -/* #undef HAVE_OPENSSL_IA32CAP_LOC */ - /* Define to 1 if you have the `snprintf' function. */ #define HAVE_SNPRINTF 1 @@ -212,6 +185,12 @@ /* Define to 1 if you have the `cmocka_set_test_filter' function. */ /* #undef HAVE_CMOCKA_SET_TEST_FILTER */ +/* Define to 1 if we have support for blowfish */ +/* #undef HAVE_BLOWFISH */ + +/* Define to 1 if we have support for ML-KEM */ +/* #undef HAVE_MLKEM */ + /*************************** LIBRARIES ***************************/ /* Define to 1 if you have the `crypto' library (-lcrypto). */ @@ -229,6 +208,10 @@ /* Define to 1 if you have the `cmocka' library (-lcmocka). */ /* #undef HAVE_CMOCKA */ +/* Define to 1 if you have the `libfido2' library (-lfido2). + * This is required for interacting with FIDO2/U2F devices over USB-HID. */ +/* #undef HAVE_LIBFIDO2 */ + /**************************** OPTIONS ****************************/ #define HAVE_GCC_THREAD_LOCAL_STORAGE 1 @@ -236,6 +219,7 @@ #define HAVE_FALLTHROUGH_ATTRIBUTE 1 #define HAVE_UNUSED_ATTRIBUTE 1 +/* #undef HAVE_WEAK_ATTRIBUTE */ #define HAVE_CONSTRUCTOR_ATTRIBUTE 1 #define HAVE_DESTRUCTOR_ATTRIBUTE 1 @@ -262,6 +246,14 @@ /* Define to 1 if you want to enable DH group exchange algorithms */ /* #undef WITH_GEX */ +/* Define to 1 if you want to enable insecure none cipher and MAC */ +/* #undef WITH_INSECURE_NONE */ + +/* Define to 1 if you want to allow libssh to execute arbitrary commands from + * configuration files or options (match exec, proxy commands and OpenSSH-based + * proxy-jumps). */ +/* #undef WITH_EXEC */ + /* Define to 1 if you want to enable blowfish cipher support */ /* #undef WITH_BLOWFISH_CIPHER */ @@ -280,6 +272,15 @@ /* Define to 1 if you want to enable NaCl support */ /* #undef WITH_NACL */ +/* Define to 1 if you want to enable PKCS #11 URI support */ +/* #undef WITH_PKCS11_URI */ + +/* Define to 1 if we want to build a support for PKCS #11 provider. */ +/* #undef WITH_PKCS11_PROVIDER */ + +/* Define to 1 if you want to enable FIDO2/U2F support */ +/* #undef WITH_FIDO2 */ + /*************************** ENDIAN *****************************/ /* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most diff --git a/src/Common/SSHWrapper.cpp b/src/Common/SSHWrapper.cpp index a7933f260327..36457d28ac8f 100644 --- a/src/Common/SSHWrapper.cpp +++ b/src/Common/SSHWrapper.cpp @@ -21,36 +21,14 @@ namespace ErrorCodes namespace { - -class SSHString +struct SSHStringDeleter { -public: - explicit SSHString(std::string_view input) - { - if (string = ssh_string_new(input.size()); string == nullptr) - throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString"); - if (int rc = ssh_string_fill(string, input.data(), input.size()); rc != SSH_OK) - throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString"); - } - - explicit SSHString(ssh_string other) { string = other; } - - ssh_string get() { return string; } - - String toString() - { - return {ssh_string_get_char(string), ssh_string_len(string)}; - } - - ~SSHString() - { - ssh_string_free(string); - } - -private: - ssh_string string; + void operator()(char * ptr) const { ssh_string_free_char(ptr); } +}; +struct CStringDeleter +{ + void operator()(char * ptr) const { std::free(ptr); } }; - } SSHKey SSHKeyFactory::makePrivateKeyFromFile(String filename, String passphrase) @@ -119,20 +97,31 @@ bool SSHKey::isEqual(const SSHKey & other) const String SSHKey::signString(std::string_view input) const { - SSHString input_str(input); - ssh_string output = nullptr; - if (int rc = pki_sign_string(key, input_str.get(), &output); rc != SSH_OK) - throw Exception(ErrorCodes::LIBSSH_ERROR, "Error singing with ssh key"); - SSHString output_str(output); - return output_str.toString(); + char * signature = nullptr; + if (int rc = sshsig_sign(input.data(), input.size(), key, nullptr, "clickhouse", SSHSIG_DIGEST_SHA2_256, &signature); rc != SSH_OK) + throw Exception(ErrorCodes::LIBSSH_ERROR, "Error signing with ssh key"); + std::unique_ptr sig_ptr(signature); + return String(sig_ptr.get()); } bool SSHKey::verifySignature(std::string_view signature, std::string_view original) const { - SSHString sig(signature); - SSHString orig(original); - int rc = pki_verify_string(key, sig.get(), orig.get()); - return rc == SSH_OK; + ssh_key verify_key = nullptr; + String sig_str(signature); + int rc = sshsig_verify(original.data(), original.size(), sig_str.c_str(), "clickhouse", &verify_key); + if (rc != SSH_OK) + { + if (verify_key != nullptr) + ssh_key_free(verify_key); + return false; + } + bool keys_match = false; + if (verify_key != nullptr) + { + keys_match = (ssh_key_cmp(key, verify_key, SSH_KEY_CMP_PUBLIC) == 0); + ssh_key_free(verify_key); + } + return keys_match; } bool SSHKey::isPrivate() const @@ -145,14 +134,6 @@ bool SSHKey::isPublic() const return ssh_key_is_public(key); } -namespace -{ - struct CStringDeleter - { - void operator()(char * ptr) const { std::free(ptr); } - }; -} - String SSHKey::getBase64() const { char * buf = nullptr; diff --git a/src/Server/SSH/SSHPtyHandler.cpp b/src/Server/SSH/SSHPtyHandler.cpp index 9e70077e69db..105b39fdd1ec 100644 --- a/src/Server/SSH/SSHPtyHandler.cpp +++ b/src/Server/SSH/SSHPtyHandler.cpp @@ -479,7 +479,9 @@ SSHPtyHandler::~SSHPtyHandler() void SSHPtyHandler::run() { ::ssh::SSHEvent event; - SessionCallback sdata(session, server, socket().peerAddress(), options); + auto peer_addr = socket().peerAddress(); + socket().close(); + SessionCallback sdata(session, server, peer_addr, options); session.handleKeyExchange(); event.addSession(session); int max_iterations = options.auth_timeout_seconds * 1000 / options.event_poll_interval_milliseconds; diff --git a/src/Server/SSH/SSHPtyHandlerFactory.h b/src/Server/SSH/SSHPtyHandlerFactory.h index f02a554804b2..1865bb57c744 100644 --- a/src/Server/SSH/SSHPtyHandlerFactory.h +++ b/src/Server/SSH/SSHPtyHandlerFactory.h @@ -5,6 +5,7 @@ #if USE_SSH && defined(OS_LINUX) #include +#include #include #include @@ -28,6 +29,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int SSH_EXCEPTION; } class SSHPtyHandlerFactory : public TCPServerConnectionFactory @@ -79,9 +81,11 @@ class SSHPtyHandlerFactory : public TCPServerConnectionFactory LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString()); ::ssh::libsshLogger::initialize(); ::ssh::SSHSession session; - session.disableSocketOwning(); session.disableDefaultConfig(); - ssh_bind.acceptFd(session, socket.sockfd()); + int duplicated_fd = dup(socket.sockfd()); + if (duplicated_fd == -1) + throw Exception(ErrorCodes::SSH_EXCEPTION, "Failed to duplicate socket file descriptor"); + ssh_bind.acceptFd(session, duplicated_fd); auto options = SSHPtyHandler::Options { diff --git a/src/Server/SSH/SSHSession.cpp b/src/Server/SSH/SSHSession.cpp index 8227b0d495d4..de0cd39ce4c7 100644 --- a/src/Server/SSH/SSHSession.cpp +++ b/src/Server/SSH/SSHSession.cpp @@ -3,6 +3,7 @@ #if USE_SSH && defined(OS_LINUX) #include +#include #include #include @@ -19,7 +20,7 @@ namespace ErrorCodes namespace ssh { -SSHSession::SSHSession() : session(ssh_new()) +SSHSession::SSHSession() : session(ssh_new()), disconnected(false) { if (!session) throw DB::Exception(DB::ErrorCodes::SSH_EXCEPTION, "Failed to create ssh_session"); @@ -39,7 +40,9 @@ SSHSession::SSHSession(SSHSession && rhs) noexcept SSHSession & SSHSession::operator=(SSHSession && rhs) noexcept { this->session = rhs.session; + this->disconnected = rhs.disconnected; rhs.session = nullptr; + rhs.disconnected = true; return *this; } @@ -64,14 +67,6 @@ void SSHSession::disableDefaultConfig() throw DB::Exception(DB::ErrorCodes::SSH_EXCEPTION, "Failed disabling default config for ssh session due due to {}", getError()); } -void SSHSession::disableSocketOwning() -{ - bool owns_socket = false; - int rc = ssh_options_set(session, SSH_OPTIONS_OWNS_SOCKET, &owns_socket); - if (rc != SSH_OK) - throw DB::Exception(DB::ErrorCodes::SSH_EXCEPTION, "Failed disabling socket owning for ssh session due to {}", getError()); -} - void SSHSession::setPeerHost(const String & host) { int rc = ssh_options_set(session, SSH_OPTIONS_HOST, host.c_str()); @@ -106,7 +101,17 @@ void SSHSession::handleKeyExchange() void SSHSession::disconnect() { - ssh_disconnect(session); + std::lock_guard lock(disconnect_mutex); + if (!disconnected) + { + // Ensure disconnect is called only once per session + disconnected = true; + // Use global mutex to prevent concurrent ssh_disconnect() calls across ALL sessions + // because libssh has shared global state that is not thread-safe + static std::mutex global_disconnect_mutex; + std::lock_guard global_lock(global_disconnect_mutex); + ssh_disconnect(session); + } } String SSHSession::getError() diff --git a/src/Server/SSH/SSHSession.h b/src/Server/SSH/SSHSession.h index 93736b19a354..43765b98b2d5 100644 --- a/src/Server/SSH/SSHSession.h +++ b/src/Server/SSH/SSHSession.h @@ -5,6 +5,7 @@ #if USE_SSH && defined(OS_LINUX) #include +#include #include struct ssh_session_struct; @@ -30,8 +31,6 @@ class SSHSession /// Disable reading default libssh configuration void disableDefaultConfig(); - /// Disable session from closing socket. Can be used when a socket is passed. - void disableSocketOwning(); /// Connect / disconnect void connect(); @@ -53,6 +52,8 @@ class SSHSession private: SessionPtr session = nullptr; + bool disconnected = false; + std::mutex disconnect_mutex; }; } From f837db0a3a6d51eedc8209a6f52ad382a100b663 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Dec 2025 13:24:01 +0000 Subject: [PATCH 036/113] Backport #91270 to 25.8: Fix inserting into CoalescingMergeTree column with Tuple of JSON/Dynamic and LowCardinality --- .../Algorithms/SummingSortedAlgorithm.cpp | 41 +++++-------------- ...uple_low_cardinality_and_dynamic.reference | 3 ++ ...tree_tuple_low_cardinality_and_dynamic.sql | 12 ++++++ 3 files changed, 26 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.reference create mode 100644 tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.sql diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 468e05e53835..3bd8068ffdbb 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -500,19 +500,12 @@ static void postprocessChunk( auto column = std::move(columns[next_column]); ++next_column; - if (!desc.is_agg_func_type && !desc.is_simple_agg_func_type && isTuple(desc.function->getResultType())) + if (!desc.aggregate_all_columns && !desc.is_agg_func_type && !desc.is_simple_agg_func_type && isTuple(desc.function->getResultType())) { - if (desc.aggregate_all_columns) - { - res_columns[desc.column_numbers[0]] = column; - } - else - { - /// Unpack tuple into block. - size_t tuple_size = desc.column_numbers.size(); - for (size_t i = 0; i < tuple_size; ++i) - res_columns[desc.column_numbers[i]] = assert_cast(*column).getColumnPtr(i); - } + /// Unpack tuple into block. + const size_t tuple_size = desc.column_numbers.size(); + for (size_t i = 0; i < tuple_size; ++i) + res_columns[desc.column_numbers[i]] = assert_cast(*column).getColumnPtr(i); } else if (desc.nested_type) { @@ -590,26 +583,14 @@ void SummingSortedAlgorithm::SummingMergedData::initialize(const DB::Block & hea for (const auto & desc : def.columns_to_aggregate) { // Wrap aggregated columns in a tuple to match function signature - if (!desc.is_agg_func_type && !desc.is_simple_agg_func_type && isTuple(desc.function->getResultType())) + if (!desc.aggregate_all_columns && !desc.is_agg_func_type && !desc.is_simple_agg_func_type && isTuple(desc.function->getResultType())) { - if (desc.aggregate_all_columns) - { - auto column = desc.real_type->createColumn(); - size_t tuple_size = static_cast(*column).tupleSize(); - MutableColumns tuple_columns(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = static_cast(*column).getColumnPtr(i)->cloneEmpty(); - new_columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); - } - else - { - size_t tuple_size = desc.column_numbers.size(); - MutableColumns tuple_columns(tuple_size); - for (size_t i = 0; i < tuple_size; ++i) - tuple_columns[i] = std::move(columns[desc.column_numbers[i]]); + const size_t tuple_size = desc.column_numbers.size(); + MutableColumns tuple_columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = std::move(columns[desc.column_numbers[i]]); - new_columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); - } + new_columns.emplace_back(ColumnTuple::create(std::move(tuple_columns))); } else { diff --git a/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.reference b/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.reference new file mode 100644 index 000000000000..390bec1d914d --- /dev/null +++ b/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.reference @@ -0,0 +1,3 @@ +Dynamic paths +a +Shared data parhs diff --git a/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.sql b/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.sql new file mode 100644 index 000000000000..cd8ddcf8900a --- /dev/null +++ b/tests/queries/0_stateless/03748_coalescing_merge_tree_tuple_low_cardinality_and_dynamic.sql @@ -0,0 +1,12 @@ +set mutations_sync=1; +drop table if exists test; +create table test (id UInt64, t Tuple(a LowCardinality(String), json JSON)) engine=CoalescingMergeTree order by id settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, index_granularity=32, merge_max_block_size=32; +insert into test select number, tuple('str', '{}') from numbers(100); +alter table test update t = tuple('str', '{"a" : 42}') where id > 90; +optimize table test final; +select 'Dynamic paths'; +select distinct arrayJoin(JSONDynamicPaths(t.json)) from test; +select 'Shared data parhs'; +select distinct arrayJoin(JSONSharedDataPaths(t.json)) from test; +drop table test; + From 1d1dec68eb6fafc8dfdedfc6e93f3a467134c88d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Dec 2025 15:14:17 +0000 Subject: [PATCH 037/113] Backport #91210 to 25.8: Add assertion for having empty metadata_files --- .../ObjectStorage/DataLakes/Iceberg/Utils.cpp | 16 ++++++++++++++++ .../03401_several_iceberg_tables_in_one_dir.sql | 1 + 2 files changed, 17 insertions(+) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp index 8cc23f59d729..bbb9d110ab9f 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp @@ -699,6 +699,22 @@ MetadataFileWithInfo getLatestMetadataFileAndVersion( } } + if (metadata_files_with_versions.empty()) + { + if (table_uuid.has_value() && use_table_uuid_for_metadata_file_selection) + { + throw Exception( + ErrorCodes::FILE_DOESNT_EXIST, + "The metadata file for Iceberg table with path {} and table UUID {} doesn't exist", + table_path, + table_uuid.value()); + } + throw Exception( + ErrorCodes::FILE_DOESNT_EXIST, + "The metadata file for Iceberg table with path {} doesn't exist", + table_path); + } + /// Get the latest version of metadata file: v.metadata.json const ShortMetadataFileInfo & latest_metadata_file_info = [&]() { diff --git a/tests/queries/0_stateless/03401_several_iceberg_tables_in_one_dir.sql b/tests/queries/0_stateless/03401_several_iceberg_tables_in_one_dir.sql index 2f73135fcfe6..bb537a9f5f07 100644 --- a/tests/queries/0_stateless/03401_several_iceberg_tables_in_one_dir.sql +++ b/tests/queries/0_stateless/03401_several_iceberg_tables_in_one_dir.sql @@ -4,6 +4,7 @@ SELECT * FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_table_uuid = 'ea8d1178-7756-4b89-b21f-00e9f31fe03e') ORDER BY id; SELECT * FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_table_uuid = 'A90EED4CF74B4E5BB630096FB9D09021') ORDER BY id; SELECT * FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_table_uuid = '6f6f6407_c6A5465f_A808ea8900_e35a38') ORDER BY id; +SELECT * FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_table_uuid = '88005553-5352-8222-8993-abacaba01010') ORDER BY id; -- { serverError FILE_DOESNT_EXIST } SELECT count() FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_file_path = 'metadata/00001-aec4e034-3f73-48f7-87ad-51b7b42a8db7.metadata.json'); SELECT count() FROM icebergS3(s3_conn, filename='merged_several_tables_test', SETTINGS iceberg_metadata_file_path = 'metadata/00001-2aad93a8-a893-4943-8504-f6021f83ecab.metadata.json'); From 6d5db7e82d3d3bbbd0629bb1d13d6afabe6dccdb Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 3 Dec 2025 16:35:23 +0100 Subject: [PATCH 038/113] Fix --- src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp index bbb9d110ab9f..0af80e6fb98d 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp @@ -701,18 +701,18 @@ MetadataFileWithInfo getLatestMetadataFileAndVersion( if (metadata_files_with_versions.empty()) { - if (table_uuid.has_value() && use_table_uuid_for_metadata_file_selection) + if (table_uuid.has_value()) { throw Exception( ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} and table UUID {} doesn't exist", - table_path, + configuration_ptr->getPathForRead().path, table_uuid.value()); } throw Exception( ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} doesn't exist", - table_path); + configuration_ptr->getPathForRead().path); } /// Get the latest version of metadata file: v.metadata.json From 2f33c29c82147b91e8323e06093a4924a3bf9103 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 2 Dec 2025 16:57:06 +0100 Subject: [PATCH 039/113] Fix build --- .../Combinators/AggregateFunctionDistinct.h | 2 +- src/Columns/ColumnObjectDeprecated.h | 6 ++-- src/Columns/ColumnTuple.cpp | 1 - src/Columns/ColumnUnique.h | 33 +------------------ src/Columns/IColumn.cpp | 2 +- src/Common/ColumnsHashing.h | 2 +- src/Interpreters/AggregationMethod.cpp | 24 +++----------- src/Interpreters/InterpreterSelectQuery.cpp | 1 - .../TTL/TTLAggregationAlgorithm.cpp | 1 + 9 files changed, 12 insertions(+), 60 deletions(-) diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h index eedf13a7281e..6351664d20ad 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h @@ -119,7 +119,7 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi auto settings = IColumn::SerializationSettings::createForAggregationState(); auto cur_ref = columns[i]->serializeValueIntoArena(row_num, *arena, begin, &settings); value.data = cur_ref.data - value.size; - value.size += cur_ref.size + value.size += cur_ref.size; } Set::LookupResult it; diff --git a/src/Columns/ColumnObjectDeprecated.h b/src/Columns/ColumnObjectDeprecated.h index c7dd2b66158c..d0d3bee34739 100644 --- a/src/Columns/ColumnObjectDeprecated.h +++ b/src/Columns/ColumnObjectDeprecated.h @@ -250,9 +250,9 @@ class ColumnObjectDeprecated final : public COWHelper #include #include -#include #include diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index f39092602298..85cceba778f3 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -457,12 +457,8 @@ void ColumnUnique::collectSerializedValueSizes(PaddedPODArray -<<<<<<< HEAD -std::string_view ColumnUnique::serializeValueIntoArena( +StringRef ColumnUnique::serializeValueIntoArena( size_t n, Arena & arena, char const *& begin, const IColumn::SerializationSettings * settings) const -======= -StringRef ColumnUnique::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const ->>>>>>> backport/25.8/90880 { if (is_nullable) { @@ -502,34 +498,7 @@ char * ColumnUnique::serializeValueIntoMemory(size_t n, char * memor } template -<<<<<<< HEAD size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(ReadBuffer & in, const IColumn::SerializationSettings * settings) -======= -StringRef ColumnUnique::serializeAggregationStateValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - if (is_nullable) - { - static constexpr auto s = sizeof(UInt8); - - auto * pos = arena.allocContinue(s, begin); - UInt8 flag = (n == getNullValueIndex() ? 1 : 0); - unalignedStore(pos, flag); - - if (n == getNullValueIndex()) - return StringRef(pos, s); - - auto nested_ref = column_holder->serializeAggregationStateValueIntoArena(n, arena, begin); - - /// serializeAggregationStateValueIntoArena may reallocate memory. Have to use ptr from nested_ref.data and move it back. - return StringRef(nested_ref.data - s, nested_ref.size + s); - } - - return column_holder->serializeAggregationStateValueIntoArena(n, arena, begin); -} - -template -size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(ReadBuffer & in) ->>>>>>> backport/25.8/90880 { if (is_nullable) { diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index b940304b4195..937f15da89b3 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -173,7 +173,7 @@ void IColumn::batchSerializeValueIntoMemoryWithNull( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method batchSerializeValueIntoMemoryWithNull is not supported for {}", getName()); } -void IColumn::collectSerializedValueSizes(PaddedPODArray & sizes, const UInt8 * is_null, const SerializationSettings * settings) const +void IColumn::collectSerializedValueSizes(PaddedPODArray & /* sizes */, const UInt8 * /* is_null */, const SerializationSettings * /*settings*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method collectSerializedValueSizes is not supported for {}", getName()); } diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 6e1f47cac5b0..dca81c4162ec 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -369,7 +369,7 @@ struct HashMethodSerialized PaddedPODArray row_sizes; size_t total_size = 0; IColumn::SerializationSettings serialization_settings; - PaddedPODArray serialized_buffer; + PODArray serialized_buffer; std::vector serialized_keys; HashMethodSerialized(const ColumnRawPtrs & key_columns_, const Sizes & /*key_sizes*/, const HashMethodContextPtr & context) diff --git a/src/Interpreters/AggregationMethod.cpp b/src/Interpreters/AggregationMethod.cpp index 8106ce3bef65..d45049ed3654 100644 --- a/src/Interpreters/AggregationMethod.cpp +++ b/src/Interpreters/AggregationMethod.cpp @@ -52,12 +52,8 @@ template struct AggregationMethodOneNumber; template -<<<<<<< HEAD void AggregationMethodStringNoCache::insertKeyIntoColumns( - std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) -======= -void AggregationMethodStringNoCache::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) ->>>>>>> backport/25.8/90880 + StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) { if constexpr (nullable) { @@ -76,12 +72,8 @@ template struct AggregationMethodStringNoCache; template -<<<<<<< HEAD void AggregationMethodFixedString::insertKeyIntoColumns( - std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) -======= -void AggregationMethodFixedString::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) ->>>>>>> backport/25.8/90880 + StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) { assert_cast(*key_columns[0]).insertData(key.data, key.size); } @@ -91,11 +83,7 @@ template struct AggregationMethodFixedString -<<<<<<< HEAD -void AggregationMethodFixedStringNoCache::insertKeyIntoColumns(std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) -======= -void AggregationMethodFixedStringNoCache::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) ->>>>>>> backport/25.8/90880 +void AggregationMethodFixedStringNoCache::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings *) { if constexpr (nullable) assert_cast(*key_columns[0]).insertData(key.data, key.size); @@ -211,12 +199,8 @@ template struct AggregationMethodKeysFixed -<<<<<<< HEAD void AggregationMethodSerialized::insertKeyIntoColumns( - std::string_view key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings) -======= -void AggregationMethodSerialized::insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) ->>>>>>> backport/25.8/90880 + StringRef key, std::vector & key_columns, const Sizes &, const IColumn::SerializationSettings * settings) { ReadBufferFromString buf({key.data, key.size}); for (auto & column : key_columns) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 679602c038e7..684922d596ce 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2184,7 +2184,6 @@ static void executeMergeAggregatedImpl( settings[Setting::max_block_size], settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], settings[Setting::serialize_string_in_memory_with_zero_byte]); - auto grouping_sets_params = getAggregatorGroupingSetsParams(aggregation_keys_list, keys); auto merging_aggregated = std::make_unique( diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index 79f45ff1f255..8acf2c24ae36 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -66,6 +66,7 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm( /*only_merge=*/false, settings[Setting::optimize_group_by_constant_keys], settings[Setting::min_chunk_bytes_for_parallel_parsing], + /*stats_collecting_params_=*/{}, settings[Setting::serialize_string_in_memory_with_zero_byte]); aggregator = std::make_unique(header, params); From dcdc888dd96226f6cbdd0f9c6d69933f09e3d6d1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 4 Dec 2025 00:34:02 +0000 Subject: [PATCH 040/113] Backport #91396 to 25.8: Remove Sparse columns from header in merge algorithms --- src/Processors/Merges/Algorithms/MergedData.cpp | 3 +++ ...ing_merge_tree_sparse_columns_in_header.reference | 0 ...oalescing_merge_tree_sparse_columns_in_header.sql | 12 ++++++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.reference create mode 100644 tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.sql diff --git a/src/Processors/Merges/Algorithms/MergedData.cpp b/src/Processors/Merges/Algorithms/MergedData.cpp index d2aa197c5f7f..61681632eef5 100644 --- a/src/Processors/Merges/Algorithms/MergedData.cpp +++ b/src/Processors/Merges/Algorithms/MergedData.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -27,6 +28,8 @@ void MergedData::initialize(const Block & header, const IMergingAlgorithm::Input for (size_t i = 0; i != columns.size(); ++i) { + /// Sometimes header can contain Sparse columns, we don't support Sparse in merge algorithms. + columns[i] = recursiveRemoveSparse(std::move(columns[i]))->assumeMutable(); if (columns[i]->hasDynamicStructure()) columns[i]->takeDynamicStructureFromSourceColumns(source_columns[i]); } diff --git a/tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.reference b/tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.sql b/tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.sql new file mode 100644 index 000000000000..27327607f5a0 --- /dev/null +++ b/tests/queries/0_stateless/03751_summing_coalescing_merge_tree_sparse_columns_in_header.sql @@ -0,0 +1,12 @@ +drop table if exists src; +create table src (x UInt64) engine=MergeTree order by tuple(); +insert into src select 0 from numbers(1000000); +drop table if exists dst; +create table dst (x UInt64) engine=CoalescingMergeTree order by tuple(); +insert into dst select * from src; +drop table dst; +create table dst (x UInt64) engine=SummingMergeTree order by tuple(); +insert into dst select * from src; +drop table dst; +drop table src; + From b66cc20883151f0fb623f35bd7ba7226c7ae2784 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 4 Dec 2025 14:13:53 +0000 Subject: [PATCH 041/113] Backport #90927 to 25.8: Fix SummingMergeTree aggregation for Nested LowCardinality columns --- .../Merges/Algorithms/SummingSortedAlgorithm.cpp | 2 +- ...33_summing_merge_tree_nested_low_cardinality.reference | 1 + .../03733_summing_merge_tree_nested_low_cardinality.sql | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.reference create mode 100644 tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.sql diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 3bd8068ffdbb..753f0a21230b 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -415,7 +415,7 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( { const ColumnWithTypeAndName & key_col = header.safeGetByPosition(*column_num_it); const String & name = key_col.name; - const IDataType & nested_type = *assert_cast(*key_col.type).getNestedType(); + const IDataType & nested_type = *recursiveRemoveLowCardinality(assert_cast(*key_col.type).getNestedType()); if (column_num_it == map.second.begin() || endsWith(name, "ID") diff --git a/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.reference b/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.reference new file mode 100644 index 000000000000..e11be14429f0 --- /dev/null +++ b/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.reference @@ -0,0 +1 @@ +lol 10 ['a','b'] [4,6] diff --git a/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.sql b/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.sql new file mode 100644 index 000000000000..6b43952640bf --- /dev/null +++ b/tests/queries/0_stateless/03733_summing_merge_tree_nested_low_cardinality.sql @@ -0,0 +1,8 @@ +CREATE TABLE sums (key LowCardinality(String), sumOfSums UInt64, sumsMap Nested (key LowCardinality(String), sum UInt64)) ENGINE = SummingMergeTree PRIMARY KEY (key); + +INSERT INTO sums (key, sumOfSums, sumsMap.key, sumsMap.sum) VALUES ('lol', 3, ['a', 'b'], [1, 2]); +INSERT INTO sums (key, sumOfSums, sumsMap.key, sumsMap.sum) VALUES ('lol', 7, ['a', 'b'], [3, 4]); + +OPTIMIZE TABLE sums; + +SELECT * FROM sums; From 43943e7e339b27bf630ba68b3fa7a6b894b95b7a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 5 Dec 2025 11:14:41 +0000 Subject: [PATCH 042/113] Backport #91386 to 25.8: Reset async_insert value in the queue --- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- ...03748_async_insert_reset_setting.reference | 20 ++++++ .../03748_async_insert_reset_setting.sh | 72 +++++++++++++++++++ 3 files changed, 94 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03748_async_insert_reset_setting.reference create mode 100755 tests/queries/0_stateless/03748_async_insert_reset_setting.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 7c48a8322f71..c50904547f03 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -826,8 +826,8 @@ try /// 'resetParser' doesn't work for parallel parsing. key.settings->set("input_format_parallel_parsing", false); /// It maybe insert into distributed table. - /// It doesn't make sense to make insert into destination tables asynchronous. - key.settings->set("async_insert", false); + /// We want the remote part to decide if the insert will be async or not. + key.settings->setDefaultValue("async_insert"); insert_context->makeQueryContext(); diff --git a/tests/queries/0_stateless/03748_async_insert_reset_setting.reference b/tests/queries/0_stateless/03748_async_insert_reset_setting.reference new file mode 100644 index 000000000000..7d9d0e6aacec --- /dev/null +++ b/tests/queries/0_stateless/03748_async_insert_reset_setting.reference @@ -0,0 +1,20 @@ +2 +4 +1 test1 +1 test1 +2 test2 +2 test2 +3 test3 +3 test3 +4 test4 +4 test4 +5 test5 +5 test5 +6 test6 +6 test6 +7 test7 +7 test7 +8 test8 +8 test8 +9 test9 +9 test9 diff --git a/tests/queries/0_stateless/03748_async_insert_reset_setting.sh b/tests/queries/0_stateless/03748_async_insert_reset_setting.sh new file mode 100755 index 000000000000..8816d406fcbf --- /dev/null +++ b/tests/queries/0_stateless/03748_async_insert_reset_setting.sh @@ -0,0 +1,72 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SYNC_USER="${CLICKHOUSE_DATABASE}_sync_user" +ASYNC_USER="${CLICKHOUSE_DATABASE}_async_user" + +${CLICKHOUSE_CLIENT} --multiquery < Date: Mon, 8 Dec 2025 13:23:04 +0000 Subject: [PATCH 043/113] Backport #91573 to 25.8: Disallow ALTERs for Object Storage table engines --- .../ObjectStorage/StorageObjectStorageConfiguration.h | 11 ++++++++++- .../03629_storage_s3_disallow_index_alter.reference | 0 .../03629_storage_s3_disallow_index_alter.sql | 10 ++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.reference create mode 100644 tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.sql diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 6929c7018d67..223ced7ff203 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -213,7 +213,16 @@ class StorageObjectStorageConfiguration const std::optional & /*format_settings*/) {} virtual void checkMutationIsPossible(const MutationCommands & /*commands*/) {} - virtual void checkAlterIsPossible(const AlterCommands & /*commands*/) {} + virtual void checkAlterIsPossible(const AlterCommands & commands) + { + for (const auto & command : commands) + { + if (!command.isCommentAlter()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", + command.type, getEngineName()); + } + } + virtual void alter(const AlterCommands & /*params*/, ContextPtr /*context*/) {} virtual const DataLakeStorageSettings & getDataLakeSettings() const diff --git a/tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.reference b/tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.sql b/tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.sql new file mode 100644 index 000000000000..fb0384aa8d39 --- /dev/null +++ b/tests/queries/0_stateless/03629_storage_s3_disallow_index_alter.sql @@ -0,0 +1,10 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on S3 +-- Issue: https://github.com/ClickHouse/ClickHouse/issues/87059 + +DROP TABLE IF EXISTS test_03629; +CREATE TABLE test_03629 (a UInt64) ENGINE = S3(s3_conn, filename='test_03629_{_partition_id}', format='Native') PARTITION BY a; +ALTER TABLE test_03629 ADD INDEX a_idx a TYPE set(0); -- { serverError NOT_IMPLEMENTED } +ALTER TABLE test_03629 ADD PROJECTION a_proj (SELECT a + 1 ORDER BY a); -- { serverError NOT_IMPLEMENTED } + +DROP TABLE test_03629; From e9adc3a61dbf9a5812a12d51abcfb2e36f4279ef Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Dec 2025 13:24:39 +0000 Subject: [PATCH 044/113] Backport #91040 to 25.8: Fix reading dynamic subcolumns from materialized columns in mutations --- src/Interpreters/TreeRewriter.cpp | 7 +------ src/Interpreters/TreeRewriter.h | 3 --- ...ubcolumns_of_materialized_columns_in_mutation.reference | 1 + ...3741_subcolumns_of_materialized_columns_in_mutation.sql | 7 +++++++ 4 files changed, 9 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.reference create mode 100644 tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 40ceb2303a5c..e1eed6fc6f27 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1024,11 +1024,6 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) source_columns.insert(source_columns.end(), columns_from_storage.begin(), columns_from_storage.end()); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - source_columns_ordinary = metadata_snapshot->getColumns().getOrdinary(); - } - else - { - source_columns_ordinary = source_columns; } source_columns_set = removeDuplicateColumns(source_columns); @@ -1225,7 +1220,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select /// Check for subcolumns in unknown required columns. if (!unknown_required_source_columns.empty() && (!storage || storage->supportsSubcolumns())) { - for (const NameAndTypePair & pair : source_columns_ordinary) + for (const NameAndTypePair & pair : required_source_columns) { for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();) { diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 28ff6988b076..aee585969bcf 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -34,9 +34,6 @@ struct TreeRewriterResult /// Same as above but also record alias columns which are expanded. This is for RBAC access check. Names required_source_columns_before_expanding_alias_columns; - /// Set of columns that object columns are not extended. This is for distinguishing JSON and Tuple type. - NamesAndTypesList source_columns_ordinary; - NameSet missed_subcolumns; /// Set of alias columns that are expanded to their alias expressions. We still need the original columns to check access permission. diff --git a/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.reference b/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.reference new file mode 100644 index 000000000000..0967ef424bce --- /dev/null +++ b/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.reference @@ -0,0 +1 @@ +{} diff --git a/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.sql b/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.sql new file mode 100644 index 000000000000..187011654ab4 --- /dev/null +++ b/tests/queries/0_stateless/03741_subcolumns_of_materialized_columns_in_mutation.sql @@ -0,0 +1,7 @@ +drop table if exists test; +create table test (s String, json JSON materialized s) engine=MergeTree order by tuple(); +insert into test select '{"a" : 42}'; +alter table test update s = '{}' where json.a = 42 settings mutations_sync=1; +select * from test; +drop table test; + From 94e64933095651a6f19e78dcdd0fb48f78a6630b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Dec 2025 14:15:18 +0000 Subject: [PATCH 045/113] Backport #91437 to 25.8: explicit POCO request for REST-polaris catalog --- src/Databases/DataLake/RestCatalog.cpp | 53 +++++++++++++++++--------- 1 file changed, 35 insertions(+), 18 deletions(-) diff --git a/src/Databases/DataLake/RestCatalog.cpp b/src/Databases/DataLake/RestCatalog.cpp index 3006c68e6083..cd409cadbd26 100644 --- a/src/Databases/DataLake/RestCatalog.cpp +++ b/src/Databases/DataLake/RestCatalog.cpp @@ -28,6 +28,11 @@ #include #include #include +#include +#include +#include +#include +#include namespace DB::ErrorCodes @@ -203,12 +208,11 @@ std::string RestCatalog::retrieveAccessToken() const /// 1. support oauth2-server-uri /// https://github.com/apache/iceberg/blob/918f81f3c3f498f46afcea17c1ac9cdc6913cb5c/open-api/rest-catalog-open-api.yaml#L183C82-L183C99 - DB::HTTPHeaderEntries headers; - headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); - headers.emplace_back("Accepts", "application/json; charset=UTF-8"); - Poco::URI url; DB::ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback; + size_t body_size = 0; + String body; + if (oauth_server_uri.empty() && !oauth_server_use_request_body) { url = Poco::URI(base_url / oauth_tokens_endpoint); @@ -223,11 +227,20 @@ std::string RestCatalog::retrieveAccessToken() const } else { + String encoded_auth_scope; + String encoded_client_id; + String encoded_client_secret; + Poco::URI::encode(auth_scope, auth_scope, encoded_auth_scope); + Poco::URI::encode(client_id, client_id, encoded_client_id); + Poco::URI::encode(client_secret, client_secret, encoded_client_secret); + + body = fmt::format( + "grant_type=client_credentials&scope={}&client_id={}&client_secret={}", + encoded_auth_scope, encoded_client_id, encoded_client_secret); + body_size = body.size(); out_stream_callback = [&](std::ostream & os) { - os << fmt::format( - "grant_type=client_credentials&scope={}&client_id={}&client_secret={}", - auth_scope, client_id, client_secret); + os << body; }; if (oauth_server_uri.empty()) @@ -237,19 +250,23 @@ std::string RestCatalog::retrieveAccessToken() const } const auto & context = getContext(); - auto wb = DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withMethod(Poco::Net::HTTPRequest::HTTP_POST) - .withSettings(context->getReadSettings()) - .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) - .withHostFilter(&context->getRemoteHostFilter()) - .withOutCallback(std::move(out_stream_callback)) - .withSkipNotFound(false) - .withHeaders(headers) - .create(credentials); + auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + auto session = makeHTTPSession(DB::HTTPConnectionGroupType::HTTP, url, timeouts, {}); + + Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, url.getPathAndQuery(), + Poco::Net::HTTPMessage::HTTP_1_1); + request.setContentType("application/x-www-form-urlencoded"); + request.setContentLength(body_size); + request.set("Accept", "application/json"); + + std::ostream & os = session->sendRequest(request); + out_stream_callback(os); + + Poco::Net::HTTPResponse response; + std::istream & rs = session->receiveResponse(response); std::string json_str; - readJSONObjectPossiblyInvalid(json_str, *wb); + Poco::StreamCopier::copyToString(rs, json_str); Poco::JSON::Parser parser; Poco::Dynamic::Var res_json = parser.parse(json_str); From b8d98870a5f74c88d2f65677a7b6280dc6267c9f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Dec 2025 17:14:39 +0000 Subject: [PATCH 046/113] Backport #90666 to 25.8: Fix toLowCardinality type cast error with nullable columns --- src/Functions/toLowCardinality.cpp | 10 ++++ src/Storages/MergeTree/KeyCondition.cpp | 6 ++- ...7_tolowcardinality_nullable_cast.reference | 8 +++ .../03727_tolowcardinality_nullable_cast.sql | 54 +++++++++++++++++++ 4 files changed, 76 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.reference create mode 100644 tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.sql diff --git a/src/Functions/toLowCardinality.cpp b/src/Functions/toLowCardinality.cpp index e3c9fd71f51e..3c64199a5537 100644 --- a/src/Functions/toLowCardinality.cpp +++ b/src/Functions/toLowCardinality.cpp @@ -3,10 +3,17 @@ #include #include #include +#include namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { @@ -40,6 +47,9 @@ class FunctionToLowCardinality: public IFunction if (arg.type->lowCardinality()) return arg.column; + if (!res_type->lowCardinality()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected LowCardinality type as result type for toLowCardinality function, got: {}", res_type->getName()); + auto column = res_type->createColumn(); typeid_cast(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size()); return column; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d3d9f2b8dbcb..ca991937b68a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1128,8 +1128,10 @@ bool applyFunctionChainToColumn( return false; result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); - result_type = removeLowCardinality(func->getResultType()); - result_column = func->execute({{result_column, argument_type, ""}}, result_type, result_column->size(), /* dry_run = */ false); + auto func_result_type = func->getResultType(); + result_column = func->execute({{result_column, argument_type, ""}}, func_result_type, result_column->size(), /* dry_run = */ false); + result_column = result_column->convertToFullColumnIfLowCardinality(); + result_type = removeLowCardinality(func_result_type); // Transforming nullable columns to the nested ones, in case no nulls found if (result_column->isNullable()) diff --git a/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.reference b/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.reference new file mode 100644 index 000000000000..5af65742c057 --- /dev/null +++ b/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.reference @@ -0,0 +1,8 @@ +1 a +4 a +2 b +3 \N +4 a +test +\N +1 diff --git a/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.sql b/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.sql new file mode 100644 index 000000000000..95419ee1071a --- /dev/null +++ b/tests/queries/0_stateless/03727_tolowcardinality_nullable_cast.sql @@ -0,0 +1,54 @@ +-- Test for issue #89412: Bad cast from ColumnNullable to ColumnLowCardinality + +DROP TABLE IF EXISTS test_tolowcardinality_nullable; + +-- Test 1: Original fiddle query from issue #89412 +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Nullable(Int)) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY (toLowCardinality(c0)) SETTINGS allow_nullable_key = 1; +INSERT INTO TABLE t0 (c0) VALUES (0); +DELETE FROM t0 WHERE c0 = 1; +DROP TABLE t0; + +-- Test 2: Using toLowCardinality with Nullable in PARTITION BY +CREATE TABLE test_tolowcardinality_nullable +( + id UInt32, + str Nullable(String) +) +ENGINE = MergeTree() +PARTITION BY toLowCardinality(str) +ORDER BY id +SETTINGS allow_nullable_key = 1; + +INSERT INTO test_tolowcardinality_nullable VALUES (1, 'a'), (2, 'b'), (3, NULL), (4, 'a'); + +-- Query that triggers KeyCondition optimization with toLowCardinality +SELECT * FROM test_tolowcardinality_nullable WHERE toLowCardinality(str) = 'a' ORDER BY id; + +-- Mutation that also uses the partition key +ALTER TABLE test_tolowcardinality_nullable DELETE WHERE id = 1 SETTINGS mutations_sync = 2; + +SELECT * FROM test_tolowcardinality_nullable ORDER BY id; + +DROP TABLE test_tolowcardinality_nullable; + +-- Test 2: Direct toLowCardinality on Nullable column +SELECT toLowCardinality(materialize(toNullable('test'))) AS result; +SELECT toLowCardinality(materialize(CAST(NULL AS Nullable(String)))) AS result; + +-- Test 3: toLowCardinality in WHERE clause with Nullable +DROP TABLE IF EXISTS test_tolowcardinality_where; + +CREATE TABLE test_tolowcardinality_where +( + id UInt32, + val Nullable(String) +) +ENGINE = MergeTree() +ORDER BY id; + +INSERT INTO test_tolowcardinality_where VALUES (1, 'x'), (2, 'y'), (3, NULL); + +SELECT id FROM test_tolowcardinality_where WHERE toLowCardinality(val) = 'x' ORDER BY id; + +DROP TABLE test_tolowcardinality_where; From 64290ce5fbf78576dde448fbe335f97004ec678e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 8 Dec 2025 17:32:30 +0000 Subject: [PATCH 047/113] Backport #90950 to 25.8: CI: Fix tweak calculation in legacy version helper --- tests/ci/git_helper.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/git_helper.py b/tests/ci/git_helper.py index 511f985bc2ab..9505ff5e3e02 100644 --- a/tests/ci/git_helper.py +++ b/tests/ci/git_helper.py @@ -257,7 +257,7 @@ def _update_tags(self, suppress_stderr: bool = False) -> None: self.latest_tag = self.run("git describe --tags --abbrev=0", stderr=stderr) # Format should be: {latest_tag}-{commits_since_tag}-g{sha_short} self.commits_since_latest = int( - self.run(f"git rev-list {self.latest_tag}..HEAD --count") + self.run(f"git rev-list {self.latest_tag}..HEAD --first-parent --count") ) if self.latest_tag.endswith("-new"): # We won't change the behaviour of the the "latest_tag" @@ -268,7 +268,7 @@ def _update_tags(self, suppress_stderr: bool = False) -> None: stderr=stderr, ) self.commits_since_new = int( - self.run(f"git rev-list {self.new_tag}..HEAD --count") + self.run(f"git rev-list {self.new_tag}..HEAD --first-parent --count") ) @staticmethod From c1dbd2926664a6613277351fbab338872032e471 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 9 Dec 2025 03:37:15 +0000 Subject: [PATCH 048/113] Backport #90928 to 25.8: Fix handling global grants with wildcard revokes --- src/Access/AccessRights.cpp | 4 ++-- src/Access/tests/gtest_access_rights_ops.cpp | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 29bf46479cf9..558478f97953 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -239,8 +239,8 @@ namespace { case GLOBAL_LEVEL: return AccessFlags::allFlagsGrantableOnGlobalLevel(); case DATABASE_LEVEL: return AccessFlags::allFlagsGrantableOnDatabaseLevel() | AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); - case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel() | AccessFlags::allSourceFlags(); - case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); + case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel() | AccessFlags::allSourceFlags() | AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); + case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel() | AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel(); } chassert(false); } diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 43006f0f8b26..86b622cdda85 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -547,6 +547,21 @@ TEST(AccessRights, Filter) ASSERT_EQ(res.size(), 0); } +TEST(AccessRights, RevokeWithParameters) +{ + AccessRights root; + root.grantWithGrantOption(AccessType::SELECT); + root.grantWithGrantOption(AccessType::CREATE_USER); + root.revokeWildcard(AccessType::SELECT, "default", "zoo"); + ASSERT_EQ(root.toString(), "GRANT SELECT ON *.* WITH GRANT OPTION, GRANT CREATE USER ON * WITH GRANT OPTION, REVOKE SELECT ON default.zoo*"); + + root = {}; + root.grantWithGrantOption(AccessType::SELECT); + root.grantWithGrantOption(AccessType::CREATE_USER); + root.revokeWildcard(AccessType::SELECT, "default", "foo", "bar"); + ASSERT_EQ(root.toString(), "GRANT SELECT ON *.* WITH GRANT OPTION, GRANT CREATE USER ON * WITH GRANT OPTION, REVOKE SELECT(bar*) ON default.foo"); +} + TEST(AccessRights, ParialRevokeWithGrantOption) { AccessRights root; From ff1b7e469743b6d7218db96526b6efd7330608b1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 9 Dec 2025 13:24:38 +0000 Subject: [PATCH 049/113] Backport #91524 to 25.8: Fix logical error caused by asterisks argument in `remote` function --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 8 +++++ ...table_function_argument_asterisk.reference | 0 ...03749_table_function_argument_asterisk.sql | 36 +++++++++++++++++++ 3 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/03749_table_function_argument_asterisk.reference create mode 100644 tests/queries/0_stateless/03749_table_function_argument_asterisk.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 34df4b8d256d..a89a57f29b46 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -5067,6 +5067,14 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, if (auto * expression_list = table_function_argument->as()) { + if (expression_list->getNodes().empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function {} argument at position {} resolved to an empty expression list (parsed as: {})", + table_function_name, + table_function_argument_index + 1, + table_function_argument->formatASTForErrorMessage()); + for (auto & expression_list_node : expression_list->getNodes()) result_table_function_arguments.push_back(expression_list_node); } diff --git a/tests/queries/0_stateless/03749_table_function_argument_asterisk.reference b/tests/queries/0_stateless/03749_table_function_argument_asterisk.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03749_table_function_argument_asterisk.sql b/tests/queries/0_stateless/03749_table_function_argument_asterisk.sql new file mode 100644 index 000000000000..d438ee1e19c0 --- /dev/null +++ b/tests/queries/0_stateless/03749_table_function_argument_asterisk.sql @@ -0,0 +1,36 @@ +SELECT * +FROM remote(*, '127.{1,2}', view( + SELECT 2 +)); -- { serverError BAD_ARGUMENTS } + +SELECT * +FROM remote(*, view( + SELECT 2 +)); -- { serverError BAD_ARGUMENTS } + +SELECT * +FROM remote(*, '127.{1,2}', view( + SELECT toLowCardinality(2) +)); -- { serverError BAD_ARGUMENTS } + +SELECT * +FROM remote(*, '127.{1,2}', view( + SELECT 1 + FROM numbers(1) + GROUP BY toLowCardinality(2) +)); -- { serverError BAD_ARGUMENTS } + +SELECT DISTINCT '/01650_drop_part_and_deduplication_partitioned_table/blocks/', 60, k1 +FROM remote(*, '127.{1,2}', view(SELECT 1 AS k1, 65535, 2 AS k2, 3 AS v +FROM numbers(2, cityHash64(k1)) +WHERE toLowCardinality(60) GROUP BY GROUPING SETS ((toLowCardinality(2))) +HAVING equals(k1, toNullable(60)))) FINAL; -- { serverError BAD_ARGUMENTS } + +SELECT * FROM numbers(*, 2); -- { serverError BAD_ARGUMENTS } + +SELECT * FROM numbers(2, *); -- { serverError BAD_ARGUMENTS } + +SELECT * FROM numbers_mt(2, *); -- { serverError BAD_ARGUMENTS } + +SELECT * +FROM generateSeries(*, 1, 3); -- { serverError BAD_ARGUMENTS } From 9e7bad98a68f17ea524c36b9439095d066cef6e3 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 9 Dec 2025 20:26:33 +0100 Subject: [PATCH 050/113] Update TreeRewriter.cpp --- src/Interpreters/TreeRewriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index e1eed6fc6f27..0456a5d53a0f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1193,7 +1193,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select /// Collect missed object subcolumns if (!unknown_required_source_columns.empty()) { - for (const NameAndTypePair & pair : source_columns_ordinary) + for (const NameAndTypePair & pair : required_source_columns) { for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();) { From a977f4716c7970a554d1732d255146c977af5d7c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 Dec 2025 12:33:44 +0100 Subject: [PATCH 051/113] Update TreeRewriter.h --- src/Interpreters/TreeRewriter.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index aee585969bcf..28ff6988b076 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -34,6 +34,9 @@ struct TreeRewriterResult /// Same as above but also record alias columns which are expanded. This is for RBAC access check. Names required_source_columns_before_expanding_alias_columns; + /// Set of columns that object columns are not extended. This is for distinguishing JSON and Tuple type. + NamesAndTypesList source_columns_ordinary; + NameSet missed_subcolumns; /// Set of alias columns that are expanded to their alias expressions. We still need the original columns to check access permission. From e1432be85f5d9982b0dd8e9ffeb6562d8347da13 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 10 Dec 2025 12:34:14 +0100 Subject: [PATCH 052/113] Update TreeRewriter.cpp --- src/Interpreters/TreeRewriter.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0456a5d53a0f..8ba17a0e937a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1024,6 +1024,11 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) source_columns.insert(source_columns.end(), columns_from_storage.begin(), columns_from_storage.end()); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + source_columns_ordinary = metadata_snapshot->getColumns().getOrdinary(); + } + else + { + source_columns_ordinary = source_columns; } source_columns_set = removeDuplicateColumns(source_columns); @@ -1193,7 +1198,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select /// Collect missed object subcolumns if (!unknown_required_source_columns.empty()) { - for (const NameAndTypePair & pair : required_source_columns) + for (const NameAndTypePair & pair : source_columns_ordinary) { for (auto it = unknown_required_source_columns.begin(); it != unknown_required_source_columns.end();) { From 3db28c46b5ee78b813f4f15252420689bb45a1e8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 10 Dec 2025 12:22:52 +0000 Subject: [PATCH 053/113] Backport #90490 to 25.8: Fix a bug about `toDateTimeOrNull` with negative argument --- src/Formats/SchemaInferenceUtils.cpp | 6 +- src/IO/ReadHelpers.cpp | 44 ++++++++--- src/IO/ReadHelpers.h | 79 ++++++++++++------- src/IO/parseDateTimeBestEffort.cpp | 26 +++--- .../01186_conversion_to_nullable.reference | 6 +- .../01556_accurate_cast_or_null.reference | 6 +- .../03149_asof_join_ddb_timestamps.reference | 8 +- ...te_time_or_null_negative_arg_bug.reference | 38 +++++++++ ..._to_date_time_or_null_negative_arg_bug.sql | 58 ++++++++++++++ 9 files changed, 210 insertions(+), 61 deletions(-) create mode 100644 tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.reference create mode 100644 tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.sql diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 940b85065542..837947997fb2 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -791,7 +791,7 @@ namespace ReadBufferFromString buf(field); DayNum tmp; - return tryReadDateText(tmp, buf, DateLUT::instance(), /*allowed_delimiters=*/"-/:") && buf.eof(); + return tryReadDateText(tmp, buf, DateLUT::instance(), /*allowed_delimiters=*/"-/:", /*saturate_on_overflow=*/false) && buf.eof(); } DataTypePtr tryInferDateTimeOrDateTime64(std::string_view field, const FormatSettings & settings) @@ -827,7 +827,7 @@ namespace switch (settings.date_time_input_format) { case FormatSettings::DateTimeInputFormat::Basic: - if (tryReadDateTimeText(tmp, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":") && buf.eof()) + if (tryReadDateTimeText(tmp, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":", /*saturate_on_overflow=*/false) && buf.eof()) return std::make_shared(); break; case FormatSettings::DateTimeInputFormat::BestEffort: @@ -846,7 +846,7 @@ namespace switch (settings.date_time_input_format) { case FormatSettings::DateTimeInputFormat::Basic: - if (tryReadDateTime64Text(tmp, 9, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":") && buf.eof()) + if (tryReadDateTime64Text(tmp, 9, buf, DateLUT::instance(), /*allowed_date_delimiters=*/"-/:", /*allowed_time_delimiters=*/":", /*saturate_on_overflow=*/false) && buf.eof()) return std::make_shared(9); break; case FormatSettings::DateTimeInputFormat::BestEffort: diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c3575ca52bc6..410f3b8562ab 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1458,7 +1458,13 @@ template bool readDateTextFallback(LocalDate &, ReadBuffer &, const char * template -ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters, const char * allowed_time_delimiters) +ReturnType readDateTimeTextFallback( + time_t & datetime, + ReadBuffer & buf, + const DateLUTImpl & date_lut, + const char * allowed_date_delimiters, + const char * allowed_time_delimiters, + bool saturate_on_overflow) { static constexpr bool throw_exception = std::is_same_v; @@ -1576,15 +1582,29 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D } else { - auto datetime_maybe = tryToMakeDateTime(date_lut, year, month, day, hour, minute, second); - if (!datetime_maybe) - return false; + if (saturate_on_overflow) + { + /// Use saturating version - makeDateTime saturates out-of-range years + if (unlikely(year == 0)) + datetime = 0; + else + datetime = makeDateTime(date_lut, year, month, day, hour, minute, second); + } + else + { + /// Use non-saturating version - return false for out-of-range values + auto datetime_maybe = tryToMakeDateTime(date_lut, year, month, day, hour, minute, second); + if (!datetime_maybe) + return false; - /// For usual DateTime check if value is within supported range - if (!dt64_mode && (*datetime_maybe < 0 || *datetime_maybe > UINT32_MAX)) - return false; + if constexpr (!dt64_mode) + { + if (*datetime_maybe < 0 || *datetime_maybe > static_cast(UINT32_MAX)) + return false; + } - datetime = *datetime_maybe; + datetime = *datetime_maybe; + } } } else @@ -1620,10 +1640,10 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D return ReturnType(true); } -template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); -template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); -template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); -template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *, bool); +template void readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *, bool); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *, bool); +template bool readDateTimeTextFallback(time_t &, ReadBuffer &, const DateLUTImpl &, const char *, const char *, bool); template ReturnType readTimeTextFallback(time_t & time, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters, const char * allowed_time_delimiters) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 02caaf26736d..fc2864545fa1 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -619,7 +619,7 @@ inline bool tryToConvertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_delimiters = nullptr) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_delimiters = nullptr, bool saturate_on_overflow = true) { static constexpr bool throw_exception = std::is_same_v; @@ -636,12 +636,22 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU if (!readDateTextImpl(local_date, buf, allowed_delimiters)) return false; - auto ret = tryToMakeDayNum(date_lut, local_date.year(), local_date.month(), local_date.day()); - if (!ret) - return false; + if (saturate_on_overflow) + { + /// Use saturating versions - makeDayNum saturates out-of-range years, convertToDayNum saturates to 0 or 0xFFFF + ExtendedDayNum ret = makeDayNum(date_lut, local_date.year(), local_date.month(), local_date.day()); + convertToDayNum(date, ret); + } + else + { + /// Use non-saturating versions - return false for out-of-range values + auto ret = tryToMakeDayNum(date_lut, local_date.year(), local_date.month(), local_date.day()); + if (!ret) + return false; - if (!tryToConvertToDayNum(date, *ret)) - return false; + if (!tryToConvertToDayNum(date, *ret)) + return false; + } return true; } @@ -685,9 +695,9 @@ inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf, const char * all return readDateTextImpl(date, buf, allowed_delimiters); } -inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_delimiters = nullptr) +inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_delimiters = nullptr, bool saturate_on_overflow = true) { - return readDateTextImpl(date, buf, time_zone, allowed_delimiters); + return readDateTextImpl(date, buf, time_zone, allowed_delimiters, saturate_on_overflow); } inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_delimiters = nullptr) @@ -815,7 +825,7 @@ inline T parseFromStringWithoutAssertEOF(std::string_view str) } template -ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr); +ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr, bool saturate_on_overflow = true); template ReturnType readTimeTextFallback(time_t & time, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr); @@ -824,7 +834,7 @@ ReturnType readTimeTextFallback(time_t & time, ReadBuffer & buf, const DateLUTIm * As an exception, also supported parsing of unix timestamp in form of decimal number. */ template -inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) +inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr, bool saturate_on_overflow = true) { static constexpr bool throw_exception = std::is_same_v; @@ -898,15 +908,30 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons } else { - auto datetime_maybe = tryToMakeDateTime(date_lut, year, month, day, hour, minute, second); - if (!datetime_maybe) - return false; - - /// For usual DateTime check if value is within supported range - if (!dt64_mode && (*datetime_maybe < 0 || *datetime_maybe > UINT32_MAX)) - return false; - - datetime = *datetime_maybe; + if (saturate_on_overflow) + { + /// Use saturating version - makeDateTime saturates out-of-range years + if (unlikely(year == 0)) + datetime = 0; + else + datetime = makeDateTime(date_lut, year, month, day, hour, minute, second); + } + else + { + /// Use non-saturating version - return false for out-of-range values + auto datetime_maybe = tryToMakeDateTime(date_lut, year, month, day, hour, minute, second); + if (!datetime_maybe) + return false; + + /// For usual DateTime check if value is within supported range + if constexpr (!dt64_mode) + { + if (*datetime_maybe < 0 || *datetime_maybe > static_cast(UINT32_MAX)) + return false; + } + + datetime = *datetime_maybe; + } } if (dt_long) @@ -919,7 +944,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons /// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN. return readIntTextImpl(datetime, buf); } - return readDateTimeTextFallback(datetime, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); + return readDateTimeTextFallback(datetime, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow); } /** In hhh:mm:ss format, according to specified time zone. @@ -1107,7 +1132,7 @@ inline ReturnType readTimeTextImpl(time_t & time, ReadBuffer & buf, const DateLU } template -inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) +inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut, const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr, bool saturate_on_overflow = true) { static constexpr bool throw_exception = std::is_same_v; @@ -1121,7 +1146,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re { try { - readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); + readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow); } catch (const DB::Exception &) { @@ -1131,7 +1156,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } else { - if (!readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters)) + if (!readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow)) return ReturnType(false); } } @@ -1366,14 +1391,14 @@ inline bool tryReadTimeText(time_t & time, ReadBuffer & buf, const DateLUTImpl & return readTimeTextImpl(time, buf, time_zone, allowed_date_delimiters, allowed_time_delimiters); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr, bool saturate_on_overflow = true) { - return readDateTimeTextImpl(datetime, buf, time_zone, allowed_date_delimiters, allowed_time_delimiters); + return readDateTimeTextImpl(datetime, buf, time_zone, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr, bool saturate_on_overflow = true) { - return readDateTimeTextImpl(datetime64, scale, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters); + return readDateTimeTextImpl(datetime64, scale, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow); } inline bool tryReadTime64Text(Time64 & time64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance(), const char * allowed_date_delimiters = nullptr, const char * allowed_time_delimiters = nullptr) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index f4981482df2f..5b2c7ed5a9c7 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -13,8 +13,8 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int CANNOT_PARSE_DATETIME; +extern const int LOGICAL_ERROR; +extern const int CANNOT_PARSE_DATETIME; } @@ -753,7 +753,7 @@ ReturnType parseDateTimeBestEffortImpl( } }; - if constexpr (std::is_same_v) + if constexpr (!strict || std::is_same_v) { if (has_time_zone_offset) { @@ -764,10 +764,12 @@ ReturnType parseDateTimeBestEffortImpl( { res = local_time_zone.makeDateTime(year, month, day_of_month, hour, minute, second); } + + if constexpr (std::is_same_v) + return true; } else { - if (has_time_zone_offset) { auto res_maybe = utc_time_zone.tryToMakeDateTime(year, month, day_of_month, hour, minute, second); @@ -775,9 +777,11 @@ ReturnType parseDateTimeBestEffortImpl( return false; /// For usual DateTime check if value is within supported range - if (!is_64 && (*res_maybe < 0 || *res_maybe > UINT32_MAX)) - return false; - + if constexpr (!is_64) + { + if (*res_maybe < 0 || *res_maybe > UINT32_MAX) + return false; + } res = *res_maybe; adjust_time_zone(); } @@ -788,9 +792,11 @@ ReturnType parseDateTimeBestEffortImpl( return false; /// For usual DateTime check if value is within supported range - if (!is_64 && (*res_maybe < 0 || *res_maybe > UINT32_MAX)) - return false; - + if constexpr (!is_64) + { + if (*res_maybe < 0 || *res_maybe > UINT32_MAX) + return false; + } res = *res_maybe; } diff --git a/tests/queries/0_stateless/01186_conversion_to_nullable.reference b/tests/queries/0_stateless/01186_conversion_to_nullable.reference index b05ab29a9952..e4c1fd7c40bf 100644 --- a/tests/queries/0_stateless/01186_conversion_to_nullable.reference +++ b/tests/queries/0_stateless/01186_conversion_to_nullable.reference @@ -10,12 +10,12 @@ 256 2020-12-24 \N +1970-01-01 \N -\N -\N +2149-06-06 2020-12-24 01:02:03 \N -\N +1970-01-01 02:00:00 \N 2020-12-24 01:02:03.00 \N diff --git a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference index 6b5e76bcd42a..5187a19cc726 100644 --- a/tests/queries/0_stateless/01556_accurate_cast_or_null.reference +++ b/tests/queries/0_stateless/01556_accurate_cast_or_null.reference @@ -36,13 +36,13 @@ 2023-05-30 14:38:20 1970-01-01 00:00:19 1970-01-01 19:26:40 -\N -\N +1970-01-01 00:00:00 +2106-02-07 06:28:15 \N \N \N 2023-05-30 -\N +2149-06-06 1970-01-20 \N \N diff --git a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference index 9447377eabda..7cfc85d23a5d 100644 --- a/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference +++ b/tests/queries/0_stateless/03149_asof_join_ddb_timestamps.reference @@ -7,6 +7,7 @@ 2023-03-21 19:00:00 3 2023-03-21 20:00:00 3 2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 2023-03-21 13:00:00 0 2023-03-21 14:00:00 1 2023-03-21 15:00:00 2 @@ -16,6 +17,7 @@ 2023-03-21 19:00:00 3 2023-03-21 20:00:00 3 2023-03-21 21:00:00 3 +2106-02-07 06:28:15 9 2023-03-21 12:00:00 \N 2023-03-21 13:00:00 0 2023-03-21 14:00:00 1 @@ -26,7 +28,7 @@ 2023-03-21 19:00:00 3 2023-03-21 20:00:00 3 2023-03-21 21:00:00 3 -\N \N +2106-02-07 06:28:15 9 \N \N 2023-03-21 12:00:00 0 2023-03-21 13:00:00 0 @@ -38,7 +40,7 @@ 2023-03-21 19:00:00 3 2023-03-21 20:00:00 3 2023-03-21 21:00:00 3 -\N 0 +2106-02-07 06:28:15 9 \N 0 2023-03-21 12:00:00 \N 2023-03-21 13:00:00 \N @@ -50,5 +52,5 @@ 2023-03-21 19:00:00 \N 2023-03-21 20:00:00 \N 2023-03-21 21:00:00 \N -\N \N +2106-02-07 06:28:15 \N \N \N diff --git a/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.reference b/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.reference new file mode 100644 index 000000000000..ed23201e10dc --- /dev/null +++ b/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.reference @@ -0,0 +1,38 @@ +toDateOrNull: +1960-01-01 1970-01-01 +1800-01-01 1970-01-01 +3000-01-01 2149-06-06 +toDateTimeOrNull: +1960-01-01 00:00:00 1970-01-01 00:00:00 +1800-01-01 00:00:00 1970-01-01 00:00:00 +3000-01-01 00:00:00 2106-02-07 06:28:15 +toDateTime64OrNull: +1800-01-01 00:00:00 1900-01-01 00:00:00.000 +3000-01-01 00:00:00 2299-12-31 00:00:00.000 +accurateCastOrNull to Date: +1960-01-01 1970-01-01 +1800-01-01 1970-01-01 +3000-01-01 2149-06-06 +accurateCastOrNull to DateTime: +1960-01-01 1970-01-01 00:00:00 +1800-01-01 1970-01-01 00:00:00 +3000-01-01 2106-02-07 06:28:15 +accurateCastOrNull to DateTime (best_effort): +1960-01-01 1970-01-01 00:00:00 +1800-01-01 1970-01-01 00:00:00 +3000-01-01 2106-02-07 06:28:15 +accurateCastOrNull to DateTime (best_effort_us): +1960-01-01 1970-01-01 00:00:00 +1800-01-01 1970-01-01 00:00:00 +3000-01-01 2106-02-07 06:28:15 +accurateCastOrNull to DateTime64: +1800-01-01 1900-01-01 00:00:00.000 +3000-01-01 2299-12-31 00:00:00.000 +accurateCastOrNull to DateTime64 (best_effort): +1960-01-01 1960-01-01 00:00:00.000 +1800-01-01 1900-01-01 00:00:00.000 +3000-01-01 2299-12-31 00:00:00.000 +accurateCastOrNull to DateTime64 (best_effort_us): +1960-01-01 1960-01-01 00:00:00.000 +1800-01-01 1900-01-01 00:00:00.000 +3000-01-01 2299-12-31 00:00:00.000 diff --git a/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.sql b/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.sql new file mode 100644 index 000000000000..8653b99dfadd --- /dev/null +++ b/tests/queries/0_stateless/03724_to_date_time_or_null_negative_arg_bug.sql @@ -0,0 +1,58 @@ +set session_timezone='UTC'; +-- toDateOrNull: pre-epoch and far future dates +select 'toDateOrNull:'; +select '1960-01-01' as input, toDateOrNull('1960-01-01') as result; +select '1800-01-01' as input, toDateOrNull('1800-01-01') as result; +select '3000-01-01' as input, toDateOrNull('3000-01-01') as result; + +-- toDateTimeOrNull: pre-epoch and far future datetimes +select 'toDateTimeOrNull:'; +select '1960-01-01 00:00:00' as input, toDateTimeOrNull('1960-01-01 00:00:00') as result; +select '1800-01-01 00:00:00' as input, toDateTimeOrNull('1800-01-01 00:00:00') as result; +select '3000-01-01 00:00:00' as input, toDateTimeOrNull('3000-01-01 00:00:00') as result; + +-- toDateTime64OrNull: pre-epoch and far future datetimes +select 'toDateTime64OrNull:'; +select '1800-01-01 00:00:00' as input, toDateTime64OrNull('1800-01-01 00:00:00') as result; +select '3000-01-01 00:00:00' as input, toDateTime64OrNull('3000-01-01 00:00:00') as result; + +-- accurateCastOrNull to Date +select 'accurateCastOrNull to Date:'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'Date') as result; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'Date') as result; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'Date') as result; + +-- accurateCastOrNull to DateTime +select 'accurateCastOrNull to DateTime:'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'DateTime') as result; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime') as result; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime') as result; + +-- accurateCastOrNull to DateTime with best_effort mode +select 'accurateCastOrNull to DateTime (best_effort):'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort'; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort'; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort'; + +-- accurateCastOrNull to DateTime with best_effort_us mode +select 'accurateCastOrNull to DateTime (best_effort_us):'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort_us'; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort_us'; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime') as result settings cast_string_to_date_time_mode='best_effort_us'; + +-- accurateCastOrNull to DateTime64 +select 'accurateCastOrNull to DateTime64:'; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime64') as result; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime64') as result; + +-- accurateCastOrNull to DateTime64 with best_effort mode +select 'accurateCastOrNull to DateTime64 (best_effort):'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort'; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort'; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort'; + +-- accurateCastOrNull to DateTime64 with best_effort_us mode +select 'accurateCastOrNull to DateTime64 (best_effort_us):'; +select '1960-01-01' as input, accurateCastOrNull('1960-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort_us'; +select '1800-01-01' as input, accurateCastOrNull('1800-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort_us'; +select '3000-01-01' as input, accurateCastOrNull('3000-01-01', 'DateTime64') as result settings cast_string_to_date_time_mode='best_effort_us'; \ No newline at end of file From b44c8d3394d00e2b7fd58253fee07b11580ebb59 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 10 Dec 2025 13:24:23 +0000 Subject: [PATCH 054/113] Backport #91664 to 25.8: Fix memory usage in join with additional filter --- src/Interpreters/ConcurrentHashJoin.cpp | 12 +- src/Interpreters/HashJoin/AddedColumns.h | 10 -- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- src/Interpreters/HashJoin/HashJoinMethods.h | 23 +-- .../HashJoin/HashJoinMethodsImpl.h | 166 +++++++++++------- src/Interpreters/HashJoin/HashJoinResult.cpp | 13 +- src/Interpreters/HashJoin/HashJoinResult.h | 4 + src/Interpreters/IJoin.cpp | 2 +- src/Interpreters/IJoin.h | 3 + src/Interpreters/MergeJoin.cpp | 4 +- .../Transforms/JoiningTransform.cpp | 11 +- ..._complex_condition_result_memory.reference | 5 + ..._on_complex_condition_result_memory.sql.j2 | 50 ++++++ 13 files changed, 206 insertions(+), 99 deletions(-) create mode 100644 tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.reference create mode 100644 tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.sql.j2 diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 55f3ac445731..2f5ea35ed60d 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -320,17 +320,23 @@ class ConcurrentHashJoinResult : public IJoinResult if (!current_result) { if (next_block >= dispatched_blocks.size()) - return {Block(), true}; + return {Block(), nullptr, true}; current_result = hash_joins[next_block]->data->joinScatteredBlock(std::move(dispatched_blocks[next_block])); - ++next_block; } auto data = current_result->next(); if (data.is_last) + { + if (data.next_block) + dispatched_blocks[next_block] = std::move(*data.next_block); + else + ++next_block; current_result.reset(); + } + bool is_last = next_block >= dispatched_blocks.size() && data.is_last; - return {std::move(data.block), is_last}; + return {std::move(data.block), nullptr, is_last}; } }; diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index eed9156420fa..5155f7e4d007 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -265,14 +265,4 @@ class AddedColumns } }; -/// Adapter class to pass into addFoundRowAll -/// In joinRightColumnsWithAdditionalFilter we don't want to add rows directly into AddedColumns, -/// because they need to be filtered by additional_filter_expression. -class PreSelectedRows : public std::vector -{ -public: - void appendFromBlock(const RowRef * row_ref, bool /* has_default */) { this->emplace_back(row_ref); } - static constexpr bool isLazy() { return false; } -}; - } diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index ee188db0e499..c2671c8e9d2b 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1036,7 +1036,7 @@ IJoinResult::JoinResultBlock CrossJoinResult::next() bool is_last = left_row >= rows_total; res = res.cloneWithColumns(std::move(dst_columns)); - return {res, is_last}; + return {res, nullptr, is_last}; } JoinResultPtr HashJoin::joinBlockImplCross(Block block) const diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 63709239dc36..4241c4e129ee 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -114,7 +114,7 @@ class HashJoinMethods bool & all_values_unique); template - static void switchJoinRightColumns( + static size_t switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, const ScatteredBlock::Selector & selector, @@ -122,7 +122,7 @@ class HashJoinMethods JoinStuff::JoinUsedFlags & used_flags); template - static void joinRightColumnsSwitchNullability( + static size_t joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -130,7 +130,7 @@ class HashJoinMethods JoinStuff::JoinUsedFlags & used_flags); template - static void joinRightColumnsSwitchMultipleDisjuncts( + static size_t joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -147,7 +147,7 @@ class HashJoinMethods JoinCommon::JoinMask::Kind join_mask_kind, typename AddedColumns, typename Selector> - static void joinRightColumns( + static size_t joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -161,7 +161,7 @@ class HashJoinMethods bool check_null_map, typename AddedColumns, typename Selector> - static void joinRightColumnsSwitchJoinMaskKind( + static size_t joinRightColumnsSwitchJoinMaskKind( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -176,7 +176,7 @@ class HashJoinMethods JoinCommon::JoinMask::Kind join_mask_kind, typename AddedColumns, typename Selector> - static void joinRightColumns( + static size_t joinRightColumns( KeyGetter & key_getter, const Map * map, AddedColumns & added_columns, @@ -184,23 +184,16 @@ class HashJoinMethods const Selector & selector); template - static void joinRightColumnsSwitchJoinMaskKind( + static size_t joinRightColumnsSwitchJoinMaskKind( KeyGetter & key_getter, const Map * map, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags, const Selector & selector); - template - static ColumnPtr buildAdditionalFilter( - const Selector & selector, - const std::vector & selected_rows, - const std::vector & row_replicate_offset, - AddedColumns & added_columns); - /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template - static void joinRightColumnsWithAddtitionalFilter( + static size_t joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, diff --git a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h index 34cf52055382..ee4e90f8da04 100644 --- a/src/Interpreters/HashJoin/HashJoinMethodsImpl.h +++ b/src/Interpreters/HashJoin/HashJoinMethodsImpl.h @@ -109,11 +109,20 @@ JoinResultPtr HashJoinMethods::joinBlockImpl( else added_columns.reserve(join_features.need_replication); - switchJoinRightColumns(maps_, added_columns, block.getSelector(), join.data->type, *join.used_flags); + size_t processed_rows = switchJoinRightColumns(maps_, added_columns, block.getSelector(), join.data->type, *join.used_flags); /// Do not hold memory for join_on_keys anymore added_columns.join_on_keys.clear(); - return std::make_unique( + std::optional next_scattered_block; + if (0 < processed_rows && processed_rows < block.rows()) + { + auto [raw_block, raw_selector] = std::move(block).detachData(); + auto split_selector = raw_selector.split(processed_rows); + block = ScatteredBlock(raw_block, std::move(split_selector.first)); + next_scattered_block = ScatteredBlock(std::move(raw_block), std::move(split_selector.second)); + } + + auto join_result = std::make_unique( std::move(added_columns.lazy_output), std::move(added_columns.columns), std::move(added_columns.offsets_to_replicate), @@ -130,6 +139,10 @@ JoinResultPtr HashJoinMethods::joinBlockImpl( join_features.need_filter, is_join_get }); + + if (next_scattered_block) + join_result->setNextBlock(std::move(next_scattered_block.value())); + return join_result; } template @@ -213,7 +226,7 @@ void HashJoinMethods::insertFromBlockImplTypeCas template template -void HashJoinMethods::switchJoinRightColumns( +size_t HashJoinMethods::switchJoinRightColumns( const std::vector & mapv, AddedColumns & added_columns, const ScatteredBlock::Selector & selector, @@ -263,7 +276,7 @@ void HashJoinMethods::switchJoinRightColumns( template template -void HashJoinMethods::joinRightColumnsSwitchNullability( +size_t HashJoinMethods::joinRightColumnsSwitchNullability( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -271,16 +284,16 @@ void HashJoinMethods::joinRightColumnsSwitchNull JoinStuff::JoinUsedFlags & used_flags) { if (added_columns.need_filter) - joinRightColumnsSwitchMultipleDisjuncts( + return joinRightColumnsSwitchMultipleDisjuncts( std::forward>(key_getter_vector), mapv, added_columns, selector, used_flags); else - joinRightColumnsSwitchMultipleDisjuncts( + return joinRightColumnsSwitchMultipleDisjuncts( std::forward>(key_getter_vector), mapv, added_columns, selector, used_flags); } template template -void HashJoinMethods::joinRightColumnsSwitchMultipleDisjuncts( +size_t HashJoinMethods::joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -293,7 +306,7 @@ void HashJoinMethods::joinRightColumnsSwitchMult if (added_columns.additional_filter_expression) { const bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1; - joinRightColumnsWithAddtitionalFilter( + return joinRightColumnsWithAddtitionalFilter( std::forward>(key_getter_vector), mapv, added_columns, @@ -301,8 +314,6 @@ void HashJoinMethods::joinRightColumnsSwitchMult selector, need_filter, mark_per_row_used); - - return; } } @@ -314,20 +325,20 @@ void HashJoinMethods::joinRightColumnsSwitchMult if (mapv.size() > 1 || added_columns.join_on_keys.empty()) { if (std::ranges::any_of(added_columns.join_on_keys, [](const auto & elem) { return elem.null_map; })) - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( std::move(key_getter_vector), mapv, added_columns, used_flags, selector.getRange()); else - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( std::move(key_getter_vector), mapv, added_columns, used_flags, selector.getRange()); } else { chassert(key_getter_vector.size() == 1); if (added_columns.join_on_keys.at(0).null_map) - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( key_getter_vector.at(0), mapv.at(0), added_columns, used_flags, selector.getRange()); else - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( key_getter_vector.at(0), mapv.at(0), added_columns, used_flags, selector.getRange()); } } @@ -336,20 +347,20 @@ void HashJoinMethods::joinRightColumnsSwitchMult if (mapv.size() > 1 || added_columns.join_on_keys.empty()) { if (std::ranges::any_of(added_columns.join_on_keys, [](const auto & elem) { return elem.null_map; })) - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( std::move(key_getter_vector), mapv, added_columns, used_flags, selector.getIndexes()); else - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( std::move(key_getter_vector), mapv, added_columns, used_flags, selector.getIndexes()); } else { chassert(key_getter_vector.size() == 1); if (added_columns.join_on_keys.at(0).null_map) - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( key_getter_vector.at(0), mapv.at(0), added_columns, used_flags, selector.getIndexes()); else - joinRightColumnsSwitchJoinMaskKind( + return joinRightColumnsSwitchJoinMaskKind( key_getter_vector.at(0), mapv.at(0), added_columns, used_flags, selector.getIndexes()); } } @@ -459,7 +470,7 @@ template < JoinCommon::JoinMask::Kind join_mask_kind, typename AddedColumns, typename Selector> -void HashJoinMethods::joinRightColumns( +size_t HashJoinMethods::joinRightColumns( KeyGetter & key_getter, const Map * map, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags, const Selector & selector) { static constexpr bool flag_per_row = false; // Always false in single map case @@ -532,11 +543,12 @@ void HashJoinMethods::joinRightColumns( } added_columns.applyLazyDefaults(); + return 0; } template template -void HashJoinMethods::joinRightColumnsSwitchJoinMaskKind( +size_t HashJoinMethods::joinRightColumnsSwitchJoinMaskKind( KeyGetter & key_getter, const Map * map, AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags, const Selector & selector) { switch (added_columns.join_on_keys.at(0).join_mask_column.getKind()) @@ -564,7 +576,7 @@ template < JoinCommon::JoinMask::Kind join_mask_kind, typename AddedColumns, typename Selector> -void HashJoinMethods::joinRightColumns( +size_t HashJoinMethods::joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -587,10 +599,16 @@ void HashJoinMethods::joinRightColumns( Arena pool; if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = IColumn::Offsets(rows); + { + added_columns.offsets_to_replicate.clear(); + added_columns.offsets_to_replicate.reserve(rows); + } + + size_t max_joined_rows = added_columns.max_joined_block_rows > 0 ? added_columns.max_joined_block_rows : std::numeric_limits::max(); IColumn::Offset current_offset = 0; - for (size_t i = 0; i < rows; ++i) + size_t i = 0; + for (; i < rows && current_offset < max_joined_rows; ++i) { size_t ind = 0; if constexpr (std::is_same_v, ScatteredBlock::Indexes>) @@ -640,12 +658,11 @@ void HashJoinMethods::joinRightColumns( } if constexpr (join_features.need_replication) - { - added_columns.offsets_to_replicate[i] = current_offset; - } + added_columns.offsets_to_replicate.push_back(current_offset); } added_columns.applyLazyDefaults(); + return i; } template @@ -656,7 +673,7 @@ template < bool check_null_map, typename AddedColumns, typename Selector> -void HashJoinMethods::joinRightColumnsSwitchJoinMaskKind( +size_t HashJoinMethods::joinRightColumnsSwitchJoinMaskKind( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -666,28 +683,27 @@ void HashJoinMethods::joinRightColumnsSwitchJoin using Kind = JoinCommon::JoinMask::Kind; if (std::ranges::all_of(added_columns.join_on_keys, [](const auto & elem) { return elem.join_mask_column.getKind() == Kind::AllTrue; })) { - joinRightColumns( + return joinRightColumns( std::move(key_getter_vector), mapv, added_columns, used_flags, selector); } else if (std::ranges::all_of(added_columns.join_on_keys, [](const auto & elem) { return elem.join_mask_column.getKind() == Kind::AllFalse; })) { - joinRightColumns( + return joinRightColumns( std::move(key_getter_vector), mapv, added_columns, used_flags, selector); } else { - joinRightColumns( + return joinRightColumns( std::move(key_getter_vector), mapv, added_columns, used_flags, selector); } } -template template -ColumnPtr HashJoinMethods::buildAdditionalFilter( +static ColumnPtr buildAdditionalFilter( const Selector & selector, - const std::vector & selected_rows, - const std::vector & row_replicate_offset, - AddedColumns & added_columns) + const PODArray & selected_rows, + const IColumn::Offsets & row_replicate_offset, + const AddedColumns & added_columns) { ColumnPtr result_column; do @@ -723,7 +739,6 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter { if (rhs_pos_it != added_columns.additional_filter_required_rhs_pos.end() && pos == rhs_pos_it->first) { - const auto & req_col = *req_cols_it; required_columns.emplace_back(nullptr, req_col.type, req_col.name); @@ -749,14 +764,13 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter col_name); auto new_col = src_col->column->cloneEmpty(); - size_t prev_left_offset = 0; for (size_t i = 0; i < row_replicate_offset.size(); ++i) { - const size_t & left_offset = row_replicate_offset[i]; - size_t rows = left_offset - prev_left_offset; + size_t rows = row_replicate_offset[i] - row_replicate_offset[i - 1]; if (rows) + { new_col->insertManyFrom(*src_col->column, selector[i], rows); - prev_left_offset = left_offset; + } } required_columns.push_back({std::move(new_col), src_col->type, col_name}); } @@ -801,7 +815,7 @@ ColumnPtr HashJoinMethods::buildAdditionalFilter template template -void HashJoinMethods::joinRightColumnsWithAddtitionalFilter( +size_t HashJoinMethods::joinRightColumnsWithAddtitionalFilter( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, @@ -811,7 +825,7 @@ void HashJoinMethods::joinRightColumnsWithAddtit bool flag_per_row [[maybe_unused]]) { constexpr JoinFeatures join_features; - const size_t left_block_rows = selector.size(); + size_t left_block_rows = selector.size(); if (need_filter) { added_columns.filter = IColumn::Filter(left_block_rows, 0); @@ -823,21 +837,38 @@ void HashJoinMethods::joinRightColumnsWithAddtit if constexpr (join_features.need_replication) added_columns.offsets_to_replicate = IColumn::Offsets(left_block_rows); - std::vector row_replicate_offset; - row_replicate_offset.reserve(left_block_rows); - using FindResult = typename KeyGetter::FindResult; - PreSelectedRows selected_rows; + + /// Adapter class to pass into addFoundRowAll + /// We don't want to add rows directly into AddedColumns, because they need to be filtered by additional_filter_expression. + class PreSelectedRows + { + public: + explicit PreSelectedRows(PODArray & container_) : container(container_) {} + void appendFromBlock(const RowRef * row_ref, bool /* has_default */) { container.push_back(row_ref); } + static constexpr bool isLazy() { return false; } + + PODArray & container; + }; + + PODArray selected_rows; selected_rows.reserve(left_block_rows); std::vector find_results; find_results.reserve(left_block_rows); IColumn::Offset total_added_rows = 0; - IColumn::Offset current_added_rows = 0; + + IColumn::Offsets row_replicate_offset; + row_replicate_offset.reserve(left_block_rows); + + size_t max_joined_rows = added_columns.max_joined_block_rows; + if (max_joined_rows == 0) + max_joined_rows = std::numeric_limits::max(); { pool = std::make_unique(); - current_added_rows = 0; - for (size_t ind : selector) + IColumn::Offset current_added_rows = 0; + + for (auto ind : selector) { KnownRowsHolder all_flag_known_rows; KnownRowsHolder single_flag_know_rows; @@ -857,31 +888,39 @@ void HashJoinMethods::joinRightColumnsWithAddtit find_results.push_back(find_result); /// We don't add missing in addFoundRowAll here. we will add it after filter is applied. /// it's different from `joinRightColumns`. + PreSelectedRows selected_rows_view{selected_rows}; if (flag_per_row) - addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); + addFoundRowAll(mapped, selected_rows_view, current_added_rows, all_flag_known_rows, nullptr); else - addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); + addFoundRowAll(mapped, selected_rows_view, current_added_rows, single_flag_know_rows, nullptr); } + } row_replicate_offset.push_back(current_added_rows); + + + if (current_added_rows >= max_joined_rows) + break; } - } - if (selected_rows.size() != current_added_rows) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}", - selected_rows.size(), - current_added_rows, - row_replicate_offset.size()); + if (selected_rows.size() != current_added_rows) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}", + selected_rows.size(), + current_added_rows, + row_replicate_offset.size()); + } - auto filter_col = buildAdditionalFilter(selector, selected_rows, row_replicate_offset, added_columns); + left_block_rows = row_replicate_offset.size(); { + auto filter_col = buildAdditionalFilter(selector, selected_rows, row_replicate_offset, added_columns); + const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); size_t prev_replicated_row = 0; - auto selected_right_row_it = selected_rows.begin(); + auto * selected_right_row_it = selected_rows.begin(); size_t find_result_index = 0; for (size_t i = 0, n = row_replicate_offset.size(); i < n; ++i) { @@ -1013,10 +1052,11 @@ void HashJoinMethods::joinRightColumnsWithAddtit if constexpr (join_features.need_replication) { - added_columns.offsets_to_replicate.resize_assume_reserved(left_block_rows); - added_columns.filter.resize_assume_reserved(left_block_rows); + added_columns.offsets_to_replicate.resize(left_block_rows); + added_columns.filter.resize(left_block_rows); } added_columns.applyLazyDefaults(); + return left_block_rows; } } diff --git a/src/Interpreters/HashJoin/HashJoinResult.cpp b/src/Interpreters/HashJoin/HashJoinResult.cpp index 4c137daa4ac5..4b855d258def 100644 --- a/src/Interpreters/HashJoin/HashJoinResult.cpp +++ b/src/Interpreters/HashJoin/HashJoinResult.cpp @@ -246,10 +246,17 @@ static size_t getAvgBytesPerRow(const Block & block) return block.allocatedBytes() / std::max(1, block.rows()); } +void HashJoinResult::setNextBlock(ScatteredBlock && block) +{ + next_scattered_block.emplace(std::move(block)); +} + IJoinResult::JoinResultBlock HashJoinResult::next() { + ScatteredBlock * next_block_ptr = next_scattered_block ? &next_scattered_block.value() : nullptr; + if (!scattered_block) - return {}; + return {Block(), next_block_ptr, true}; size_t avg_bytes_per_row = properties.avg_joined_bytes_per_row + getAvgBytesPerRow(scattered_block->getSourceBlock()); auto num_lhs_rows = numLeftRowsForNextBlock(next_row, offsets, properties.max_joined_block_rows, properties.max_joined_block_bytes, avg_bytes_per_row); @@ -268,7 +275,7 @@ IJoinResult::JoinResultBlock HashJoinResult::next() std::span{matched_rows}); scattered_block.reset(); - return {std::move(block), true}; + return {std::move(block), next_block_ptr, true}; } const size_t prev_offset = next_row ? offsets[next_row - 1] : 0; @@ -388,7 +395,7 @@ IJoinResult::JoinResultBlock HashJoinResult::next() if (is_last) scattered_block.reset(); - return {std::move(block), is_last}; + return {std::move(block), next_block_ptr, is_last}; } } diff --git a/src/Interpreters/HashJoin/HashJoinResult.h b/src/Interpreters/HashJoin/HashJoinResult.h index 6206d97e4d4c..82c60a87360c 100644 --- a/src/Interpreters/HashJoin/HashJoinResult.h +++ b/src/Interpreters/HashJoin/HashJoinResult.h @@ -35,11 +35,15 @@ class HashJoinResult : public IJoinResult JoinResultBlock next() override; + void setNextBlock(ScatteredBlock && block); + private: const LazyOutput lazy_output; const Properties properties; std::optional scattered_block; + /// Next unprocessed block + std::optional next_scattered_block; MutableColumns columns; const IColumn::Offsets offsets; diff --git a/src/Interpreters/IJoin.cpp b/src/Interpreters/IJoin.cpp index 4fe2b2a7186c..7299f6c7307c 100644 --- a/src/Interpreters/IJoin.cpp +++ b/src/Interpreters/IJoin.cpp @@ -10,7 +10,7 @@ class JoinResultFromBlock : public IJoinResult JoinResultBlock next() override { - return {std::move(block), true}; + return {std::move(block), nullptr, true}; } private: diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 5df10c3463a7..05d5cb6c6d41 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -55,6 +55,9 @@ class IJoinResult struct JoinResultBlock { Block block; + /// Pointer to next block to process, if any. + /// Should be used once we got last result and is_last is true. + ScatteredBlock * next_block = nullptr; bool is_last = true; }; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 389701333a2a..d484dc8a6561 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -722,12 +722,12 @@ class MergeJoinResult : public IJoinResult if (!not_processed) { merge_join.joinBlock(block, not_processed); - return {std::move(block), !not_processed.has_value()}; + return {std::move(block), nullptr, !not_processed.has_value()}; } block = not_processed->block; merge_join.joinBlock(block, not_processed); - return {std::move(block), !not_processed.has_value()}; + return {std::move(block), nullptr, !not_processed.has_value()}; } }; diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index c667ce6c4cf2..f4080aa9d691 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -134,7 +134,7 @@ void JoiningTransform::work() { chassert(!output_chunk.has_value()); transform(input_chunk); - has_input = join_result != nullptr; + has_input = input_chunk.hasRows() || join_result != nullptr; } else { @@ -170,6 +170,8 @@ void JoiningTransform::work() } } +/// transform should consume the input chunk and set the output chunk +/// if not all data is consumed it may be set to the chunk and transform will be called again void JoiningTransform::transform(Chunk & chunk) { if (!initialized) @@ -220,6 +222,13 @@ Block JoiningTransform::readExecute(Chunk & chunk) } auto data = join_result->next(); + if (data.is_last && data.next_block) + { + data.next_block->filterBySelector(); + auto next_block = std::move(*data.next_block).getSourceBlock(); + chunk.setColumns(next_block.getColumns(), next_block.rows()); + } + if (data.is_last) join_result.reset(); diff --git a/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.reference b/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.reference new file mode 100644 index 000000000000..a3f15f5d7699 --- /dev/null +++ b/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.reference @@ -0,0 +1,5 @@ +18750570 +25000810 +25005842 +25000810 +25005842 diff --git a/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.sql.j2 b/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.sql.j2 new file mode 100644 index 000000000000..55d5c30e5742 --- /dev/null +++ b/tests/queries/0_stateless/03748_join_on_complex_condition_result_memory.sql.j2 @@ -0,0 +1,50 @@ +-- Tags: long + +SET query_plan_join_swap_table = 0; +SET enable_analyzer = 1; + +SET max_memory_usage = '50M'; + +WITH t1 AS ( + SELECT number % 2 AS id, sipHash64(number, 1) % 2 as attr FROM numbers(5_000) +), +t2 AS ( + SELECT number % 2 AS id, sipHash64(number, 2) % 2 as attr FROM numbers(5_000) +) +SELECT sum(ignore(*) + 1) +FROM t1 +LEFT JOIN t2 +ON t1.id = t2.id OR (t1.attr = t2.attr) +; + +{% for join_algorithm in ['hash', 'parallel_hash'] -%} + +WITH t1 AS ( + SELECT number % 2 AS id, sipHash64(number, 1) % 2 as attr FROM numbers(10_000) +), +t2 AS ( + SELECT number % 2 AS id, sipHash64(number, 2) % 2 as attr FROM numbers(10_000) +) +SELECT sum(ignore(*) + 1) +FROM t1 +LEFT JOIN t2 +ON t1.id = t2.id AND (t1.attr != t2.attr) +SETTINGS join_algorithm = '{{ join_algorithm }}'; + +-- more rows to use two-level map +WITH t1 AS ( + SELECT number % 2 AS id, sipHash64(number, 1) % 2 as attr FROM numbers(10_000) +), +t2 AS ( + SELECT number AS id, sipHash64(number, 2) % 2 as attr FROM numbers(10_000) + UNION ALL + SELECT number % 2 AS id, sipHash64(number, 2) % 2 as attr FROM numbers(10_000) +) +SELECT sum(ignore(*) + 1) +FROM t1 +LEFT JOIN t2 +ON t1.id = t2.id AND (t1.attr != t2.attr) +SETTINGS join_algorithm = '{{ join_algorithm }}', + max_threads = 8; -- with too many hash tables query consumes >50M memory + +{% endfor -%} From 6299d7e69c4d434f7447b8deaf7ee4a9344a86f1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 10 Dec 2025 15:15:42 +0000 Subject: [PATCH 055/113] Backport #91750 to 25.8: Fix `too many open files` errors in integration tests --- tests/integration/helpers/client.py | 45 +++++++++++++++-------------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index e6923d13b9df..d4245a1934f0 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -181,9 +181,9 @@ def __init__( self, command, stdin=None, timeout=None, ignore_error=False, parse=False, stdout_file_path=None, stderr_file_path=None, env = {} ): # Write data to tmp file to avoid PIPEs and execution blocking - stdin_file = tempfile.TemporaryFile(mode="w+") - stdin_file.write(stdin) - stdin_file.seek(0) + self.stdin_file = tempfile.TemporaryFile(mode="w+") + self.stdin_file.write(stdin) + self.stdin_file.seek(0) self.stdout_file = tempfile.TemporaryFile() if stdout_file_path is None else stdout_file_path self.stderr_file = tempfile.TemporaryFile() if stderr_file_path is None else stderr_file_path self.ignore_error = ignore_error @@ -196,7 +196,7 @@ def __init__( env["TSAN_OPTIONS"] = "use_sigaltstack=0 verbosity=0" self.process = sp.Popen( command, - stdin=stdin_file, + stdin=self.stdin_file, stdout=self.stdout_file, stderr=self.stderr_file, env=env, @@ -225,13 +225,24 @@ def remove_trash_from_stderr(self, stderr): ] return "\n".join(lines) - def get_answer(self): - self.process.wait(timeout=DEFAULT_QUERY_TIMEOUT) - self.stdout_file.seek(0) - self.stderr_file.seek(0) + def wait_and_read_output(self): + try: + self.process.wait(timeout=DEFAULT_QUERY_TIMEOUT) + self.stdout_file.seek(0) + self.stderr_file.seek(0) + + stdout = self.stdout_file.read().decode("utf-8", errors="replace") + stderr = self.stderr_file.read().decode("utf-8", errors="replace") - stdout = self.stdout_file.read().decode("utf-8", errors="replace") - stderr = self.stderr_file.read().decode("utf-8", errors="replace") + return stdout, stderr + + finally: + self.stdin_file.close() + self.stdout_file.close() + self.stderr_file.close() + + def get_answer(self): + stdout, stderr = self.wait_and_read_output() if ( self.timer is not None @@ -264,12 +275,7 @@ def get_answer(self): return stdout def get_error(self): - self.process.wait(timeout=DEFAULT_QUERY_TIMEOUT) - self.stdout_file.seek(0) - self.stderr_file.seek(0) - - stdout = self.stdout_file.read().decode("utf-8", errors="replace") - stderr = self.stderr_file.read().decode("utf-8", errors="replace") + stdout, stderr = self.wait_and_read_output() if ( self.timer is not None @@ -288,12 +294,7 @@ def get_error(self): return stderr def get_answer_and_error(self): - self.process.wait(timeout=DEFAULT_QUERY_TIMEOUT) - self.stdout_file.seek(0) - self.stderr_file.seek(0) - - stdout = self.stdout_file.read().decode("utf-8", errors="replace") - stderr = self.stderr_file.read().decode("utf-8", errors="replace") + stdout, stderr = self.wait_and_read_output() if ( self.timer is not None From ba389aee60b25a49a13196ee5ca4262f2190cc9f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 10 Dec 2025 19:12:58 +0000 Subject: [PATCH 056/113] Backport #91827 to 25.8: Fix LOGICAL_ERROR in RecursiveCTEChunkGenerator --- src/Processors/Sources/RecursiveCTESource.cpp | 8 ++++--- .../03755_nested_recursive_cte.reference | 0 .../03755_nested_recursive_cte.sql | 21 +++++++++++++++++++ 3 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03755_nested_recursive_cte.reference create mode 100644 tests/queries/0_stateless/03755_nested_recursive_cte.sql diff --git a/src/Processors/Sources/RecursiveCTESource.cpp b/src/Processors/Sources/RecursiveCTESource.cpp index 4a160465626e..62edbdc6645f 100644 --- a/src/Processors/Sources/RecursiveCTESource.cpp +++ b/src/Processors/Sources/RecursiveCTESource.cpp @@ -38,7 +38,7 @@ namespace ErrorCodes namespace { -std::vector collectTableNodesWithStorage(const StoragePtr & storage, IQueryTreeNode * root) +std::vector collectTableNodesWithTemporaryTableName(const std::string & temporary_table_name, IQueryTreeNode * root) { std::vector result; @@ -51,7 +51,7 @@ std::vector collectTableNodesWithStorage(const StoragePtr & storage nodes_to_process.pop_back(); auto * table_node = subtree_node->as(); - if (table_node && table_node->getStorageID() == storage->getStorageID()) + if (table_node && table_node->getTemporaryTableName() == temporary_table_name) result.push_back(table_node); for (auto & child : subtree_node->getChildren()) @@ -77,7 +77,9 @@ class RecursiveCTEChunkGenerator chassert(recursive_cte_union_node_typed.hasRecursiveCTETable()); auto & recursive_cte_table = recursive_cte_union_node_typed.getRecursiveCTETable(); - recursive_table_nodes = collectTableNodesWithStorage(recursive_cte_table->storage, recursive_cte_union_node.get()); + + const auto & cte_name = recursive_cte_union_node_typed.getCTEName(); + recursive_table_nodes = collectTableNodesWithTemporaryTableName(cte_name, recursive_cte_union_node.get()); if (recursive_table_nodes.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION query {} is not recursive", recursive_cte_union_node->formatASTForErrorMessage()); diff --git a/tests/queries/0_stateless/03755_nested_recursive_cte.reference b/tests/queries/0_stateless/03755_nested_recursive_cte.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03755_nested_recursive_cte.sql b/tests/queries/0_stateless/03755_nested_recursive_cte.sql new file mode 100644 index 000000000000..d7c09e50cc8c --- /dev/null +++ b/tests/queries/0_stateless/03755_nested_recursive_cte.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (x Int32) ENGINE = Memory; +INSERT INTO t0 VALUES (1); + +-- The original problematic query pattern - inner CTE references outer CTE +-- Using count() to get deterministic output regardless of how many rows are produced before hitting the limit +SET max_recursive_cte_evaluation_depth = 5; +SET enable_analyzer = 1; + +SELECT count() > 0 FROM ( + WITH RECURSIVE q AS ( + SELECT 1 FROM t0 UNION ALL + (WITH RECURSIVE x AS + (SELECT 1 FROM t0 UNION ALL + (SELECT 1 FROM q WHERE FALSE UNION ALL + SELECT 1 FROM x WHERE FALSE)) + SELECT 1 FROM x)) + SELECT 1 FROM q +); -- { serverError TOO_DEEP_RECURSION } + +DROP TABLE t0; From 81cfe7ce94b581cab548d70095c14c4544aede70 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 11 Dec 2025 10:14:17 +0000 Subject: [PATCH 057/113] Backport #91923 to 25.8: Add a test for optimize_skip_unused_shards with type cast --- ...757_optimize_skip_unused_shards_with_type_cast.reference | 2 ++ .../03757_optimize_skip_unused_shards_with_type_cast.sql | 6 ++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.reference create mode 100644 tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql diff --git a/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.reference b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.reference new file mode 100644 index 000000000000..422c2b7ab3b3 --- /dev/null +++ b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.reference @@ -0,0 +1,2 @@ +a +b diff --git a/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql new file mode 100644 index 000000000000..eb2b6758069f --- /dev/null +++ b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql @@ -0,0 +1,6 @@ +drop table if exists t1; + +create table t1 (Col LowCardinality(String)) engine = MergeTree; +insert into t1 values ('a'), ('b'), ('c'); + +select * from remote('127.{1,2}', currentDatabase(), t1, multiIf(Col = 'a', 0, Col = 'b', 1, -1)) where Col in ('a', 'b') order by all settings optimize_skip_unused_shards=1; From 232f3d1ddb0ea8905f96f97c42a55cfe9a416ccd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 11 Dec 2025 13:01:14 +0100 Subject: [PATCH 058/113] Fix --- src/Interpreters/evaluateConstantExpression.cpp | 2 +- .../03757_optimize_skip_unused_shards_with_type_cast.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 792f7f29038f..230599735c4e 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -612,7 +612,7 @@ namespace if (!type->equals(*node->result_type)) { - cast_col = tryCastColumn(column, value->result_type, node->result_type); + cast_col = tryCastColumn(column, type, node->result_type); if (!cast_col) return {}; const auto & col_nullable = assert_cast(*cast_col); diff --git a/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql index eb2b6758069f..8b733194c765 100644 --- a/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql +++ b/tests/queries/0_stateless/03757_optimize_skip_unused_shards_with_type_cast.sql @@ -1,6 +1,6 @@ drop table if exists t1; -create table t1 (Col LowCardinality(String)) engine = MergeTree; +create table t1 (Col LowCardinality(String)) engine = MergeTree order by (); insert into t1 values ('a'), ('b'), ('c'); select * from remote('127.{1,2}', currentDatabase(), t1, multiIf(Col = 'a', 0, Col = 'b', 1, -1)) where Col in ('a', 'b') order by all settings optimize_skip_unused_shards=1; From 553fed070cd8136ad94a6360404fa14961ca248a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 11 Dec 2025 14:14:07 +0000 Subject: [PATCH 059/113] =?UTF-8?q?Backport=20#91957=20to=2025.8:=20Backpo?= =?UTF-8?q?rt=20fix=2003644=5Fobject=5Fstorage=5Fcorrelated=5Fsubqueries?= =?UTF-8?q?=20to=20release=20br=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../0_stateless/03644_object_storage_correlated_subqueries.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03644_object_storage_correlated_subqueries.sql b/tests/queries/0_stateless/03644_object_storage_correlated_subqueries.sql index 6ac7e423d185..4492d1e2195f 100644 --- a/tests/queries/0_stateless/03644_object_storage_correlated_subqueries.sql +++ b/tests/queries/0_stateless/03644_object_storage_correlated_subqueries.sql @@ -12,5 +12,5 @@ WHERE n1.c1 > ( SELECT AVG(n2.c1) FROM s3('http://localhost:11111/test/test-data-03644_object_storage.csv', 'test', 'testtest') AS n2 WHERE n2.c1 < n1.c1 -) +) ORDER BY n1.c1 SETTINGS allow_experimental_correlated_subqueries = 1; From eb3e75140cf15dd5e9341d825dc203b1d15b0cc5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 12 Dec 2025 01:42:04 +0000 Subject: [PATCH 060/113] Backport #92009 to 25.8: Fix flaky 00906_low_cardinality_cache --- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index d34550dd5ac3..81574ac6f0c1 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -2,7 +2,7 @@ SET max_rows_to_read = '100M', max_execution_time = 600; drop table if exists lc_00906; -create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; -insert into lc_00906 select '0123456789' from numbers(100000000); +create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi', vertical_merge_algorithm_min_rows_to_activate=100000000; +insert into lc_00906 select '0123456789' from numbers(100000000) SETTINGS max_insert_threads=6, max_threads=4; select count(), b from lc_00906 group by b; drop table if exists lc_00906; From 3b3470c95747999083a6af350927e0e7254e4e98 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 15 Dec 2025 06:19:30 +0000 Subject: [PATCH 061/113] Backport #91420 to 25.8: Fix crash in `dictGetDescendants` caused by `NULL` when dictionary type has Hierarchy support but no column is `HIERARCHICAL` --- src/Functions/FunctionsExternalDictionaries.h | 13 ++++- ...749_dictionary_hierarchical_null.reference | 45 ++++++++++++++ .../03749_dictionary_hierarchical_null.sql | 58 +++++++++++++++++++ 3 files changed, 115 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03749_dictionary_hierarchical_null.reference create mode 100644 tests/queries/0_stateless/03749_dictionary_hierarchical_null.sql diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 95130ba9450e..76251b874e25 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -94,7 +94,7 @@ class FunctionDictHelper : public WithContext return getDictionary(dict_name_col->getValue()); } - static const DictionaryAttribute & getDictionaryHierarchicalAttribute(const std::shared_ptr & dictionary) + static void checkDictionaryHierarchySupport(const std::shared_ptr & dictionary) { const auto & dictionary_structure = dictionary->getStructure(); auto hierarchical_attribute_index_optional = dictionary_structure.hierarchical_attribute_index; @@ -103,6 +103,14 @@ class FunctionDictHelper : public WithContext throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary {} does not support hierarchy", dictionary->getFullName()); + } + + static const DictionaryAttribute & getDictionaryHierarchicalAttribute(const std::shared_ptr & dictionary) + { + checkDictionaryHierarchySupport(dictionary); + + const auto & dictionary_structure = dictionary->getStructure(); + auto hierarchical_attribute_index_optional = dictionary_structure.hierarchical_attribute_index; size_t hierarchical_attribute_index = *hierarchical_attribute_index_optional; const auto & hierarchical_attribute = dictionary_structure.attributes[hierarchical_attribute_index]; @@ -1316,6 +1324,9 @@ class FunctionDictGetDescendantsOverloadResolverImpl final : public IFunctionOve FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { auto dictionary = dictionary_helper->getDictionary(arguments[0].column); + + FunctionDictHelper::checkDictionaryHierarchySupport(dictionary); + auto hierarchical_parent_to_child_index = dictionary->getHierarchicalIndex(); size_t level = Strategy::default_level; diff --git a/tests/queries/0_stateless/03749_dictionary_hierarchical_null.reference b/tests/queries/0_stateless/03749_dictionary_hierarchical_null.reference new file mode 100644 index 000000000000..b7f0ef51d182 --- /dev/null +++ b/tests/queries/0_stateless/03749_dictionary_hierarchical_null.reference @@ -0,0 +1,45 @@ +-- {echoOn } + +DROP DICTIONARY IF EXISTS d0; +CREATE DICTIONARY d0 (c0 Int) PRIMARY KEY (c0) SOURCE(NULL()) LAYOUT(FLAT()) LIFETIME(1); +SELECT dictGetDescendants('d0', 'c0', 1); -- { serverError UNSUPPORTED_METHOD } +SELECT dictGetDescendants('d0', 'c0', NULL); -- { serverError UNSUPPORTED_METHOD } +SELECT dictGetDescendants('d0', NULL, 1); -- { serverError UNSUPPORTED_METHOD } +SELECT dictGetChildren('d0', 'c0'); -- { serverError UNSUPPORTED_METHOD } +SELECT dictGetChildren('d0', NULL); -- { serverError UNSUPPORTED_METHOD } +SELECT dictGetChildren(NULL, NULL); -- { serverError UNSUPPORTED_METHOD } +DROP DICTIONARY IF EXISTS hierarchical_dictionary; +DROP TABLE IF EXISTS hierarchy_source; +CREATE TABLE hierarchy_source +( + id UInt64, + parent_id UInt64, + name String +) ENGINE = Memory; +INSERT INTO hierarchy_source VALUES +(0, 0, 'Root'), +(1, 0, 'Level 1 - Node 1'), +(2, 1, 'Level 2 - Node 2'), +(3, 1, 'Level 2 - Node 3'), +(4, 2, 'Level 3 - Node 4'), +(5, 2, 'Level 3 - Node 5'), +(6, 3, 'Level 3 - Node 6'); +CREATE DICTIONARY hierarchical_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL, + name String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'hierarchy_source')) +LAYOUT(HASHED()) +LIFETIME(0); +SELECT dictGetDescendants('hierarchical_dictionary', 1, 1); +[3,2] +SELECT dictGetDescendants('hierarchical_dictionary', NULL, 1); +\N +SELECT dictGetDescendants('hierarchical_dictionary', 1, NULL); -- { serverError BAD_ARGUMENTS } +SELECT dictGetChildren('hierarchical_dictionary', 1); +[3,2] +SELECT dictGetChildren('hierarchical_dictionary', NULL); +\N diff --git a/tests/queries/0_stateless/03749_dictionary_hierarchical_null.sql b/tests/queries/0_stateless/03749_dictionary_hierarchical_null.sql new file mode 100644 index 000000000000..10ddcccc4e71 --- /dev/null +++ b/tests/queries/0_stateless/03749_dictionary_hierarchical_null.sql @@ -0,0 +1,58 @@ +-- {echoOn } + +DROP DICTIONARY IF EXISTS d0; +CREATE DICTIONARY d0 (c0 Int) PRIMARY KEY (c0) SOURCE(NULL()) LAYOUT(FLAT()) LIFETIME(1); + +SELECT dictGetDescendants('d0', 'c0', 1); -- { serverError UNSUPPORTED_METHOD } + +SELECT dictGetDescendants('d0', 'c0', NULL); -- { serverError UNSUPPORTED_METHOD } + +SELECT dictGetDescendants('d0', NULL, 1); -- { serverError UNSUPPORTED_METHOD } + +SELECT dictGetChildren('d0', 'c0'); -- { serverError UNSUPPORTED_METHOD } + +SELECT dictGetChildren('d0', NULL); -- { serverError UNSUPPORTED_METHOD } + +SELECT dictGetChildren(NULL, NULL); -- { serverError UNSUPPORTED_METHOD } + + +DROP DICTIONARY IF EXISTS hierarchical_dictionary; +DROP TABLE IF EXISTS hierarchy_source; + +CREATE TABLE hierarchy_source +( + id UInt64, + parent_id UInt64, + name String +) ENGINE = Memory; + +INSERT INTO hierarchy_source VALUES +(0, 0, 'Root'), +(1, 0, 'Level 1 - Node 1'), +(2, 1, 'Level 2 - Node 2'), +(3, 1, 'Level 2 - Node 3'), +(4, 2, 'Level 3 - Node 4'), +(5, 2, 'Level 3 - Node 5'), +(6, 3, 'Level 3 - Node 6'); + + +CREATE DICTIONARY hierarchical_dictionary +( + id UInt64, + parent_id UInt64 HIERARCHICAL, + name String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'hierarchy_source')) +LAYOUT(HASHED()) +LIFETIME(0); + +SELECT dictGetDescendants('hierarchical_dictionary', 1, 1); + +SELECT dictGetDescendants('hierarchical_dictionary', NULL, 1); + +SELECT dictGetDescendants('hierarchical_dictionary', 1, NULL); -- { serverError BAD_ARGUMENTS } + +SELECT dictGetChildren('hierarchical_dictionary', 1); + +SELECT dictGetChildren('hierarchical_dictionary', NULL); From 646cdad126818b663ab1303d1475ca0cc043cd92 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 15 Dec 2025 17:19:31 +0000 Subject: [PATCH 062/113] Backport #88350 to 25.8: count returns wrong answer --- .../RemoveUnusedProjectionColumnsPass.cpp | 34 ---------- src/Analyzer/UnionNode.cpp | 29 +++++++++ src/Parsers/SelectUnionMode.h | 2 +- .../03652_union_columns_2.reference | 8 +++ .../0_stateless/03652_union_columns_2.sql | 65 +++++++++++++++++++ 5 files changed, 103 insertions(+), 35 deletions(-) create mode 100644 tests/queries/0_stateless/03652_union_columns_2.reference create mode 100644 tests/queries/0_stateless/03652_union_columns_2.sql diff --git a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp index c9b60f0d4722..a190b312946d 100644 --- a/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp +++ b/src/Analyzer/Passes/RemoveUnusedProjectionColumnsPass.cpp @@ -183,40 +183,6 @@ void RemoveUnusedProjectionColumnsPass::run(QueryTreeNodePtr & query_tree_node, if (query_node->isDistinct()) continue; } - else - { - auto * union_node = query_or_union_node->as(); - chassert(union_node != nullptr); - - /// We can't remove unused projections in the case of EXCEPT and INTERSECT - /// because it can lead to incorrect query results. Example: - /// - /// SELECT count() - /// FROM - /// ( - /// SELECT - /// 1 AS a, - /// 2 AS b - /// INTERSECT ALL - /// SELECT - /// 1, - /// 1 - /// ) - /// - /// Will be transformed into the following query with output 1 instead of 0: - /// - /// SELECT count() - /// FROM - /// ( - /// SELECT - /// 1 AS a, -- we must keep at least 1 column - /// INTERSECT ALL - /// SELECT - /// 1 - /// ); - if (union_node->getUnionMode() > SelectUnionMode::UNION_DISTINCT) - continue; - } auto used_projection_indexes = convertUsedColumnNamesToUsedProjectionIndexes(query_or_union_node, used_columns); updateUsedProjectionIndexes(query_or_union_node, used_projection_indexes); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 34a4b2b69749..4fb43b120eef 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -122,6 +122,35 @@ void UnionNode::removeUnusedProjectionColumns(const std::unordered_set & if (recursive_cte_table) return; + /// We can't remove unused projections in the case of EXCEPT and INTERSECT + /// because it can lead to incorrect query results. Example: + /// + /// SELECT count() + /// FROM + /// ( + /// SELECT + /// 1 AS a, + /// 2 AS b + /// INTERSECT ALL + /// SELECT + /// 1, + /// 1 + /// ) + /// + /// Will be transformed into the following query with output 1 instead of 0: + /// + /// SELECT count() + /// FROM + /// ( + /// SELECT + /// 1 AS a, -- we must keep at least 1 column + /// INTERSECT ALL + /// SELECT + /// 1 + /// ); + if (union_mode > SelectUnionMode::UNION_DISTINCT) + return; + auto & query_nodes = getQueries().getNodes(); for (auto & query_node : query_nodes) { diff --git a/src/Parsers/SelectUnionMode.h b/src/Parsers/SelectUnionMode.h index fc0a766eb668..8089dac529c1 100644 --- a/src/Parsers/SelectUnionMode.h +++ b/src/Parsers/SelectUnionMode.h @@ -7,7 +7,7 @@ namespace DB { enum class SelectUnionMode : uint8_t { - UNION_DEFAULT, + UNION_DEFAULT = 0, UNION_ALL, UNION_DISTINCT, EXCEPT_DEFAULT, diff --git a/tests/queries/0_stateless/03652_union_columns_2.reference b/tests/queries/0_stateless/03652_union_columns_2.reference new file mode 100644 index 000000000000..0bda99fa6081 --- /dev/null +++ b/tests/queries/0_stateless/03652_union_columns_2.reference @@ -0,0 +1,8 @@ +run enable-analyzer=1 +0 5 +1 1 +2 1 +run enable-analyzer=1 ignore +0 5 +1 1 +2 1 diff --git a/tests/queries/0_stateless/03652_union_columns_2.sql b/tests/queries/0_stateless/03652_union_columns_2.sql new file mode 100644 index 000000000000..3594aea23d21 --- /dev/null +++ b/tests/queries/0_stateless/03652_union_columns_2.sql @@ -0,0 +1,65 @@ +CREATE TABLE left (g UInt32, i UInt32) + ORDER BY (g, i); + +INSERT INTO left VALUES +(0, 1), (0, 2), (0, 3), (0, 4), (0, 5), (2, 0); + +CREATE TABLE right (g UInt32, i UInt32) + ORDER BY (g, i); + +INSERT INTO right VALUES +(0,0), (0, 3), (0, 4), (0, 6), (1, 0); + +SET enable_analyzer = 1; + +SELECT 'run enable-analyzer=1'; +with differences as + ( + ( + select g, i from left + where g BETWEEN 0 and 10 + EXCEPT ALL + select g, i from right + where g BETWEEN 0 and 10 + ) + UNION ALL + ( + select g, i from right + where g BETWEEN 0 and 10 + EXCEPT ALL + select g, i from left + where g BETWEEN 0 and 10 + ) + ), +diff_counts as + ( + select g, count(*) from differences group by g + ) +select * from diff_counts +ORDER BY g; + +SELECT 'run enable-analyzer=1 ignore'; +with differences as + ( + ( + select g, i from left + where g BETWEEN 0 and 10 + EXCEPT ALL + select g, i from right + where g BETWEEN 0 and 10 + ) + UNION ALL + ( + select g, i from right + where g BETWEEN 0 and 10 + EXCEPT ALL + select g, i from left + where g BETWEEN 0 and 10 + ) + ), +diff_counts as + ( + select g, count(ignore(*)) from differences group by g + ) +select * from diff_counts +ORDER BY g; From 1c531f23536426579f178bf89e33bcac85108639 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Dec 2025 12:20:25 +0000 Subject: [PATCH 063/113] Backport #88879 to 25.8: Fix handling of non materialized Nested(Tuple(...)) (added via ALTER) --- src/DataTypes/ObjectUtils.cpp | 39 ++++++++++++++++--- .../03672_nested_array_nested_tuple.reference | 2 + .../03672_nested_array_nested_tuple.sql | 13 +++++++ 3 files changed, 49 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03672_nested_array_nested_tuple.reference create mode 100644 tests/queries/0_stateless/03672_nested_array_nested_tuple.sql diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index b351eb8f297a..b8f01cdc9c7b 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -70,12 +70,19 @@ DataTypePtr getBaseTypeOfArray(const DataTypePtr & type) DataTypePtr getBaseTypeOfArray(DataTypePtr type, const Names & tuple_elements) { auto it = tuple_elements.begin(); + + /// Get underlying type for array, but w/o processing tuple elements that are not part of the nested, so it is done in 3 steps: + /// 1. Find Nested type (since it can be part of Tuple/Array) + /// 2. Process all Nested types (this is Array(Tuple()), it is responsibility of the caller to re-create proper Array nesting) + /// 3. Strip all nested arrays (it is responsibility of the caller to re-create proper Array nesting) + + /// 1. Find Nested type (since it can be part of Tuple/Array) while (true) { - if (const auto * type_array = typeid_cast(type.get())) - { + if (type->hasCustomName()) + break; + else if (const auto * type_array = typeid_cast(type.get())) type = type_array->getNestedType(); - } else if (const auto * type_tuple = typeid_cast(type.get())) { if (it == tuple_elements.end()) @@ -84,16 +91,38 @@ DataTypePtr getBaseTypeOfArray(DataTypePtr type, const Names & tuple_elements) auto pos = type_tuple->tryGetPositionByName(*it); if (!pos) break; - ++it; + type = type_tuple->getElement(*pos); } else - { break; + } + + /// 2. Process all Nested types (this is Array(Tuple()), it is responsibility of the caller to re-create proper Array nesting) + while (type->hasCustomName()) + { + if (const auto * type_nested = typeid_cast(type->getCustomName())) + { + if (it == tuple_elements.end()) + break; + + const auto & names = type_nested->getNames(); + auto pos = std::find(names.begin(), names.end(), *it); + if (pos == names.end()) + break; + ++it; + + type = type_nested->getElements().at(std::distance(names.begin(), pos)); } + else + break; } + /// 3. Strip all nested arrays (it is responsibility of the caller to re-create proper Array nesting) + while (const auto * type_array = typeid_cast(type.get())) + type = type_array->getNestedType(); + return type; } diff --git a/tests/queries/0_stateless/03672_nested_array_nested_tuple.reference b/tests/queries/0_stateless/03672_nested_array_nested_tuple.reference new file mode 100644 index 000000000000..0e143f4925ce --- /dev/null +++ b/tests/queries/0_stateless/03672_nested_array_nested_tuple.reference @@ -0,0 +1,2 @@ +[1,2,3] [(0),(0),(0)] +[1,2,3] [((0)),((0)),((0))] diff --git a/tests/queries/0_stateless/03672_nested_array_nested_tuple.sql b/tests/queries/0_stateless/03672_nested_array_nested_tuple.sql new file mode 100644 index 000000000000..19060b088dff --- /dev/null +++ b/tests/queries/0_stateless/03672_nested_array_nested_tuple.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS nest; +CREATE TABLE nest (nested_field Nested(e1 Int32)) ENGINE = MergeTree() ORDER BY nested_field.e1; +INSERT INTO nest (nested_field.e1) VALUES ([1, 2, 3]); +ALTER TABLE nest ADD COLUMN nested_field.e2 Array(Tuple(some_value Int32)); +OPTIMIZE TABLE nest FINAL; +SELECT * FROM nest; + +DROP TABLE IF EXISTS nest_2; +CREATE TABLE nest_2 (nested_field Nested(e1 Int32)) ENGINE = MergeTree() ORDER BY nested_field.e1; +INSERT INTO nest_2 (nested_field.e1) VALUES ([1, 2, 3]); +ALTER TABLE nest_2 ADD COLUMN nested_field.e2 Array(Tuple(some_value Tuple(another_value Int32))); +OPTIMIZE TABLE nest_2 FINAL; +SELECT * FROM nest_2; From 555616b049033f63a15fb9783947d8cd1d03ae52 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Dec 2025 14:14:15 +0000 Subject: [PATCH 064/113] Backport #92210 to 25.8: Fix UDF replace in format --- .../UserDefinedSQLFunctionVisitor.cpp | 17 ----------------- .../UserDefined/UserDefinedSQLFunctionVisitor.h | 2 +- .../03760_udf_group_by_format_crash.reference | 0 .../03760_udf_group_by_format_crash.sh | 14 ++++++++++++++ 4 files changed, 15 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/03760_udf_group_by_format_crash.reference create mode 100755 tests/queries/0_stateless/03760_udf_group_by_format_crash.sh diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp index 345f1c514ad3..d87901e81bb0 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.cpp @@ -37,14 +37,6 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast, ContextPtr context_) { chassert(ast); - if (const auto * function = ast->template as()) - { - std::unordered_set udf_in_replace_process; - auto replace_result = tryToReplaceFunction(*function, udf_in_replace_process, context_); - if (replace_result) - ast = replace_result; - } - for (auto & child : ast->children) { if (!child) @@ -69,15 +61,6 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast, ContextPtr context_) } } -void UserDefinedSQLFunctionVisitor::visit(IAST * ast, ContextPtr context_) -{ - if (!ast) - return; - - for (auto & child : ast->children) - visit(child, context_); -} - namespace { bool isVariadic(const ASTPtr & arg) diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h index 633f68db3487..e9bcd145d701 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h @@ -23,8 +23,8 @@ class UserDefinedSQLFunctionVisitor { public: static void visit(ASTPtr & ast, ContextPtr context_); + private: - static void visit(IAST *, ContextPtr context_); static ASTPtr tryToReplaceFunction(const ASTFunction & function, std::unordered_set & udf_in_replace_process, ContextPtr context_); }; diff --git a/tests/queries/0_stateless/03760_udf_group_by_format_crash.reference b/tests/queries/0_stateless/03760_udf_group_by_format_crash.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03760_udf_group_by_format_crash.sh b/tests/queries/0_stateless/03760_udf_group_by_format_crash.sh new file mode 100755 index 000000000000..b6f923b65fe8 --- /dev/null +++ b/tests/queries/0_stateless/03760_udf_group_by_format_crash.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP FUNCTION IF EXISTS ${CLICKHOUSE_DATABASE}_f0; + CREATE FUNCTION ${CLICKHOUSE_DATABASE}_f0 AS (p0) -> p0; + + CREATE TABLE t0 (c0 Int) ENGINE = MergeTree() ORDER BY ${CLICKHOUSE_DATABASE}_f0(${CLICKHOUSE_DATABASE}_f0(COLUMNS('1'))) SETTINGS log_formatted_queries = 1; -- { serverError BAD_ARGUMENTS }; + + DROP FUNCTION ${CLICKHOUSE_DATABASE}_f0; +" \ No newline at end of file From 8cb89621ef4e5d7af038cd256aad2fa024fce2e4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Dec 2025 15:15:48 +0000 Subject: [PATCH 065/113] Backport #91886 to 25.8: Add repair mechanism for JSON with duplicated path in shared data and dynamic paths --- src/Columns/ColumnObject.cpp | 114 ++++++++++++++++-- src/Columns/ColumnObject.h | 8 +- src/Columns/tests/gtest_column_object.cpp | 30 +++++ .../Serializations/SerializationObject.cpp | 2 +- 4 files changed, 140 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 6b5b0e6a90da..6a20a2e132d9 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -728,10 +729,8 @@ void ColumnObject::insertFromSharedDataAndFillRemainingDynamicPaths(const DB::Co { /// Deserialize binary value into dynamic column from shared data. if (it->second->size() != current_size) - { - src_object_column.validateDynamicPathsAndSharedData(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of dynamic path {}: {} != {}", path, it->second->size(), current_size); - } + deserializeValueFromSharedData(src_shared_data_values, i, *it->second); } else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) @@ -2070,11 +2069,12 @@ int ColumnObject::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_d return 1; } -void ColumnObject::validateDynamicPathsAndSharedData(size_t shared_data_offset) const +void ColumnObject::repairDuplicatesInDynamicPathsAndSharedData(size_t offset) { if (dynamic_paths.empty()) return; + /// First, check if all dynamic paths have correct sizes, just in case. size_t expected_size = shared_data->size(); for (const auto & [path, column] : dynamic_paths) { @@ -2082,16 +2082,108 @@ void ColumnObject::validateDynamicPathsAndSharedData(size_t shared_data_offset) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of dynamic path {}: {} != {}", path, column->size(), expected_size); } + /// Second, iterate over paths in shared data and check if we have any path that is also present in dynamic paths. const auto & shared_data_offsets = getSharedDataOffsets(); - const auto [shared_data_paths, _] = getSharedDataPathsAndValues(); - size_t shared_data_paths_start = shared_data_offsets[ssize_t(shared_data_offset) - 1]; - size_t shared_data_paths_end = shared_data_offsets.back(); - for (size_t i = shared_data_paths_start; i != shared_data_paths_end; ++i) + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + /// Remember the first row with duplicates if any. We will start repair from this row. + std::optional first_row_with_duplicates = std::nullopt; + size_t size = shared_data_offsets.size(); + for (size_t i = offset; i < size; ++i) + { + size_t shared_data_start = shared_data_offsets[i - 1]; + size_t shared_data_end = shared_data_offsets[i]; + for (size_t j = shared_data_start; j < shared_data_end; ++j) + { + if (dynamic_paths.contains(shared_data_paths->getDataAt(j))) + { + /// Duplicate is found, no need to iterate further, we need to start repair. + first_row_with_duplicates = i; + break; + } + } + + if (first_row_with_duplicates) + break; + } + + if (!first_row_with_duplicates) + return; + + LOG_TRACE(getLogger("ColumnObject"), "Repair duplicates in Object column starting from row {}", *first_row_with_duplicates); + + /// During repair we create new shared data without duplicated dynamic paths + /// update corresponding dynamic paths with values from shared data. + auto new_shared_data = shared_data->cloneResized(*first_row_with_duplicates); + const auto [new_shared_data_paths, new_shared_data_values, new_shared_data_offsets] = getSharedDataPathsValuesAndOffsets(*new_shared_data); + new_shared_data_offsets->reserve(size); + PathToColumnMap new_dynamic_paths; + for (size_t i = *first_row_with_duplicates; i < size; ++i) { - auto path = shared_data_paths->getDataAt(i); - if (dynamic_paths.contains(path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is present both in dynamic paths and in shared data", path.toString()); + size_t shared_data_start = shared_data_offsets[i - 1]; + size_t shared_data_end = shared_data_offsets[i]; + for (size_t j = shared_data_start; j < shared_data_end; ++j) + { + auto path = shared_data_paths->getDataAt(j); + auto it = dynamic_paths.find(path); + if (it == dynamic_paths.end()) + { + new_shared_data_paths->insertFrom(*shared_data_paths, j); + new_shared_data_values->insertFrom(*shared_data_values, j); + } + /// We update dynamic path with value from shared data only if dynamic path has NULL at this row. + else if (it->second->isNullAt(i)) + { + auto new_it = new_dynamic_paths.find(path); + if (new_it == new_dynamic_paths.end()) + { + new_it = new_dynamic_paths.emplace(path, it->second->cloneResized(i)).first; + new_it->second->reserve(size); + } + + deserializeValueFromSharedData(shared_data_values, j, *new_it->second); + } + /// Situation when both values in dynamic path and shared data are non-NULL should not be possible + /// and we cannot repair it anyhow. Throw logical error exception in this case. + else + { + auto value = shared_data_values->getDataAt(j); + ReadBufferFromMemory buf(value); + auto type_from_shared_data = decodeDataType(buf); + if (!isNothing(type_from_shared_data)) + { + auto type_from_dynamic_path = dynamic_paths_ptrs.find(path)->second->getTypeAt(i); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Path {} is present both in dynamic paths and shared data and has two non-null values at the row {}." + "Value type in dynamic paths: {}. Value type in shared data: {}", + path, + i, + type_from_dynamic_path->getName(), + type_from_shared_data->getName()); + } + } + } + + new_shared_data_offsets->push_back(new_shared_data_paths->size()); + /// Update new dynamic paths that were not updated with value from shared data in this row. + for (auto & [path, column] : new_dynamic_paths) + { + if (column->size() == i) + column->insertFrom(*dynamic_paths.at(path), i); + } } + + for (auto & [path, column] : new_dynamic_paths) + { + if (column->size() != size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of new dynamic path {}: {} != {}", path, column->size(), size); + dynamic_paths_ptrs[path] = assert_cast(column.get()); + dynamic_paths[path] = std::move(column); + } + + if (new_shared_data->size() != size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size of new shared data: {} != {}", new_shared_data->size(), size); + shared_data = std::move(new_shared_data); } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 9a5b05a8c6c2..9cda2d2bb6b0 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -282,8 +282,12 @@ class ColumnObject final : public COWHelper, ColumnO /// Insert all the data from shared data with specified path to dynamic column. static void fillPathColumnFromSharedData(IColumn & path_column, StringRef path, const ColumnPtr & shared_data_column, size_t start, size_t end); - /// Validate that all dynamic paths have correct sizes and that shared data doesn't contain any dynamic paths. - void validateDynamicPathsAndSharedData(size_t shared_data_offset = 0) const; + /// Due to previous bugs we can have an invalid state where we have some path + /// both in shared data and in dynamic paths and only one value is not NULL. + /// This methods repairs the column and removes this duplicate by removing path + /// and value from shared data and keeping only dynamic path containing non-Null value. + /// offset argument - is the offset from which we should check for duplicates. + void repairDuplicatesInDynamicPathsAndSharedData(size_t offset = 0); private: class SortedPathsIterator; diff --git a/src/Columns/tests/gtest_column_object.cpp b/src/Columns/tests/gtest_column_object.cpp index 2970b44a51fb..4a0ca225617d 100644 --- a/src/Columns/tests/gtest_column_object.cpp +++ b/src/Columns/tests/gtest_column_object.cpp @@ -414,3 +414,33 @@ TEST(ColumnObject, rollback) ASSERT_EQ((*typed_paths.at("a.a"))[0], Field{1u}); ASSERT_EQ((*dynamic_paths.at("a.c"))[1], Field{"ccc"}); } + +TEST(ColumnObject, RepairDuplicatesInDynamicPathsAndSharedData) +{ + auto type_with_dynamic_paths = DataTypeFactory::instance().get("JSON"); + auto column_with_dynamic_paths = type_with_dynamic_paths->createColumn(); + auto & column_object_with_dynamic_paths = assert_cast(*column_with_dynamic_paths); + column_object_with_dynamic_paths.insert(Object{{"a", Field{1u}}}); + column_object_with_dynamic_paths.insert(Object{{"b", Field{1u}}}); + column_object_with_dynamic_paths.insert(Object{{"c", Field{1u}}}); + column_object_with_dynamic_paths.insert(Object{{"d", Field{1u}}}); + + auto type_with_shared_data_paths = DataTypeFactory::instance().get("JSON(max_dynamic_paths=0)"); + auto column_with_shared_data_paths = type_with_shared_data_paths->createColumn(); + auto & column_object_with_shared_data_paths = assert_cast(*column_with_shared_data_paths); + column_object_with_shared_data_paths.insert(Object{}); + column_object_with_shared_data_paths.insert(Object{{"a", Field{1u}}, {"c", Field{1u}}}); + column_object_with_shared_data_paths.insert(Object{{"d", Field{1u}}, {"b", Field{1u}}}); + column_object_with_shared_data_paths.insert(Object{}); + + std::unordered_map dynamic_paths; + for (const auto & [path, column] : column_object_with_dynamic_paths.getDynamicPaths()) + dynamic_paths[path] = IColumn::mutate(column); + + auto column_object = ColumnObject::create({}, std::move(dynamic_paths), IColumn::mutate(column_object_with_shared_data_paths.getSharedDataPtr()), 4, 4, 16); + column_object->repairDuplicatesInDynamicPathsAndSharedData(0); + ASSERT_EQ((*column_object)[0], (Object{{"a", Field(1u)}})); + ASSERT_EQ((*column_object)[1], (Object{{"a", Field(1u)}, {"b", Field(1u)}, {"c", Field(1u)}})); + ASSERT_EQ((*column_object)[2], (Object{{"b", Field(1u)}, {"c", Field(1u)}, {"d", Field(1u)}})); + ASSERT_EQ((*column_object)[3], (Object{{"d", Field(1u)}})); +} diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 6108fc5fc528..a9030100796c 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -1037,7 +1037,7 @@ void SerializationObject::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); settings.path.pop_back(); - column_object.validateDynamicPathsAndSharedData(shared_data_previous_size); + column_object.repairDuplicatesInDynamicPathsAndSharedData(shared_data_previous_size); } void SerializationObject::serializeBinary(const Field & field, WriteBuffer & ostr, const DB::FormatSettings & settings) const From 6d5def69e943de4f0deb76fa15aefd67c9080746 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Dec 2025 21:17:29 +0000 Subject: [PATCH 066/113] Backport #84611 to 25.8: Distributed INSERT SELECT with WHERE --- src/Interpreters/InterpreterInsertQuery.cpp | 2 - .../test.py | 54 +++++++++++++++++++ .../test_s3_cluster_insert_select/test.py | 40 ++++++++++++++ 3 files changed, 94 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 619a2799a0bf..0d59042499d2 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -633,8 +633,6 @@ static bool isInsertSelectTrivialEnoughForDistributedExecution(const ASTInsertQu /// TODO: replace with QueryTree analysis after switching to analyzer completely return (!select_query->distinct && !select_query->limit_with_ties - && !select_query->prewhere() - && !select_query->where() && !select_query->groupBy() && !select_query->having() && !select_query->orderBy() diff --git a/tests/integration/test_parallel_replicas_insert_select/test.py b/tests/integration/test_parallel_replicas_insert_select/test.py index 977abed7dfad..89fa7a292abc 100644 --- a/tests/integration/test_parallel_replicas_insert_select/test.py +++ b/tests/integration/test_parallel_replicas_insert_select/test.py @@ -274,3 +274,57 @@ def test_insert_select_with_constant(start_cluster, max_parallel_replicas, paral ) == f"{populate_count}\n" ) + + +@pytest.mark.parametrize( + "max_parallel_replicas", + [ + pytest.param(2), + pytest.param(3), + ], +) +@pytest.mark.parametrize( + "parallel_replicas_local_plan", + [ + pytest.param(False), + pytest.param(True), + ] +) +def test_insert_select_where(start_cluster, max_parallel_replicas, parallel_replicas_local_plan): + populate_count = 1_000_000 + count = int(populate_count / 10) + cluster_name = "test_1_shard_3_replicas" + + source_table = "t_source" + create_tables(source_table, populate_count=populate_count, skip_last_replica=False) + target_table = "t_target" + create_tables(target_table, populate_count=0, skip_last_replica=False) + + query_id = str(uuid.uuid4()) + node1.query( + f"INSERT INTO {target_table} SELECT * FROM {source_table} WHERE key % 10 = 0", + settings={ + "parallel_distributed_insert_select": 2, + "enable_parallel_replicas": 2, + "max_parallel_replicas": max_parallel_replicas, + "cluster_for_parallel_replicas": cluster_name, + "parallel_replicas_local_plan": parallel_replicas_local_plan, + "enable_analyzer": 1, + }, + query_id=query_id + ) + node1.query(f"SYSTEM SYNC REPLICA {target_table} LIGHTWEIGHT") + assert ( + node1.query( + f"select count() from {target_table}" + ) + == f"{count}\n" + ) + + # check that query executed in distributed way + execute_on_cluster(f"SYSTEM FLUSH LOGS query_log") + number_of_queries = node1.query( + f"""SELECT count() FROM clusterAllReplicas({cluster_name}, system.query_log) WHERE current_database = currentDatabase() AND initial_query_id = '{query_id}' AND type = 'QueryFinish' AND query_kind = 'Insert'""", + settings={"skip_unavailable_shards": 1}, + ) + assert (int(number_of_queries) > 1) diff --git a/tests/integration/test_s3_cluster_insert_select/test.py b/tests/integration/test_s3_cluster_insert_select/test.py index d35ab6fb0fa7..7535640f57a4 100644 --- a/tests/integration/test_s3_cluster_insert_select/test.py +++ b/tests/integration/test_s3_cluster_insert_select/test.py @@ -187,6 +187,46 @@ def test_distributed_insert_select_to_rmt_limit(started_cluster): ) +def test_distributed_insert_select_to_rmt_where(started_cluster): + table = "t_rmt_target" + cluster_name = "cluster_1_shard_3_replicas" + + node1.query( + f"""DROP TABLE IF EXISTS {table} ON CLUSTER '{cluster_name}' SYNC;""" + ) + + node1.query( + f""" + CREATE TABLE {table} ON CLUSTER {cluster_name} (a String, b UInt64) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/32c614a9-13af-43c5-848c-a3f62a78e390/{table}', '{{replica}}') + ORDER BY (a, b); + """ + ) + + node1.query( + f""" + INSERT INTO {table} SELECT * FROM s3Cluster( + '{cluster_name}', + 'http://minio1:9001/root/data/generated/*.csv', 'minio', '{minio_secret_key}', 'CSV','a String, b UInt64' + ) WHERE b = 100 SETTINGS parallel_distributed_insert_select=2; + """ + ) + + node1.query(f"SYSTEM SYNC REPLICA {table}") + + assert ( + int( + node1.query( + f"SELECT count(*) FROM {table};" + ).strip() + ) == 99 + ) + + node1.query( + f"""DROP TABLE IF EXISTS {table} ON CLUSTER '{cluster_name}' SYNC;""" + ) + + def test_distributed_insert_select_to_rmt_cte_const(started_cluster): table = "t_rmt_target" cluster_name = "cluster_1_shard_3_replicas" From 4b88f4e9cc855c39625ce20b1a887ff1d8c84687 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Dec 2025 01:42:19 +0000 Subject: [PATCH 067/113] Backport #92223 to 25.8: Remove no active host exception in DDL worker --- src/Interpreters/DDLWorker.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index bfbc507d2246..f4b41bc80c58 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -76,7 +76,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; extern const int NO_ZOOKEEPER; - extern const int INVALID_CONFIG_PARAMETER; } constexpr const char * TASK_PROCESSED_OUT_REASON = "Task has been already processed"; @@ -1378,8 +1377,10 @@ void DDLWorker::markReplicasActive(bool /*reinitialized*/) { const auto & cluster = it.second; if (!cluster->getHostIDs().empty()) - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, "There are clusters with host ids but no local host found for this replica."); + { + LOG_WARNING(log, "There are clusters with host ids but no local host found for this replica."); + break; + } } } } From cc697ab50435cc549d8c23d2af2bcb55b47923cb Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Dec 2025 14:55:03 +0100 Subject: [PATCH 068/113] Update ColumnObject.cpp --- src/Columns/ColumnObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 6a20a2e132d9..645a62a1e181 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2146,7 +2146,7 @@ void ColumnObject::repairDuplicatesInDynamicPathsAndSharedData(size_t offset) /// and we cannot repair it anyhow. Throw logical error exception in this case. else { - auto value = shared_data_values->getDataAt(j); + auto value = shared_data_values->getDataAt(j).toView(); ReadBufferFromMemory buf(value); auto type_from_shared_data = decodeDataType(buf); if (!isNothing(type_from_shared_data)) From 2741406f6fcb48fcbfb9fb8c225396dbb64f8c06 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Dec 2025 19:13:38 +0000 Subject: [PATCH 069/113] Backport #92390 to 25.8: Fix possible crash in aggregate functions after MEMORY_LIMIT_EXCEEDED --- src/Columns/ColumnAggregateFunction.cpp | 37 ++++++++-- .../tests/gtest_column_aggregate_function.cpp | 70 +++++++++++++++++++ src/Common/FailPoint.cpp | 1 + 3 files changed, 101 insertions(+), 7 deletions(-) create mode 100644 src/Columns/tests/gtest_column_aggregate_function.cpp diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 70deb3d6ce92..4ce3c923ce9b 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -11,8 +11,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -32,6 +34,12 @@ namespace ErrorCodes extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; + extern const int MEMORY_LIMIT_EXCEEDED; +} + +namespace FailPoints +{ + extern const char column_aggregate_function_ensureOwnership_exception[]; } @@ -226,29 +234,44 @@ void ColumnAggregateFunction::ensureOwnership() size_t size_of_state = func->sizeOfData(); size_t align_of_state = func->alignOfData(); + bool inject_memory_limit_exceeded = false; + /// Avoid checking failpoint in a loop, since it uses atomic's + fiu_do_on(FailPoints::column_aggregate_function_ensureOwnership_exception, + { + inject_memory_limit_exceeded = true; + }); + + Container new_data; + new_data.resize_exact(size); + size_t rollback_pos = 0; try { for (size_t i = 0; i < size; ++i) { - ConstAggregateDataPtr old_place = data[i]; - data[i] = arena.alignedAlloc(size_of_state, align_of_state); - func->create(data[i]); + new_data[i] = arena.alignedAlloc(size_of_state, align_of_state); + func->create(new_data[i]); ++rollback_pos; - func->merge(data[i], old_place, &arena); + + if (unlikely(inject_memory_limit_exceeded)) + throw Exception(ErrorCodes::MEMORY_LIMIT_EXCEEDED, "Failpoint triggered"); + + func->merge(new_data[i], data[i], &arena); } } catch (...) { /// If we failed to take ownership, destroy all temporary data. - if (!func->hasTrivialDestructor()) + { for (size_t i = 0; i < rollback_pos; ++i) - func->destroy(data[i]); - + func->destroy(new_data[i]); + } throw; } + data = std::move(new_data); + /// Now we own all data. src.reset(); } diff --git a/src/Columns/tests/gtest_column_aggregate_function.cpp b/src/Columns/tests/gtest_column_aggregate_function.cpp new file mode 100644 index 000000000000..e1c5a57cb9fe --- /dev/null +++ b/src/Columns/tests/gtest_column_aggregate_function.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB::FailPoints +{ + extern const char column_aggregate_function_ensureOwnership_exception[]; +} + +TEST(ColumnAggregateFunction, EnsureOwnershipExceptionLeavesCorruptedState) +{ + tryRegisterAggregateFunctions(); + + using namespace DB; + + // Create the aggregate function quantileDD with relative accuracy 0.01 + AggregateFunctionFactory & factory = AggregateFunctionFactory::instance(); + DataTypes argument_types = {std::make_shared()}; + Array params = {Field(0.01), Field(0.5)}; + AggregateFunctionProperties properties; + auto aggregate_function = factory.get("quantileDD", NullsAction::EMPTY, argument_types, params, properties); + + // Create a source column with some data + auto src_column = ColumnAggregateFunction::create(aggregate_function); + Arena arena_src; + auto data_column = ColumnFloat64::create(); + data_column->insert(Field(1.0)); + data_column->insert(Field(2.0)); + data_column->insert(Field(3.0)); + const IColumn * columns[1] = {data_column.get()}; + + for (size_t i = 0; i < 3; ++i) + { + src_column->insertDefault(); + aggregate_function->add(src_column->getData()[i], columns, i, &arena_src); + } + + // Create a view column from the source - this sets src pointer + auto view_column = src_column->cloneEmpty(); + view_column->insertRangeFrom(*src_column, 0, 3); + + // Enable failpoint that will trigger an exception during ensureOwnership + // This will happen after at least one state is created and destroyed + FailPointInjection::enableFailPoint(FailPoints::column_aggregate_function_ensureOwnership_exception); + + // Try to insert - this will call ensureOwnership() which will throw + // After the exception, previously, data[] points to destroyed memory where mapping == nullptr + ASSERT_THROW({ + view_column->insertDefault(); + }, Exception); + + // Disable failpoint + FailPointInjection::disableFailPoint(FailPoints::column_aggregate_function_ensureOwnership_exception); + + /// Previously leads to a crash + view_column->insertDefault(); +} diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 0adbd8102bf9..887509a19b92 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -88,6 +88,7 @@ static struct InitFiu ONCE(execute_query_calling_empty_set_result_func_on_exception) \ ONCE(receive_timeout_on_table_status_response) \ ONCE(delta_kernel_fail_literal_visitor) \ + ONCE(column_aggregate_function_ensureOwnership_exception) \ REGULAR(keepermap_fail_drop_data) \ REGULAR(lazy_pipe_fds_fail_close) \ PAUSEABLE(infinite_sleep) \ From 67636fb7bc054288acb50e01b6127939dbdbb1ee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Dec 2025 20:13:58 +0000 Subject: [PATCH 070/113] Backport #92351 to 25.8: Fix logical error caused by `Nothing` type in `caseWithExpression` function --- src/Functions/caseWithExpression.cpp | 9 --------- tests/queries/0_stateless/03390_non_constant_case.sql | 8 +++++++- .../03444_case_with_expression_exception.reference | 0 .../0_stateless/03444_case_with_expression_exception.sql | 1 - 4 files changed, 7 insertions(+), 11 deletions(-) delete mode 100644 tests/queries/0_stateless/03444_case_with_expression_exception.reference delete mode 100644 tests/queries/0_stateless/03444_case_with_expression_exception.sql diff --git a/src/Functions/caseWithExpression.cpp b/src/Functions/caseWithExpression.cpp index e1b7ce565ed1..7eba8caf7a70 100644 --- a/src/Functions/caseWithExpression.cpp +++ b/src/Functions/caseWithExpression.cpp @@ -29,7 +29,6 @@ class FunctionCaseWithExpression : public IFunction bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForNothing() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } size_t getNumberOfArguments() const override { return 0; } String getName() const override { return name; } @@ -62,14 +61,6 @@ class FunctionCaseWithExpression : public IFunction /// Helper function to implement CASE WHEN equality semantics where NULL = NULL is true ColumnPtr caseWhenEquals(const ColumnWithTypeAndName & expr, const ColumnWithTypeAndName & when_value, size_t input_rows_count) const { - // handle Nothing type - it's an empty type that can't contain any values - // if either argument is Nothing, the result should be an empty column - if (expr.type->onlyNull() || when_value.type->onlyNull()) - { - // return a constant false column - return DataTypeUInt8().createColumnConst(input_rows_count, 0u); - } - // for CASE WHEN semantics, NULL should match NULL // we need: if (isNull(expr)) then (isNull(when)) else if (isNull(when)) then 0 else (expr = when) diff --git a/tests/queries/0_stateless/03390_non_constant_case.sql b/tests/queries/0_stateless/03390_non_constant_case.sql index c1248c44dc26..f79a7e5a7483 100644 --- a/tests/queries/0_stateless/03390_non_constant_case.sql +++ b/tests/queries/0_stateless/03390_non_constant_case.sql @@ -59,4 +59,10 @@ SELECT caseWithExpression( materialize(NULL), NULL, NULL -); -- { serverError ILLEGAL_COLUMN } +); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT caseWithExpression('C', 'A', true, 'B', false); -- { serverError BAD_ARGUMENTS } + +SELECT caseWithExpression(1, assumeNotNull(materialize(NULL)), 1, 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT count() WHERE caseWithExpression(1, assumeNotNull(materialize(NULL)), 1, 1); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } diff --git a/tests/queries/0_stateless/03444_case_with_expression_exception.reference b/tests/queries/0_stateless/03444_case_with_expression_exception.reference deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/tests/queries/0_stateless/03444_case_with_expression_exception.sql b/tests/queries/0_stateless/03444_case_with_expression_exception.sql deleted file mode 100644 index 6a1fb051df82..000000000000 --- a/tests/queries/0_stateless/03444_case_with_expression_exception.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT caseWithExpression('C', 'A', true, 'B', false); -- { serverError BAD_ARGUMENTS } From 27a910d674fbdb959e439c1ed25bda6df812958d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Dec 2025 20:16:44 +0000 Subject: [PATCH 071/113] Backport #92079 to 25.8: Fix polaris catalog: encode nested namespace --- src/Databases/DataLake/RestCatalog.cpp | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Databases/DataLake/RestCatalog.cpp b/src/Databases/DataLake/RestCatalog.cpp index cd409cadbd26..a321d3f1b325 100644 --- a/src/Databases/DataLake/RestCatalog.cpp +++ b/src/Databases/DataLake/RestCatalog.cpp @@ -92,6 +92,19 @@ std::string correctAPIURI(const std::string & uri) return std::filesystem::path(uri) / "v1"; } +String encodeNamespaceForURI(const String & namespace_name) +{ + String encoded; + for (const auto & ch : namespace_name) + { + if (ch == '.') + encoded += "%1F"; + else + encoded.push_back(ch); + } + return encoded; +} + } std::string RestCatalog::Config::toString() const @@ -500,7 +513,8 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const DB::Names RestCatalog::getTables(const std::string & base_namespace, size_t limit) const { - const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / base_namespace / "tables"; + auto encoded_namespace = encodeNamespaceForURI(base_namespace); + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / encoded_namespace / "tables"; auto buf = createReadBuffer(config.prefix / endpoint); return parseTables(*buf, base_namespace, limit); @@ -594,7 +608,7 @@ bool RestCatalog::getTableMetadataImpl( headers.emplace_back("X-Iceberg-Access-Delegation", "vended-credentials"); } - const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / namespace_name / "tables" / table_name; + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / encodeNamespaceForURI(namespace_name) / "tables" / table_name; auto buf = createReadBuffer(config.prefix / endpoint, /* params */{}, headers); if (buf->eof()) From c9936c8c3ca4e6fb6c81bf451af3da395871969a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Dec 2025 23:14:00 +0000 Subject: [PATCH 072/113] Backport #92404 to 25.8: Enable sccache for rust if used --- contrib/corrosion-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index 7c82987dfdf6..abe5d9e01c42 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -99,7 +99,7 @@ if (USE_MUSL) set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -D_LIBCPP_HAS_MUSL_LIBC=1") endif () -if(CCACHE_EXECUTABLE MATCHES "/chcache$") +if(CCACHE_EXECUTABLE MATCHES "/chcache$" OR CCACHE_EXECUTABLE MATCHES "/sccache$") message(STATUS "Using RUSTC_WRAPPER: ${CCACHE_EXECUTABLE}") set(RUSTCWRAPPER "${CCACHE_EXECUTABLE}") else() From 34891ff57529f0afb17314f30a776cf198c9dbeb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Dec 2025 10:15:13 +0000 Subject: [PATCH 073/113] Backport #88860 to 25.8: Fix premature TTL column removal causing merge failures and wrong defaults --- src/Processors/Transforms/TTLTransform.cpp | 98 ++++++++++++++---- src/Processors/Transforms/TTLTransform.h | 13 +++ .../MergeTree/IMergedBlockOutputStream.cpp | 3 +- .../MergeTree/IMergedBlockOutputStream.h | 3 +- src/Storages/MergeTree/MergeTask.cpp | 99 +++++++++++++++---- src/Storages/MergeTree/MergeTask.h | 3 + .../MergeTree/MergedBlockOutputStream.cpp | 3 +- .../MergedColumnOnlyOutputStream.cpp | 8 +- src/Storages/MergeTree/MutateTask.cpp | 20 +++- .../test_graphite_merge_tree/test.py | 23 +++++ ...43_optimize_final_vertical_merge.reference | 2 +- ...2675_sparse_columns_clear_column.reference | 1 - .../03221_merge_profile_events.reference | 2 +- ...columns_substreams_in_wide_parts.reference | 8 -- .../03701_column_ttl_fully_expired.reference | 34 +++++++ .../03701_column_ttl_fully_expired.sql | 55 +++++++++++ 16 files changed, 315 insertions(+), 60 deletions(-) create mode 100644 tests/queries/0_stateless/03701_column_ttl_fully_expired.reference create mode 100644 tests/queries/0_stateless/03701_column_ttl_fully_expired.sql diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index cfbf97769183..f8c3ff8d1627 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -31,16 +31,36 @@ static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedS return {expr.expression, where_expr.expression}; } +SharedHeader TTLTransform::addExpiredColumnsToBlock(const SharedHeader & header, const NamesAndTypesList & expired_columns_) +{ + if (expired_columns_.empty()) + return header; + + auto output_block = *header; + + for (const auto & col : expired_columns_) + { + if (output_block.has(col.name)) + continue; + + output_block.insert({col.type->createColumn(), col.type, col.name}); + } + + return std::make_shared(std::move(output_block)); +} + TTLTransform::TTLTransform( const ContextPtr & context, SharedHeader header_, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::MutableDataPartPtr & data_part_, + const NamesAndTypesList & expired_columns_, time_t current_time_, bool force_) - : IAccumulatingTransform(header_, header_) + : IAccumulatingTransform(header_, addExpiredColumnsToBlock(header_, expired_columns_)) , data_part(data_part_) + , expired_columns(expired_columns_) , log(getLogger(storage_.getLogName() + " (TTLTransform)")) { auto old_ttl_infos = data_part->ttl_infos; @@ -71,32 +91,49 @@ TTLTransform::TTLTransform( old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, getInputPort().getHeader(), storage_)); - if (metadata_snapshot_->hasAnyColumnTTL()) + const auto & storage_columns = metadata_snapshot_->getColumns(); + const auto & column_defaults = storage_columns.getDefaults(); + + auto build_default_expr = [&](const String & name) { - const auto & storage_columns = metadata_snapshot_->getColumns(); - const auto & column_defaults = storage_columns.getDefaults(); + using Result = std::pair; + auto it = column_defaults.find(name); + if (it == column_defaults.end()) + return Result{}; + const auto & column = storage_columns.get(name); + auto default_ast = it->second.expression->clone(); + default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName()); + auto syntax_result = TreeRewriter(storage_.getContext()).analyze(default_ast, storage_columns.getAll()); + auto actions = ExpressionAnalyzer{default_ast, syntax_result, storage_.getContext()}.getActions(true); + return Result{actions, default_ast->getColumnName()}; + }; + + for (const auto & expired_column : expired_columns) + { + auto [default_expression, default_column_name] = build_default_expr(expired_column.name); + expired_columns_data.emplace( + expired_column.name, ExpiredColumnData{expired_column.type, std::move(default_expression), std::move(default_column_name)}); + } + if (metadata_snapshot_->hasAnyColumnTTL()) + { + auto expired_columns_map = expired_columns.getNameToTypeMap(); for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { - ExpressionActionsPtr default_expression; - String default_column_name; - auto it = column_defaults.find(name); - if (it != column_defaults.end()) + if (!expired_columns_map.contains(name)) { - const auto & column = storage_columns.get(name); - auto default_ast = it->second.expression->clone(); - default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName()); - - auto syntax_result - = TreeRewriter(storage_.getContext()).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical()); - default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.getContext()}.getActions(true); - default_column_name = default_ast->getColumnName(); + auto [default_expression, default_column_name] = build_default_expr(name); + algorithms.emplace_back(std::make_unique( + getExpressions(description, subqueries_for_sets, context), + description, + old_ttl_infos.columns_ttl[name], + current_time_, + force_, + name, + default_expression, + default_column_name, + isCompactPart(data_part))); } - - algorithms.emplace_back(std::make_unique( - getExpressions(description, subqueries_for_sets, context), description, - old_ttl_infos.columns_ttl[name], current_time_, - force_, name, default_expression, default_column_name, isCompactPart(data_part))); } } @@ -131,6 +168,25 @@ void TTLTransform::consume(Chunk chunk) convertToFullIfSparse(chunk); auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + /// Fill expired columns with default values which will later be handled in TTLColumnAlgorithm + for (const auto & [column, data] : expired_columns_data) + { + auto default_column + = ITTLAlgorithm::executeExpressionAndGetColumn(data.default_expression, block, data.default_column_name); + if (default_column) + default_column = default_column->convertToFullColumnIfConst(); + else + default_column = data.type->createColumnConstWithDefaultValue(block.rows())->convertToFullColumnIfConst(); + + /// Expired column may pre-exist (e.g. from customized merges like ReplacingMergeTree with version key), so + /// replace it with default instead of inserting a new one. + auto * c = block.findByName(column); + if (c) + c->column = default_column; + else + block.insert(ColumnWithTypeAndName(default_column, data.type, column)); + } + for (const auto & algorithm : algorithms) algorithm->execute(block); diff --git a/src/Processors/Transforms/TTLTransform.h b/src/Processors/Transforms/TTLTransform.h index 5a337e889614..0fed59119a86 100644 --- a/src/Processors/Transforms/TTLTransform.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -20,6 +20,7 @@ class TTLTransform : public IAccumulatingTransform const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::MutableDataPartPtr & data_part_, + const NamesAndTypesList & expired_columns_, time_t current_time, bool force_ ); @@ -30,6 +31,8 @@ class TTLTransform : public IAccumulatingTransform PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + static SharedHeader addExpiredColumnsToBlock(const SharedHeader & header, const NamesAndTypesList & expired_columns_); + protected: void consume(Chunk chunk) override; Chunk generate() override; @@ -46,6 +49,16 @@ class TTLTransform : public IAccumulatingTransform /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; + + NamesAndTypesList expired_columns; + + struct ExpiredColumnData + { + DataTypePtr type; + ExpressionActionsPtr default_expression; + String default_column_name; + }; + std::unordered_map expired_columns_data; LoggerPtr log; }; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index eb904a8e2ef7..bc71cfe2f79d 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -38,11 +38,10 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( const MergeTreeDataPartPtr & data_part, NamesAndTypesList & columns, + const NameSet & empty_columns, SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums) { - const NameSet & empty_columns = data_part->expired_columns; - /// For compact part we have to override whole file with data, it's not /// worth it if (empty_columns.empty() || isCompactPart(data_part)) diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index efc5784dd43c..7a9d23e627f0 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -46,11 +46,12 @@ class IMergedBlockOutputStream } protected: - /// Remove all columns marked expired in data_part. Also, clears checksums + /// Remove all columns in @empty_columns. Also, clears checksums /// and columns array. Return set of removed files names. NameSet removeEmptyColumnsFromPart( const MergeTreeDataPartPtr & data_part, NamesAndTypesList & columns, + const NameSet & empty_columns, SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index bab5d1786384..961d1769f5b3 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -266,7 +266,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu Names sort_key_columns_vec = sorting_key_expr->getRequiredColumns(); /// Collect columns used in the sorting key expressions. - std::set key_columns; + NameSet key_columns; auto storage_columns = global_ctx->storage_columns.getNameSet(); for (const auto & name : sort_key_columns_vec) { @@ -292,6 +292,15 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) key_columns.emplace(global_ctx->merging_params.sign_column); + /// Force all columns params of Graphite mode + if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Graphite) + { + key_columns.emplace(global_ctx->merging_params.graphite_params.path_column_name); + key_columns.emplace(global_ctx->merging_params.graphite_params.time_column_name); + key_columns.emplace(global_ctx->merging_params.graphite_params.value_column_name); + key_columns.emplace(global_ctx->merging_params.graphite_params.version_column_name); + } + /// Force to merge at least one column in case of empty key if (key_columns.empty()) key_columns.emplace(global_ctx->storage_columns.front().name); @@ -303,6 +312,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu key_columns.insert(minmax_columns.begin(), minmax_columns.end()); } + key_columns.insert(global_ctx->deduplicate_by_columns.begin(), global_ctx->deduplicate_by_columns.end()); + + /// Key columns required for merge, must not be expired early. + global_ctx->merge_required_key_columns = key_columns; + const auto & skip_indexes = global_ctx->metadata_snapshot->getSecondaryIndices(); for (const auto & index : skip_indexes) @@ -468,23 +482,37 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const const auto & patch_parts = global_ctx->future_part->patch_parts; - /// Skip fully expired columns manually, since in case of - /// need_remove_expired_values is not set, TTLTransform will not be used, - /// and columns that had been removed by TTL (via TTLColumnAlgorithm) will - /// be added again with default values. + /// Determine columns that are absent in all source parts—either fully expired or never written—and mark them as + /// expired to avoid unnecessary reads or writes during merges. + /// + /// NOTE: + /// Handling missing columns that have default expressions is non-trivial and currently unresolved + /// (see https://github.com/ClickHouse/ClickHouse/issues/91127). + /// For now, we conservatively avoid expiring such columns. /// - /// Also note, that it is better to do this here, since in other places it - /// will be too late (i.e. they will be written, and we will burn CPU/disk - /// resources for this). - if (!ctx->need_remove_expired_values) + /// The main challenges include: + /// 1. A default expression may depend on other columns, which themselves may be missing or expired, + /// making it unclear whether the default should be materialized or recomputed. + /// 2. Default expressions may introduce semantic changes if re-evaluated during merges, leading to + /// non-deterministic results across parts. { - for (auto & [column_name, ttl] : global_ctx->new_data_part->ttl_infos.columns_ttl) + NameSet columns_present_in_parts; + columns_present_in_parts.reserve(global_ctx->storage_columns.size()); + + /// Collect all column names that actually exist in the source parts + for (const auto & part : global_ctx->future_part->parts) { - if (ttl.finished()) - { - global_ctx->new_data_part->expired_columns.insert(column_name); - LOG_TRACE(ctx->log, "Adding expired column {} for part {}", column_name, global_ctx->new_data_part->name); - } + for (const auto & col : part->getColumns()) + columns_present_in_parts.emplace(col.name); + } + + const auto & columns_desc = global_ctx->metadata_snapshot->getColumns(); + + /// Any storage column not present in any part and without a default expression is considered expired + for (const auto & storage_column : global_ctx->storage_columns) + { + if (!columns_present_in_parts.contains(storage_column.name) && !columns_desc.getDefault(storage_column.name)) + global_ctx->new_data_part->expired_columns.emplace(storage_column.name); } } @@ -510,8 +538,27 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const if (!expired_columns.empty()) { global_ctx->gathering_columns = global_ctx->gathering_columns.eraseNames(expired_columns); - global_ctx->merging_columns = global_ctx->merging_columns.eraseNames(expired_columns); - global_ctx->storage_columns = global_ctx->storage_columns.eraseNames(expired_columns); + + auto filter_columns = [&](const NamesAndTypesList & input, NamesAndTypesList & expired_out) + { + NamesAndTypesList result; + for (const auto & column : input) + { + bool is_expired = expired_columns.contains(column.name); + bool is_required_for_merge = global_ctx->merge_required_key_columns.contains(column.name); + + if (is_expired) + expired_out.push_back(column); + + if (!is_expired || is_required_for_merge) + result.push_back(column); + } + + return result; + }; + + global_ctx->merging_columns = filter_columns(global_ctx->merging_columns, global_ctx->merging_columns_expired_by_ttl); + global_ctx->storage_columns = filter_columns(global_ctx->storage_columns, global_ctx->storage_columns_expired_by_ttl); } global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; @@ -612,6 +659,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const case MergeAlgorithm::Horizontal: { global_ctx->merging_columns = global_ctx->storage_columns; + global_ctx->merging_columns_expired_by_ttl = global_ctx->storage_columns_expired_by_ttl; global_ctx->merging_skip_indexes = global_ctx->metadata_snapshot->getSecondaryIndices(); global_ctx->gathering_columns.clear(); global_ctx->skip_indexes_by_column.clear(); @@ -1786,11 +1834,13 @@ class TTLStep : public ITransformingStep const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeData::MutableDataPartPtr & data_part_, + const NamesAndTypesList & expired_columns_, time_t current_time, bool force_) - : ITransformingStep(input_header_, input_header_, getTraits()) + : ITransformingStep(input_header_, TTLTransform::addExpiredColumnsToBlock(input_header_, expired_columns_), getTraits()) { - transform = std::make_shared(context_, input_header_, storage_, metadata_snapshot_, data_part_, current_time, force_); + transform = std::make_shared( + context_, input_header_, storage_, metadata_snapshot_, data_part_, expired_columns_, current_time, force_); subqueries_for_sets = transform->getSubqueries(); } @@ -2014,10 +2064,17 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const PreparedSets::Subqueries subqueries; /// TTL step - if (ctx->need_remove_expired_values) + if (ctx->need_remove_expired_values || !global_ctx->merging_columns_expired_by_ttl.empty()) { auto ttl_step = std::make_unique( - merge_parts_query_plan.getCurrentHeader(), global_ctx->context, *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); + merge_parts_query_plan.getCurrentHeader(), + global_ctx->context, + *global_ctx->data, + global_ctx->metadata_snapshot, + global_ctx->new_data_part, + global_ctx->merging_columns_expired_by_ttl, + global_ctx->time_of_merge, + ctx->force_ttl); subqueries = ttl_step->getSubqueries(); ttl_step->setStepDescription("TTL step"); merge_parts_query_plan.addStep(std::move(ttl_step)); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index b4f9618d8a3a..5df407ce0843 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -197,8 +197,11 @@ class MergeTask bool cleanup{false}; NamesAndTypesList gathering_columns{}; + NameSet merge_required_key_columns{}; NamesAndTypesList merging_columns{}; + NamesAndTypesList merging_columns_expired_by_ttl{}; NamesAndTypesList storage_columns{}; + NamesAndTypesList storage_columns_expired_by_ttl{}; MergeTreeData::DataPart::Checksums checksums_gathered_columns{}; ColumnsSubstreams gathered_columns_substreams{}; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 040c8c39934c..240380c7bd02 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -231,7 +231,8 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( auto serialization_infos = new_part->getSerializationInfos(); serialization_infos.replaceData(new_serialization_infos); - files_to_remove_after_sync = removeEmptyColumnsFromPart(new_part, part_columns, serialization_infos, checksums); + files_to_remove_after_sync + = removeEmptyColumnsFromPart(new_part, part_columns, new_part->expired_columns, serialization_infos, checksums); new_part->setColumns(part_columns, serialization_infos, metadata_snapshot->getMetadataVersion()); } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index e653ce21b5be..35cb644957d3 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -97,7 +97,13 @@ MergedColumnOnlyOutputStream::fillChecksums( auto serialization_infos = new_part->getSerializationInfos(); serialization_infos.replaceData(new_serialization_infos); - auto removed_files = removeEmptyColumnsFromPart(new_part, columns, serialization_infos, checksums); + NameSet empty_columns; + for (const auto & column : writer->getColumnsSample()) + { + if (new_part->expired_columns.contains(column.name)) + empty_columns.emplace(column.name); + } + auto removed_files = removeEmptyColumnsFromPart(new_part, columns, empty_columns, serialization_infos, checksums); for (const String & removed_file : removed_files) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 164e4715acf9..7ca173420a23 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1722,7 +1722,15 @@ class MutateAllPartColumnsTask : public IExecutableTask if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) { - auto transform = std::make_shared(ctx->context, builder->getSharedHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + auto transform = std::make_shared( + ctx->context, + builder->getSharedHeader(), + *ctx->data, + ctx->metadata_snapshot, + ctx->new_data_part, + NamesAndTypesList{} /*expired_columns*/, + ctx->time_of_mutation, + true); subqueries = transform->getSubqueries(); builder->addTransform(std::move(transform)); } @@ -1988,7 +1996,15 @@ class MutateSomePartColumnsTask : public IExecutableTask if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) { - auto transform = std::make_shared(ctx->context, builder->getSharedHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + auto transform = std::make_shared( + ctx->context, + builder->getSharedHeader(), + *ctx->data, + ctx->metadata_snapshot, + ctx->new_data_part, + NamesAndTypesList{} /*expired_columns*/, + ctx->time_of_mutation, + true); subqueries = transform->getSubqueries(); builder->addTransform(std::move(transform)); } diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index 1a55ee28aa15..5e74bea6e9d3 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -254,6 +254,8 @@ def test_paths_not_matching_any_pattern(graphite_table): def test_system_graphite_retentions(graphite_table): + # Avoid flakiness caused by concurrent test runs + q("DROP TABLE IF EXISTS test.graphite2") expected = """ graphite_rollup all \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] graphite_rollup all \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] @@ -512,3 +514,24 @@ def test_wrong_rollup_config(graphite_table): assert "Age and precision should only grow up: " in str(exc.value) assert "36000:600" in str(exc.value) assert "72000:300" in str(exc.value) + + +def test_ttl_version(graphite_table): + q( + """ +DROP TABLE IF EXISTS test.graphite; +CREATE TABLE test.graphite + ( + metric String, value Float64, timestamp UInt32, date Date, + updated UInt32 TTL date + INTERVAL 1 DAY) + ENGINE = GraphiteMergeTree('graphite_rollup') + PARTITION BY toYYYYMM(date) + ORDER BY (metric, timestamp) + SETTINGS index_granularity=8192; +""" + ) + + to_insert = "one_min.x1 100 1000000000 2001-09-09 1" + q("INSERT INTO test.graphite FORMAT TSV", to_insert) + q("OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL") + q("OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL") diff --git a/tests/queries/0_stateless/00443_optimize_final_vertical_merge.reference b/tests/queries/0_stateless/00443_optimize_final_vertical_merge.reference index bdc7c3527b4c..6de4a4aa6c90 100644 --- a/tests/queries/0_stateless/00443_optimize_final_vertical_merge.reference +++ b/tests/queries/0_stateless/00443_optimize_final_vertical_merge.reference @@ -1,3 +1,3 @@ 150000 150000 150000 150000 150000 150000 -[['def']] [['','']] +[['def']] [['zzz']] 0 diff --git a/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference b/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference index 56fa4a9ebea8..6b2fafc27d43 100644 --- a/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference +++ b/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference @@ -2,5 +2,4 @@ arr Default v Sparse arr Default arr Default -v Sparse 0 [] diff --git a/tests/queries/0_stateless/03221_merge_profile_events.reference b/tests/queries/0_stateless/03221_merge_profile_events.reference index 6da168ce064a..6558ad3e5811 100644 --- a/tests/queries/0_stateless/03221_merge_profile_events.reference +++ b/tests/queries/0_stateless/03221_merge_profile_events.reference @@ -1,3 +1,3 @@ Horizontal 1 20000 3 0 480000 1 1 1 1 1 1 Vertical 1 20000 1 2 480000 1 1 1 1 1 1 1 1 -Vertical 2 400000 2 6 12800000 1 1 1 1 1 1 1 1 1 1 1 1 +Vertical 2 400000 2 3 8000000 1 1 1 1 1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/03526_columns_substreams_in_wide_parts.reference b/tests/queries/0_stateless/03526_columns_substreams_in_wide_parts.reference index d76b2a9cc700..b16d596ad0ad 100644 --- a/tests/queries/0_stateless/03526_columns_substreams_in_wide_parts.reference +++ b/tests/queries/0_stateless/03526_columns_substreams_in_wide_parts.reference @@ -36,7 +36,6 @@ t Tuple(a UInt32, b Array(UInt32)) ['t%2Ea','t%2Eb.size0','t%2Eb'] ['t%2Ea','t%2 json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -x Array(UInt32) ['x.size0','x'] ['x.size0','x'] ------------------------------------------------------------------------- Alter drop column str String ['str'] ['str'] @@ -44,7 +43,6 @@ t Tuple(a UInt32, b Array(UInt32)) ['t%2Ea','t%2Eb.size0','t%2Eb'] ['t%2Ea','t%2 json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -x Array(UInt32) ['x.size0','x'] ['x.size0','x'] ------------------------------------------------------------------------- Alter rename column str String ['str'] ['str'] @@ -52,40 +50,34 @@ tt Tuple(a UInt32, b Array(UInt32)) ['tt%2Ea','tt%2Eb.size0','tt%2Eb'] ['tt%2Ea' json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -x Array(UInt32) ['x.size0','x'] ['x.size0','x'] ------------------------------------------------------------------------- Alter drop and rename column tt Tuple(a UInt32, b Array(UInt32)) ['tt%2Ea','tt%2Eb.size0','tt%2Eb'] ['tt%2Ea','tt%2Eb.size0','tt%2Eb'] json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -str Array(UInt32) ['str.size0','str'] ['str.size0','str'] ------------------------------------------------------------------------- Alter modify column tt Tuple(a UInt32, b Array(String), c UInt32) ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -str Array(UInt32) ['str.size0','str'] ['str.size0','str'] ------------------------------------------------------------------------- Alter update column tt Tuple(a UInt32, b Array(String), c UInt32) ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.a Array(UInt32) ['nested.size0','nested%2Ea'] ['nested.size0','nested%2Ea'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -str Array(UInt32) ['str.size0','str'] ['str.size0','str'] ------------------------------------------------------------------------- Alter rename nested column tt Tuple(a UInt32, b Array(String), c UInt32) ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.aa Array(UInt32) ['nested.size0','nested%2Eaa'] ['nested.size0','nested%2Eaa'] nested.b Array(UInt32) ['nested.size0','nested%2Eb'] ['nested.size0','nested%2Eb'] -str Array(UInt32) ['str.size0','str'] ['str.size0','str'] ------------------------------------------------------------------------- Alter rename all nested column tt Tuple(a UInt32, b Array(String), c UInt32) ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] ['tt%2Ea','tt%2Eb.size0','tt%2Eb','tt%2Ec'] json JSON(a UInt32, b Array(String)) ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] ['json.object_structure','json.d.dynamic_structure','json.d.variant_discr','json.e.dynamic_structure','json.e.variant_discr','json.f.dynamic_structure','json.f.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.variant_discr','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.dynamic_structure','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.variant_discr','json.a','json.b.size0','json.b','json.d.SharedVariant','json.d.String','json.e.Int64','json.e.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).size0','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.Int64','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).g.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64)).null','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.Array(Nullable(Int64))','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).k.SharedVariant','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data.size1','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Epaths','json.f.Array(JSON(max_dynamic_types=16, max_dynamic_paths=256)).object_shared_data%2Evalues','json.f.SharedVariant','json.object_shared_data.size0','json.object_shared_data%2Epaths','json.object_shared_data%2Evalues'] nested.aaa Array(UInt32) ['nested.size0','nested%2Eaaa'] ['nested.size0','nested%2Eaaa'] nested.bbb Array(UInt32) ['nested.size0','nested%2Ebbb'] ['nested.size0','nested%2Ebbb'] -str Array(UInt32) ['str.size0','str'] ['str.size0','str'] ------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/03701_column_ttl_fully_expired.reference b/tests/queries/0_stateless/03701_column_ttl_fully_expired.reference new file mode 100644 index 000000000000..de7596dc6e98 --- /dev/null +++ b/tests/queries/0_stateless/03701_column_ttl_fully_expired.reference @@ -0,0 +1,34 @@ +-- { echo ON } + +drop table if exists x; +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1), index idx(i) type set(100)) engine MergeTree partition by indexHint(dt) order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; +system stop merges x; +insert into x values (now() - toIntervalDay(30), 1); +select i from x where i = 1; +1 +system start merges x; +optimize table x final; +-- Run OPTIMIZE twice to ensure the second merge is triggered, as the issue occurs during the second merge phase. +optimize table x final; +select i from x where i = 42; +42 +drop table x; +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1)) engine ReplacingMergeTree(i) order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; +system stop merges x; +insert into x values (now() - toIntervalDay(30), 1); +select i from x where i = 1; +1 +system start merges x; +optimize table x final; +-- Run OPTIMIZE twice to ensure the second merge is triggered, as the "Not found column i in block" issue occurs during the second merge phase. +optimize table x final; +drop table x; +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1)) engine MergeTree order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; +system stop merges x; +insert into x values (now() - toIntervalDay(30), 1), (now() - toIntervalDay(30), 2); +system start merges x; +optimize table x final; -- Ensure that column TTL is applied +optimize table x final deduplicate by dt, i; +select i from x; +42 +drop table x; diff --git a/tests/queries/0_stateless/03701_column_ttl_fully_expired.sql b/tests/queries/0_stateless/03701_column_ttl_fully_expired.sql new file mode 100644 index 000000000000..979533ec2d8a --- /dev/null +++ b/tests/queries/0_stateless/03701_column_ttl_fully_expired.sql @@ -0,0 +1,55 @@ +-- { echo ON } + +drop table if exists x; + +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1), index idx(i) type set(100)) engine MergeTree partition by indexHint(dt) order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; + +system stop merges x; + +insert into x values (now() - toIntervalDay(30), 1); + +select i from x where i = 1; + +system start merges x; + +optimize table x final; + +-- Run OPTIMIZE twice to ensure the second merge is triggered, as the issue occurs during the second merge phase. +optimize table x final; + +select i from x where i = 42; + +drop table x; + +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1)) engine ReplacingMergeTree(i) order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; + +system stop merges x; + +insert into x values (now() - toIntervalDay(30), 1); + +select i from x where i = 1; + +system start merges x; + +optimize table x final; + +-- Run OPTIMIZE twice to ensure the second merge is triggered, as the "Not found column i in block" issue occurs during the second merge phase. +optimize table x final; + +drop table x; + +create table x (dt DateTime, i Int32 default 42 ttl dt + toIntervalDay(1)) engine MergeTree order by dt settings index_granularity = 8192, min_bytes_for_wide_part = 0; + +system stop merges x; + +insert into x values (now() - toIntervalDay(30), 1), (now() - toIntervalDay(30), 2); + +system start merges x; + +optimize table x final; -- Ensure that column TTL is applied + +optimize table x final deduplicate by dt, i; + +select i from x; + +drop table x; From ab5a7b992e3212d458b095139d9d3af1cb5692e2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Thu, 18 Dec 2025 13:09:12 +0100 Subject: [PATCH 074/113] Update ColumnObject.cpp --- src/Columns/ColumnObject.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 645a62a1e181..752c3e92587b 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2146,8 +2146,8 @@ void ColumnObject::repairDuplicatesInDynamicPathsAndSharedData(size_t offset) /// and we cannot repair it anyhow. Throw logical error exception in this case. else { - auto value = shared_data_values->getDataAt(j).toView(); - ReadBufferFromMemory buf(value); + auto value = shared_data_values->getDataAt(j); + ReadBufferFromMemory buf(value.data, value.size); auto type_from_shared_data = decodeDataType(buf); if (!isNothing(type_from_shared_data)) { From d1c60c0d9fd8491b5184368286453cd0c9506e25 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Dec 2025 12:20:33 +0000 Subject: [PATCH 075/113] Backport #89098 to 25.8: CI: Fix clickhouse version tweak calculation for the fork with sync --- ci/jobs/scripts/clickhouse_version.py | 58 ++++++++++++++++++++++++--- 1 file changed, 52 insertions(+), 6 deletions(-) diff --git a/ci/jobs/scripts/clickhouse_version.py b/ci/jobs/scripts/clickhouse_version.py index f6f403fe7352..9150928f9010 100644 --- a/ci/jobs/scripts/clickhouse_version.py +++ b/ci/jobs/scripts/clickhouse_version.py @@ -51,13 +51,54 @@ def get_current_version_as_dict(cls): version = cls.get_release_version_as_dict() info = Info() try: - tweak = int( - Shell.get_output( - f"git rev-list --count {version['githash']}..HEAD", verbose=True + # Check if the commit is directly on the first-parent chain + is_on_first_parent = Shell.get_output( + f"git rev-list --first-parent HEAD | grep -q {version['githash']} && echo 'yes' || echo 'no'", + verbose=True, + ).strip() + + if is_on_first_parent == "yes": + # Commit is directly on the first-parent chain (upstream scenario) + # Use simple first-parent counting + tweak = int( + Shell.get_output( + f"git rev-list --count --first-parent {version['githash']}..HEAD", + verbose=True, + ) ) - ) - except ValueError: - # Shallow checkout + else: + # Commit is not on first-parent chain (fork with sync scenario) + # Find the merge commit where this commit entered the first-parent chain + # Iterate backwards to find the last commit that has target as ancestor (the merge point) + merge_commit = Shell.get_output( + f"commits=$(git rev-list --first-parent HEAD); " + f"prev=''; " + f"for commit in $commits; do " + f"if git merge-base --is-ancestor {version['githash']} $commit 2>/dev/null; then " + f"prev=$commit; " + f"else " + f"echo $prev; break; " + f"fi; done", + verbose=True, + ).strip() + + if merge_commit: + tweak = int( + Shell.get_output( + f"git rev-list --count --first-parent {merge_commit}..HEAD", + verbose=True, + ) + ) + else: + # Fallback if we can't find the merge point + tweak = int( + Shell.get_output( + f"git rev-list --count --first-parent {version['githash']}..HEAD", + verbose=True, + ) + ) + except (ValueError, Exception): + # Shallow checkout or other error tweak = 1 version_type = "testing" if info.pr_number == 0 and bool( @@ -95,3 +136,8 @@ def get_release_sha(cls): @classmethod def store_version_data_in_ci_pipeline(cls): Info().store_kv_data("version", cls.get_current_version_as_dict()) + + +if __name__ == "__main__": + # test; + print(CHVersion.get_current_version_as_dict()) From a8f5fec70f299d08a0431da28f79cd2d63d60cb8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Thu, 18 Dec 2025 16:08:20 +0100 Subject: [PATCH 076/113] Update ColumnObject.cpp --- src/Columns/ColumnObject.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 752c3e92587b..f09ebbc98615 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2146,8 +2146,8 @@ void ColumnObject::repairDuplicatesInDynamicPathsAndSharedData(size_t offset) /// and we cannot repair it anyhow. Throw logical error exception in this case. else { - auto value = shared_data_values->getDataAt(j); - ReadBufferFromMemory buf(value.data, value.size); + auto value = shared_data_values->getDataAt(j).toView(); + ReadBufferFromMemory buf(value.data(), value.size()); auto type_from_shared_data = decodeDataType(buf); if (!isNothing(type_from_shared_data)) { From 4aad4d71ff61c3025baddada0350646dc418a1d2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Dec 2025 16:15:52 +0000 Subject: [PATCH 077/113] Backport #92500 to 25.8: Make query memory usage detection for spilling to disk during aggregation/sorting more robust --- src/Common/MemoryTrackerUtils.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryTrackerUtils.cpp b/src/Common/MemoryTrackerUtils.cpp index d590e6703aa7..96ce9022341c 100644 --- a/src/Common/MemoryTrackerUtils.cpp +++ b/src/Common/MemoryTrackerUtils.cpp @@ -50,8 +50,13 @@ std::optional getCurrentQueryHardLimit() Int64 getCurrentQueryMemoryUsage() { /// Use query-level memory tracker - if (auto * memory_tracker_child = DB::CurrentThread::getMemoryTracker()) - if (auto * memory_tracker = memory_tracker_child->getParent()) - return memory_tracker->get(); - return 0; + auto * thread_memory_tracker = DB::CurrentThread::getMemoryTracker(); + if (!thread_memory_tracker || thread_memory_tracker->level != VariableContext::Thread) + return 0; + + auto * query_process_memory_tracker = thread_memory_tracker->getParent(); + if (!query_process_memory_tracker || query_process_memory_tracker->level != VariableContext::Process) + return 0; + + return query_process_memory_tracker->get(); } From 2fd4d2bd92d758d0716e2257ac07c57d86b31bf3 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Thu, 18 Dec 2025 23:06:47 +0100 Subject: [PATCH 078/113] Update ColumnObject.cpp --- src/Columns/ColumnObject.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index f09ebbc98615..e3fcfc64f242 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -2123,7 +2123,7 @@ void ColumnObject::repairDuplicatesInDynamicPathsAndSharedData(size_t offset) size_t shared_data_end = shared_data_offsets[i]; for (size_t j = shared_data_start; j < shared_data_end; ++j) { - auto path = shared_data_paths->getDataAt(j); + auto path = shared_data_paths->getDataAt(j).toView(); auto it = dynamic_paths.find(path); if (it == dynamic_paths.end()) { From 0c6a8ddfbca199b46e2674d9aa9118be06c5253b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 19 Dec 2025 09:11:24 +0100 Subject: [PATCH 079/113] Revert "Backport #92404 to 25.8: Enable sccache for rust if used" --- contrib/corrosion-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index abe5d9e01c42..7c82987dfdf6 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -99,7 +99,7 @@ if (USE_MUSL) set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -D_LIBCPP_HAS_MUSL_LIBC=1") endif () -if(CCACHE_EXECUTABLE MATCHES "/chcache$" OR CCACHE_EXECUTABLE MATCHES "/sccache$") +if(CCACHE_EXECUTABLE MATCHES "/chcache$") message(STATUS "Using RUSTC_WRAPPER: ${CCACHE_EXECUTABLE}") set(RUSTCWRAPPER "${CCACHE_EXECUTABLE}") else() From 98841d74afdc8f36956a04ba23695325d963f958 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 19 Dec 2025 10:19:32 +0100 Subject: [PATCH 080/113] Backport part of ClickHouse/ClickHouse#85686 --- contrib/corrosion-cmake/CMakeLists.txt | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/contrib/corrosion-cmake/CMakeLists.txt b/contrib/corrosion-cmake/CMakeLists.txt index abe5d9e01c42..c4558be470bd 100644 --- a/contrib/corrosion-cmake/CMakeLists.txt +++ b/contrib/corrosion-cmake/CMakeLists.txt @@ -134,6 +134,16 @@ if (SANITIZE STREQUAL "memory") elseif (SANITIZE STREQUAL "thread") set(RUST_CARGO_BUILD_STD "-Zbuild-std=std,panic_abort,core,alloc") list(APPEND RUSTFLAGS "-Zsanitizer=thread") +elseif (SANITIZE STREQUAL "address") + set(RUST_CARGO_BUILD_STD "-Zbuild-std=std,panic_abort,core,alloc") + list(APPEND RUSTFLAGS "-Zsanitizer=address") +endif() + +# Set metadata flag to force different hashes for different sanitizers +if (SANITIZE STREQUAL "") + list(APPEND RUSTFLAGS "-C" "metadata=no-sanitizer") +else() + list(APPEND RUSTFLAGS "-C" "metadata=with-sanitizer-${SANITIZE}") endif() list(APPEND RUSTFLAGS From e53a0bcf6841196c69555872cbcc3f13b327e5be Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Dec 2025 14:14:53 +0000 Subject: [PATCH 081/113] Backport #92175 to 25.8: release request stream in insert select --- src/Interpreters/executeQuery.cpp | 2 ++ .../03760_keep_alive_insert_select.reference | 0 .../03760_keep_alive_insert_select.sh | 27 +++++++++++++++++++ 3 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/03760_keep_alive_insert_select.reference create mode 100755 tests/queries/0_stateless/03760_keep_alive_insert_select.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2a64d4883168..f3d0c19541e6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1336,6 +1336,8 @@ static BlockIO executeQueryImpl( input_storage.setPipe(std::move(pipe)); } } + + insert_query->tail.reset(); } else { diff --git a/tests/queries/0_stateless/03760_keep_alive_insert_select.reference b/tests/queries/0_stateless/03760_keep_alive_insert_select.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03760_keep_alive_insert_select.sh b/tests/queries/0_stateless/03760_keep_alive_insert_select.sh new file mode 100755 index 000000000000..5c8ef4fb5f91 --- /dev/null +++ b/tests/queries/0_stateless/03760_keep_alive_insert_select.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +CLICKHOUSE_URL="${CLICKHOUSE_URL}&http_wait_end_of_query=1" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' \ + -d 'DROP TABLE IF EXISTS insert_number_table' +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' \ + -d 'CREATE TABLE insert_number_table (record UInt32) Engine = Memory' + +query_id=$( + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&http_headers_progress_interval_ms=10&send_progress_in_http_headers=1" \ + -d 'INSERT INTO insert_number_table (record) SELECT number FROM system.numbers LIMIT 10' 2>&1 \ + | grep -F '< X-ClickHouse-Query-Id:' | sed 's/< X-ClickHouse-Query-Id: //' | tr -d '\n\t\r' | xargs +) + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" \ + -d "system flush logs text_log" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" \ + -d "SELECT message FROM system.text_log WHERE level='Error' AND query_id='${query_id}' AND message LIKE '%Request stream is shared by multiple threads. HTTP keep alive is not possible.%'" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' \ + -d 'DROP TABLE insert_number_table' From 3b5d46b153bda407ddf7b9f2500a672bb5e49161 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Dec 2025 14:17:00 +0000 Subject: [PATCH 082/113] Backport #92006 to 25.8: Cache schema only for 1 file in globs by default in schema inference --- src/Formats/ReadSchemaUtils.cpp | 17 ++++---- src/Formats/ReadSchemaUtils.h | 7 +--- .../ObjectStorage/ReadBufferIterator.cpp | 14 +------ .../ObjectStorage/ReadBufferIterator.h | 2 - src/Storages/StorageFile.cpp | 33 +-------------- src/Storages/StorageURL.cpp | 16 +------- tests/integration/test_storage_s3/test.py | 41 +++++++++++++++++++ ..._schema_cache_for_multiple_files.reference | 3 ++ ...53_file_schema_cache_for_multiple_files.sh | 17 ++++++++ 9 files changed, 73 insertions(+), 77 deletions(-) create mode 100644 tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.reference create mode 100755 tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.sh diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 4c4a61318aea..524c2e7b4223 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -166,10 +166,7 @@ try throw Exception(ErrorCodes::LOGICAL_ERROR, "Schema from cache was returned, but format name is unknown"); if (mode == SchemaInferenceMode::DEFAULT) - { - read_buffer_iterator.setResultingSchema(*iterator_data.cached_columns); return {*iterator_data.cached_columns, *format_name}; - } schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFilePath()); continue; @@ -258,12 +255,13 @@ try if (num_rows) read_buffer_iterator.setNumRowsToLastFile(*num_rows); + if (!names_and_types.empty()) + read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); + /// In default mode, we finish when schema is inferred successfully from any file. if (mode == SchemaInferenceMode::DEFAULT) break; - if (!names_and_types.empty()) - read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } catch (...) @@ -416,6 +414,9 @@ try read_buffer_iterator.setFormatName(*format_name); } + if (format_name) + read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); + if (mode == SchemaInferenceMode::UNION) { /// For UNION mode we need to know the schema of each file, @@ -424,7 +425,6 @@ try if (!format_name) throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually"); - read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types)); schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFilePath()); } @@ -527,10 +527,7 @@ try std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }), names_and_types.end()); - auto columns = ColumnsDescription(names_and_types); - if (mode == SchemaInferenceMode::DEFAULT) - read_buffer_iterator.setResultingSchema(columns); - return {columns, *format_name}; + return {ColumnsDescription(names_and_types), *format_name}; } throw Exception( diff --git a/src/Formats/ReadSchemaUtils.h b/src/Formats/ReadSchemaUtils.h index e2187d48245a..559f87a40bd2 100644 --- a/src/Formats/ReadSchemaUtils.h +++ b/src/Formats/ReadSchemaUtils.h @@ -46,14 +46,9 @@ struct IReadBufferIterator /// Used for caching number of rows from files metadata during schema inference. virtual void setNumRowsToLastFile(size_t /*num_rows*/) {} - /// Set schema inferred from last file. Used for UNION mode to cache schema - /// per file. + /// Set schema inferred from last file. virtual void setSchemaToLastFile(const ColumnsDescription & /*columns*/) {} - /// Set resulting inferred schema. Used for DEFAULT mode to cache schema - /// for all files. - virtual void setResultingSchema(const ColumnsDescription & /*columns*/) {} - /// Set auto detected format name. virtual void setFormatName(const String & /*format_name*/) {} diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 31089036abe0..26ac22012657 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -119,20 +119,8 @@ void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) { - if (query_settings.schema_inference_use_cache - && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) - { + if (query_settings.schema_inference_use_cache) schema_cache.addColumns(getKeyForSchemaCache(*current_object_info, *format), columns); - } -} - -void ReadBufferIterator::setResultingSchema(const ColumnsDescription & columns) -{ - if (query_settings.schema_inference_use_cache - && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - schema_cache.addManyColumns(getKeysForSchemaCache(), columns); - } } void ReadBufferIterator::setFormatName(const String & format_name) diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index e14aef3863af..197edee955dd 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -27,8 +27,6 @@ class ReadBufferIterator : public IReadBufferIterator, WithContext void setSchemaToLastFile(const ColumnsDescription & columns) override; - void setResultingSchema(const ColumnsDescription & columns) override; - String getLastFilePath() const override; void setFormatName(const String & format_name) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index d57ca4f996a6..917433d6c237 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -527,8 +527,7 @@ namespace void setSchemaToLastFile(const ColumnsDescription & columns) override { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::UNION) + if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file]) return; /// For union mode, schema can be different for different files, so we need to @@ -537,17 +536,6 @@ namespace StorageFile::getSchemaCache(getContext()).addColumns(cache_key, columns); } - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::DEFAULT) - return; - - /// For default mode we cache resulting schema for all paths. - auto cache_keys = getKeysForSchemaCache(paths, *format, format_settings, getContext()); - StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - String getLastFilePath() const override { if (current_index != 0) @@ -793,8 +781,7 @@ namespace void setSchemaToLastFile(const ColumnsDescription & columns) override { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::UNION) + if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file]) return; /// For union mode, schema can be different for different files in archive, so we need to @@ -804,22 +791,6 @@ namespace schema_cache.addColumns(cache_key, columns); } - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_file] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::DEFAULT) - return; - - /// For default mode we cache resulting schema for all paths. - /// Also add schema for initial paths (maybe with globes) in cache, - /// so next time we won't iterate through files (that can be expensive). - for (const auto & archive : archive_info.paths_to_archives) - paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info.path_in_archive)); - auto & schema_cache = StorageFile::getSchemaCache(getContext()); - auto cache_keys = getKeysForSchemaCache(paths_for_schema_cache, *format, format_settings, getContext()); - schema_cache.addManyColumns(cache_keys, columns); - } - void setFormatName(const String & format_name) override { format = format_name; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index c3cd88dd6014..275f34ef781b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -925,27 +925,13 @@ namespace void setSchemaToLastFile(const ColumnsDescription & columns) override { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_url] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::UNION) + if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_url]) return; auto key = getKeyForSchemaCache(current_url_option, *format, format_settings, getContext()); StorageURL::getSchemaCache(getContext()).addColumns(key, columns); } - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef()[Setting::schema_inference_use_cache_for_url] - || getContext()->getSettingsRef()[Setting::schema_inference_mode] != SchemaInferenceMode::DEFAULT) - return; - - for (const auto & options : url_options_to_check) - { - auto keys = getKeysForSchemaCache(options, *format, format_settings, getContext()); - StorageURL::getSchemaCache(getContext()).addManyColumns(keys, columns); - } - } - void setFormatName(const String & format_name) override { format = format_name; diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 53c68fa2e5fb..342f90dc79f5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2918,3 +2918,44 @@ def test_partition_by_without_wildcard(started_cluster): PARTITION BY (b, c) """ ) + +def test_schema_inference_cache_multi_path(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + s3_path_prefix = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_schema_infer_cache" + query1 = "insert into table function s3('{}/{}', 'Parquet', '{}') settings s3_truncate_on_insert=1 values {}".format( + s3_path_prefix, + "test1.parquet", + "column1 UInt32, column2 String", + "(1, 'a'), (2, 'b')", + ) + query2 = "insert into table function s3('{}/{}', 'Parquet', '{}') settings s3_truncate_on_insert=1 values {}".format( + s3_path_prefix, + "test2.parquet", + "column1 String, column2 UInt32", + "('a', 1), ('b', 2)", + ) + + run_query(instance, query1) + run_query(instance, query2) + + # Sleep so files last modification time is in the past + time.sleep(2) + + instance.query(f"DESCRIBE TABLE s3('{s3_path_prefix}/*')") + + assert "a\t1\nb\t2\n" == instance.query( + f"SELECT * FROM s3('{s3_path_prefix}/test2.parquet')" + ) + assert "1\ta\n2\tb\n" == instance.query( + f"SELECT * FROM s3('{s3_path_prefix}/test1.parquet')" + ) + + instance.query(f"DESCRIBE TABLE url('{s3_path_prefix}/{{test1.parquet,test2.parquet}}')") + + assert "a\t1\nb\t2\n" == instance.query( + f"SELECT * FROM url('{s3_path_prefix}/test2.parquet')" + ) + assert "1\ta\n2\tb\n" == instance.query( + f"SELECT * FROM url('{s3_path_prefix}/test1.parquet')" + ) diff --git a/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.reference b/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.reference new file mode 100644 index 000000000000..91384f6a9604 --- /dev/null +++ b/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.reference @@ -0,0 +1,3 @@ +0 +Hello 42 +42 Hello diff --git a/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.sh b/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.sh new file mode 100755 index 000000000000..d08ac59da013 --- /dev/null +++ b/tests/queries/0_stateless/03753_file_schema_cache_for_multiple_files.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select 'Hello' as c1, 42 as c2 format Parquet" > $CLICKHOUSE_TEST_UNIQUE_NAME.1.parquet +$CLICKHOUSE_LOCAL -q "select 42 as c1, 'Hello' as c2 format Parquet" > $CLICKHOUSE_TEST_UNIQUE_NAME.2.parquet +$CLICKHOUSE_LOCAL -m -q " +select sleepEachRow(2); +desc file('$CLICKHOUSE_TEST_UNIQUE_NAME.*.parquet') format Null; +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.1.parquet'); +select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.2.parquet'); +" + +rm $CLICKHOUSE_TEST_UNIQUE_NAME.* From 69d3042200903a40b3ff3ae7df5b1cb6bc78ca6f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Dec 2025 14:18:14 +0000 Subject: [PATCH 083/113] Backport #91711 to 25.8: Fix possible logical error in Log engine during subcolumns reading --- src/Storages/StorageLog.cpp | 73 +++++++++++++------ ...3_log_engine_shared_prefixes_bug.reference | 0 .../03753_log_engine_shared_prefixes_bug.sql | 7 ++ 3 files changed, 58 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference create mode 100644 tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 11c6f2555bfd..b17074b7eff5 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -155,6 +155,7 @@ class LogSource final : public ISource using DeserializeStates = std::map; DeserializeStates deserialize_states; + void readPrefix(const NameAndTypePair & name_and_type, ISerialization::SubstreamsCache & cache, ISerialization::SubstreamsDeserializeStatesCache & deserialize_state_cache); void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache); bool isFinished(); }; @@ -190,8 +191,17 @@ Chunk LogSource::generate() /// How many rows to read for the next block. size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read); std::unordered_map caches; + std::unordered_map deserialize_states_caches; Block res; + /// First, read prefixes for all columns/subcolumns. + for (const auto & name_and_type : columns) + { + auto name_and_type_on_disk = getColumnOnDisk(name_and_type); + readPrefix(name_and_type_on_disk, caches[name_and_type_on_disk.getNameInStorage()], deserialize_states_caches[name_and_type_on_disk.getNameInStorage()]); + } + + /// Second, read the data of all columns/subcolumns. for (const auto & name_type : columns) { ColumnPtr column; @@ -230,6 +240,35 @@ Chunk LogSource::generate() return Chunk(res.getColumns(), num_rows); } +void LogSource::readPrefix(const NameAndTypePair & name_and_type, ISerialization::SubstreamsCache & cache, ISerialization::SubstreamsDeserializeStatesCache & deserialize_state_cache) +{ + if (deserialize_states.contains(name_and_type.name)) + return; + + auto serialization = IDataType::getSerialization(name_and_type); + + ISerialization::DeserializeBinaryBulkSettings settings; + settings.getter = [&](const ISerialization::SubstreamPath & path) -> ReadBuffer * + { + if (cache.contains(ISerialization::getSubcolumnNameForStream(path))) + return nullptr; + + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path, {}); + + const auto & data_file_it = storage.data_files_by_names.find(data_file_name); + if (data_file_it == storage.data_files_by_names.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name); + const auto & data_file = *data_file_it->second; + + size_t offset = 0; + size_t file_size = file_sizes[data_file.index]; + + auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, file_size, limited_by_file_sizes, read_settings).first; + return &it->second.compressed.value(); + }; + + serialization->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name_and_type.name], &deserialize_state_cache); +} void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache) @@ -238,35 +277,25 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu const auto & [name, type] = name_and_type; auto serialization = IDataType::getSerialization(name_and_type); - auto create_stream_getter = [&](bool stream_for_prefix) + settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * { - return [&, stream_for_prefix] (const ISerialization::SubstreamPath & path) -> ReadBuffer * - { - if (cache.contains(ISerialization::getSubcolumnNameForStream(path))) - return nullptr; + if (cache.contains(ISerialization::getSubcolumnNameForStream(path))) + return nullptr; - String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); - const auto & data_file_it = storage.data_files_by_names.find(data_file_name); - if (data_file_it == storage.data_files_by_names.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name); - const auto & data_file = *data_file_it->second; + const auto & data_file_it = storage.data_files_by_names.find(data_file_name); + if (data_file_it == storage.data_files_by_names.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name); + const auto & data_file = *data_file_it->second; - size_t offset = stream_for_prefix ? 0 : offsets[data_file.index]; - size_t file_size = file_sizes[data_file.index]; + size_t offset = offsets[data_file.index]; + size_t file_size = file_sizes[data_file.index]; - auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, file_size, limited_by_file_sizes, read_settings).first; - return &it->second.compressed.value(); - }; + auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, file_size, limited_by_file_sizes, read_settings).first; + return &it->second.compressed.value(); }; - if (!deserialize_states.contains(name)) - { - settings.getter = create_stream_getter(true); - serialization->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name], nullptr); - } - - settings.getter = create_stream_getter(false); serialization->deserializeBinaryBulkWithMultipleStreams(column, 0, max_rows_to_read, settings, deserialize_states[name], &cache); } diff --git a/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference b/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql b/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql new file mode 100644 index 000000000000..c4e3176aa286 --- /dev/null +++ b/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (c String) ENGINE = Log; +INSERT INTO TABLE test SELECT randomString(10) from numbers(1000); +INSERT INTO TABLE test SELECT randomString(10) from numbers(1000); +SELECT * FROM test ORDER BY c, c.size FORMAT Null; +DROP TABLE test; + From 9f5092ae4c7c5faede91cdc4d11749ce7922ee25 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Dec 2025 14:20:43 +0000 Subject: [PATCH 084/113] Backport #87646 to 25.8: Add MergeTree setting to limit number of dynamic subcolumns in Wide part after merge --- src/Columns/ColumnArray.cpp | 4 +-- src/Columns/ColumnArray.h | 2 +- src/Columns/ColumnBLOB.h | 2 +- src/Columns/ColumnCompressed.h | 2 +- src/Columns/ColumnDynamic.cpp | 8 ++--- src/Columns/ColumnDynamic.h | 2 +- src/Columns/ColumnMap.cpp | 4 +-- src/Columns/ColumnMap.h | 2 +- src/Columns/ColumnNullable.cpp | 4 +-- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnObject.cpp | 9 ++--- src/Columns/ColumnObject.h | 2 +- src/Columns/ColumnSparse.cpp | 4 +-- src/Columns/ColumnSparse.h | 2 +- src/Columns/ColumnTuple.cpp | 4 +-- src/Columns/ColumnTuple.h | 2 +- src/Columns/ColumnVariant.cpp | 4 +-- src/Columns/ColumnVariant.h | 2 +- src/Columns/IColumn.h | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + src/Core/SettingsFields.h | 1 + src/Interpreters/SortedBlocksWriter.cpp | 4 +++ .../Merges/AggregatingSortedTransform.h | 6 ++-- .../Algorithms/AggregatingSortedAlgorithm.cpp | 8 +++-- .../Algorithms/AggregatingSortedAlgorithm.h | 4 ++- .../Algorithms/CollapsingSortedAlgorithm.cpp | 3 +- .../Algorithms/CollapsingSortedAlgorithm.h | 1 + .../GraphiteRollupSortedAlgorithm.cpp | 3 +- .../GraphiteRollupSortedAlgorithm.h | 1 + .../Merges/Algorithms/MergedData.cpp | 2 +- src/Processors/Merges/Algorithms/MergedData.h | 5 +-- .../Algorithms/MergingSortedAlgorithm.cpp | 3 +- .../Algorithms/MergingSortedAlgorithm.h | 1 + .../Algorithms/ReplacingSortedAlgorithm.cpp | 3 +- .../Algorithms/ReplacingSortedAlgorithm.h | 1 + .../Algorithms/SummingSortedAlgorithm.cpp | 7 ++-- .../Algorithms/SummingSortedAlgorithm.h | 3 +- .../VersionedCollapsingAlgorithm.cpp | 3 +- .../Algorithms/VersionedCollapsingAlgorithm.h | 1 + .../Merges/CoalescingSortedTransform.h | 4 ++- .../Merges/CollapsingSortedTransform.h | 2 ++ .../Merges/GraphiteRollupSortedTransform.h | 2 ++ .../Merges/MergingSortedTransform.cpp | 2 ++ .../Merges/MergingSortedTransform.h | 1 + .../Merges/ReplacingSortedTransform.h | 2 ++ .../Merges/SummingSortedTransform.h | 4 ++- .../Merges/VersionedCollapsingTransform.h | 2 ++ .../QueryPlan/ReadFromMergeTree.cpp | 20 ++++++----- src/Processors/QueryPlan/SortingStep.cpp | 2 ++ .../Transforms/ColumnGathererTransform.cpp | 7 ++-- .../Transforms/ColumnGathererTransform.h | 3 ++ .../Transforms/MergeSortingTransform.cpp | 3 +- .../gtest_blocks_size_merging_streams.cpp | 30 +++++++++++++--- src/Storages/MergeTree/MergeTask.cpp | 34 +++++++++++++++---- .../MergeTree/MergeTreeDataWriter.cpp | 14 ++++---- src/Storages/MergeTree/MergeTreeSettings.cpp | 7 ++++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + ..._dynamic_subcolumns_in_wide_part.reference | 4 +++ ...ge_max_dynamic_subcolumns_in_wide_part.sql | 16 +++++++++ 59 files changed, 203 insertions(+), 81 deletions(-) create mode 100644 tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.reference create mode 100644 tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.sql diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 94fb151f05ab..216aa7f4dc17 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1377,14 +1377,14 @@ size_t ColumnArray::getNumberOfDimensions() const return 1 + nested_array->getNumberOfDimensions(); /// Every modern C++ compiler optimizes tail recursion. } -void ColumnArray::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnArray::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { Columns nested_source_columns; nested_source_columns.reserve(source_columns.size()); for (const auto & source_column : source_columns) nested_source_columns.push_back(assert_cast(*source_column).getDataPtr()); - data->takeDynamicStructureFromSourceColumns(nested_source_columns); + data->takeDynamicStructureFromSourceColumns(nested_source_columns, max_dynamic_subcolumns); } void ColumnArray::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index cd67d7157fbf..854c4504442e 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -210,7 +210,7 @@ class ColumnArray final : public COWHelper, ColumnArr size_t getNumberOfDimensions() const; bool hasDynamicStructure() const override { return getData().hasDynamicStructure(); } - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override { data->fixDynamicStructure(); } diff --git a/src/Columns/ColumnBLOB.h b/src/Columns/ColumnBLOB.h index 010560eb074e..770f45720245 100644 --- a/src/Columns/ColumnBLOB.h +++ b/src/Columns/ColumnBLOB.h @@ -201,7 +201,7 @@ class ColumnBLOB : public COWHelper, ColumnBLOB> void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwInapplicable(); } bool hasDynamicStructure() const override { throwInapplicable(); } - void takeDynamicStructureFromSourceColumns(const Columns &) override { throwInapplicable(); } + void takeDynamicStructureFromSourceColumns(const Columns &, std::optional) override { throwInapplicable(); } void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwInapplicable(); } void fixDynamicStructure() override { throwInapplicable(); } diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index d2ea123e6521..3bc0ffb16149 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -135,7 +135,7 @@ class ColumnCompressed : public COWHelper, Colum void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeDecompressed(); } bool hasDynamicStructure() const override { throwMustBeDecompressed(); } - void takeDynamicStructureFromSourceColumns(const Columns &) override { throwMustBeDecompressed(); } + void takeDynamicStructureFromSourceColumns(const Columns &, std::optional) override { throwMustBeDecompressed(); } void takeDynamicStructureFromColumn(const ColumnPtr &) override { throwMustBeDecompressed(); } void fixDynamicStructure() override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 93dd909bee60..2f012c37e9fa 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -1258,7 +1258,7 @@ bool ColumnDynamic::dynamicStructureEquals(const IColumn & rhs) const return false; } -void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Dynamic column"); @@ -1331,8 +1331,8 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source DataTypePtr result_variant_type; Statistics new_statistics(Statistics::Source::MERGE); - /// Reset max_dynamic_types to global_max_dynamic_types. - max_dynamic_types = global_max_dynamic_types; + /// Reset max_dynamic_types to global_max_dynamic_types or max_dynamic_subcolumns if set. + max_dynamic_types = max_dynamic_subcolumns ? std::min(*max_dynamic_subcolumns, global_max_dynamic_types) : global_max_dynamic_types; /// Check if the number of all dynamic types exceeds the limit. if (!canAddNewVariants(0, all_variants.size())) { @@ -1404,7 +1404,7 @@ void ColumnDynamic::takeDynamicStructureFromSourceColumns(const Columns & source auto & variant_col = getVariantColumn(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) - variant_col.getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i]); + variant_col.getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i], max_dynamic_subcolumns); } void ColumnDynamic::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index fede36a53819..4e1bbe3232f1 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -390,7 +390,7 @@ class ColumnDynamic final : public COWHelper, Colum bool hasDynamicStructure() const override { return true; } bool dynamicStructureEquals(const IColumn & rhs) const override; - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 9472985ddd01..99f95505c237 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -432,13 +432,13 @@ ColumnPtr ColumnMap::compress(bool force_compression) const }); } -void ColumnMap::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnMap::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { Columns nested_source_columns; nested_source_columns.reserve(source_columns.size()); for (const auto & source_column : source_columns) nested_source_columns.push_back(assert_cast(*source_column).getNestedColumnPtr()); - nested->takeDynamicStructureFromSourceColumns(nested_source_columns); + nested->takeDynamicStructureFromSourceColumns(nested_source_columns, max_dynamic_subcolumns); } void ColumnMap::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 222647d5710c..324d62ce6922 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -124,7 +124,7 @@ class ColumnMap final : public COWHelper, ColumnMap> bool hasDynamicStructure() const override { return nested->hasDynamicStructure(); } bool dynamicStructureEquals(const IColumn & rhs) const override; - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override { nested->fixDynamicStructure(); } }; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 109d822d041d..7c1047b29abb 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -965,13 +965,13 @@ ColumnPtr ColumnNullable::getNestedColumnWithDefaultOnNull() const return res; } -void ColumnNullable::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnNullable::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { Columns nested_source_columns; nested_source_columns.reserve(source_columns.size()); for (const auto & source_column : source_columns) nested_source_columns.push_back(assert_cast(*source_column).getNestedColumnPtr()); - nested_column->takeDynamicStructureFromSourceColumns(nested_source_columns); + nested_column->takeDynamicStructureFromSourceColumns(nested_source_columns, max_dynamic_subcolumns); } void ColumnNullable::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index d1d16ba63acc..e9c114f63ac3 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -229,7 +229,7 @@ class ColumnNullable final : public COWHelper, Col void checkConsistency() const; bool hasDynamicStructure() const override { return nested_column->hasDynamicStructure(); } - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override { nested_column->fixDynamicStructure(); } bool dynamicStructureEquals(const IColumn & rhs) const override; diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e3fcfc64f242..b93ca0987b56 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1660,7 +1660,7 @@ bool ColumnObject::dynamicStructureEquals(const IColumn & rhs) const return true; } -void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & source_columns) +void ColumnObject::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { if (!empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "takeDynamicStructureFromSourceColumns should be called only on empty Object column"); @@ -1718,7 +1718,8 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou dynamic_paths.clear(); dynamic_paths_ptrs.clear(); sorted_dynamic_paths.clear(); - max_dynamic_paths = global_max_dynamic_paths; + /// If max_dynamic_subcolumns is set, use it as max number of paths. + max_dynamic_paths = max_dynamic_subcolumns ? std::min(*max_dynamic_subcolumns, global_max_dynamic_paths) : global_max_dynamic_paths; Statistics new_statistics(Statistics::Source::MERGE); /// Check if the number of all dynamic paths exceeds the limit. @@ -1782,7 +1783,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou if (it != source_object.dynamic_paths.end()) dynamic_path_source_columns.push_back(it->second); } - column->takeDynamicStructureFromSourceColumns(dynamic_path_source_columns); + column->takeDynamicStructureFromSourceColumns(dynamic_path_source_columns, max_dynamic_subcolumns); } /// Typed paths also can contain types with dynamic structure. @@ -1792,7 +1793,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou typed_path_source_columns.reserve(source_columns.size()); for (const auto & source_column : source_columns) typed_path_source_columns.push_back(assert_cast(*source_column).typed_paths.at(path)); - column->takeDynamicStructureFromSourceColumns(typed_path_source_columns); + column->takeDynamicStructureFromSourceColumns(typed_path_source_columns, max_dynamic_subcolumns); } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 9cda2d2bb6b0..fc1426a3876e 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -202,7 +202,7 @@ class ColumnObject final : public COWHelper, ColumnO bool hasDynamicStructure() const override { return true; } bool dynamicStructureEquals(const IColumn & rhs) const override; - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index e963488a3468..4ecdd94f5070 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -876,13 +876,13 @@ ColumnSparse::Iterator ColumnSparse::getIterator(size_t n) const return Iterator(offsets_data, _size, current_offset, n); } -void ColumnSparse::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnSparse::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { Columns values_source_columns; values_source_columns.reserve(source_columns.size()); for (const auto & source_column : source_columns) values_source_columns.push_back(assert_cast(*source_column).getValuesPtr()); - values->takeDynamicStructureFromSourceColumns(values_source_columns); + values->takeDynamicStructureFromSourceColumns(values_source_columns, max_dynamic_subcolumns); } void ColumnSparse::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index b1dacdb45d42..8a1ab670998e 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -169,7 +169,7 @@ class ColumnSparse final : public COWHelper, ColumnS bool isCollationSupported() const override { return values->isCollationSupported(); } bool hasDynamicStructure() const override { return values->hasDynamicStructure(); } - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override { values->fixDynamicStructure(); } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 1e02851710bb..84696add5cd4 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -845,7 +845,7 @@ bool ColumnTuple::dynamicStructureEquals(const IColumn & rhs) const } } -void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { std::vector nested_source_columns; nested_source_columns.resize(columns.size()); @@ -860,7 +860,7 @@ void ColumnTuple::takeDynamicStructureFromSourceColumns(const Columns & source_c } for (size_t i = 0; i != columns.size(); ++i) - columns[i]->takeDynamicStructureFromSourceColumns(nested_source_columns[i]); + columns[i]->takeDynamicStructureFromSourceColumns(nested_source_columns[i], max_dynamic_subcolumns); } void ColumnTuple::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 5af44dc9cf43..70bb00fbd6a0 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -146,7 +146,7 @@ class ColumnTuple final : public COWHelper, ColumnTup bool hasDynamicStructure() const override; bool dynamicStructureEquals(const IColumn & rhs) const override; - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index 17826bbd09f9..fc3f00915eb8 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1726,7 +1726,7 @@ bool ColumnVariant::hasDynamicStructure() const return false; } -void ColumnVariant::takeDynamicStructureFromSourceColumns(const Columns & source_columns) +void ColumnVariant::takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) { /// List of source columns for each variant. In global order. std::vector variants_source_columns; @@ -1743,7 +1743,7 @@ void ColumnVariant::takeDynamicStructureFromSourceColumns(const Columns & source } for (size_t i = 0; i != num_variants; ++i) - getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i]); + getVariantByGlobalDiscriminator(i).takeDynamicStructureFromSourceColumns(variants_source_columns[i], max_dynamic_subcolumns); } void ColumnVariant::takeDynamicStructureFromColumn(const ColumnPtr & source_column) diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index 6ebebf56440a..a1bdbe7d4039 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -343,7 +343,7 @@ class ColumnVariant final : public COWHelper, Colum bool hasDynamicStructure() const override; bool dynamicStructureEquals(const IColumn & rhs) const override; - void takeDynamicStructureFromSourceColumns(const Columns & source_columns) override; + void takeDynamicStructureFromSourceColumns(const Columns & source_columns, std::optional max_dynamic_subcolumns) override; void takeDynamicStructureFromColumn(const ColumnPtr & source_column) override; void fixDynamicStructure() override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index ce5c00f0bd8a..ce5d349ecdbf 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -667,7 +667,7 @@ class IColumn : public COW [[nodiscard]] virtual bool dynamicStructureEquals(const IColumn & rhs) const { return structureEquals(rhs); } /// For columns with dynamic subcolumns this method takes dynamic structure from source columns /// and creates proper resulting dynamic structure in advance for merge of these source columns. - virtual void takeDynamicStructureFromSourceColumns(const std::vector & /*source_columns*/) {} + virtual void takeDynamicStructureFromSourceColumns(const std::vector & /*source_columns*/, std::optional /*max_dynamic_subcolumns*/) {} /// For columns with dynamic subcolumns this method takes the exact dynamic structure from provided column. virtual void takeDynamicStructureFromColumn(const ColumnPtr & /*source_column*/) {} /// For columns with dynamic subcolumns fix current dynamic structure so later inserts into this column won't change it. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index eaa78f0a3661..50f381cd6edb 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -855,6 +855,7 @@ const VersionToSettingsChangesMap & getMergeTreeSettingsChangesHistory() { addSettingsChanges(merge_tree_settings_changes_history, "25.8", { + {"merge_max_dynamic_subcolumns_in_wide_part", "auto", "auto", "Add a new setting to limit number of dynamic subcolumns in Wide part after merge regardless the parameters specified in the data type"}, {"object_serialization_version", "v2", "v2", "Add a setting to control JSON serialization versions"}, {"object_shared_data_serialization_version", "map", "map", "Add a setting to control JSON serialization versions"}, {"object_shared_data_serialization_version_for_zero_level_parts", "map", "map", "Add a setting to control JSON serialization versions for zero level parts"}, diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 273c5e1e9287..d3c54154ddd5 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -121,6 +121,7 @@ struct SettingAutoWrapper void readBinary(ReadBuffer & in) { changed = true; is_auto = false; base.readBinary(in); } Type valueOr(Type default_value) const { return is_auto ? default_value : base.value; } + std::optional valueOrNullopt() const { return is_auto ? std::optional(std::nullopt) : base.value; } }; using SettingFieldBoolAuto = SettingAutoWrapper; diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 5e4eae2f66d7..9f798a295d1a 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -176,6 +176,7 @@ TemporaryBlockStreamHolder SortedBlocksWriter::flush(const BlocksList & blocks) sort_description, rows_in_block, /*max_block_size_bytes=*/0, + /*max_dynamic_subcolumns=*/std::nullopt, SortingQueueStrategy::Default); pipeline.addTransform(std::move(transform)); @@ -268,6 +269,7 @@ SortedBlocksWriter::PremergedFiles SortedBlocksWriter::premerge() sort_description, rows_in_block, /*max_block_size_bytes=*/0, + /*max_dynamic_subcolumns=*/std::nullopt, SortingQueueStrategy::Default); pipeline.addTransform(std::move(transform)); @@ -303,6 +305,7 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function max_dynamic_subcolumns_) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, header, num_inputs, std::move(description_), max_block_size_rows, - max_block_size_bytes) + max_block_size_bytes, + max_dynamic_subcolumns_) { } diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index 958f3303570b..47ae4eff10ed 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -141,8 +141,9 @@ AggregatingSortedAlgorithm::SimpleAggregateDescription::~SimpleAggregateDescript AggregatingSortedAlgorithm::AggregatingMergedData::AggregatingMergedData( UInt64 max_block_size_rows_, UInt64 max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, ColumnsDefinition & def_) - : MergedData(false, max_block_size_rows_, max_block_size_bytes_), def(def_) + : MergedData(false, max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_), def(def_) { } @@ -257,10 +258,11 @@ AggregatingSortedAlgorithm::AggregatingSortedAlgorithm( size_t num_inputs, SortDescription description_, size_t max_block_size_rows_, - size_t max_block_size_bytes_) + size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_) : IMergingAlgorithmWithDelayedChunk(header_, num_inputs, description_) , columns_definition(defineColumns(*header_, description_)) - , merged_data(max_block_size_rows_, max_block_size_bytes_, columns_definition) + , merged_data(max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_, columns_definition) { } diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h index 6dd821bd778d..5786812d0b47 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.h @@ -24,7 +24,8 @@ class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChun size_t num_inputs, SortDescription description_, size_t max_block_size_rows_, - size_t max_block_size_bytes_); + size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_); const char * getName() const override { return "AggregatingSortedAlgorithm"; } void initialize(Inputs inputs) override; @@ -112,6 +113,7 @@ class AggregatingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChun AggregatingMergedData( UInt64 max_block_size_rows_, UInt64 max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, ColumnsDefinition & def_); void initialize(const Block & header, const IMergingAlgorithm::Inputs & inputs) override; diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 68efd6673c2c..17322a56edac 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -29,6 +29,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( bool only_positive_sign_, size_t max_block_size_rows_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, LoggerPtr log_, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, @@ -39,7 +40,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( std::move(description_), out_row_sources_buf_, max_row_refs, - std::make_unique(use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_)) + std::make_unique(use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_)) , sign_column_number(header_->getPositionByName(sign_column)) , only_positive_sign(only_positive_sign_) , throw_if_invalid_sign(throw_if_invalid_sign_) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 72486c996aa9..80ddc3017b3d 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -34,6 +34,7 @@ class CollapsingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks bool only_positive_sign_, /// For select final. Skip rows with sum(sign) < 0. size_t max_block_size_rows_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, LoggerPtr log_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false, diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index e9c7d87c619e..866e21a2e3bf 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -44,9 +44,10 @@ GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm( SortDescription description_, size_t max_block_size_rows_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, Graphite::Params params_, time_t time_of_merge_) - : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), nullptr, max_row_refs, std::make_unique(false, max_block_size_rows_, max_block_size_bytes_)) + : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), nullptr, max_row_refs, std::make_unique(false, max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_)) , graphite_rollup_merged_data(assert_cast(*merged_data)) , params(std::move(params_)) , time_of_merge(time_of_merge_) diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index 27ea23216e4f..05c7a68c17d6 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -27,6 +27,7 @@ class GraphiteRollupSortedAlgorithm final : public IMergingAlgorithmWithSharedCh SortDescription description_, size_t max_block_size_rows_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, Graphite::Params params_, time_t time_of_merge_); diff --git a/src/Processors/Merges/Algorithms/MergedData.cpp b/src/Processors/Merges/Algorithms/MergedData.cpp index 61681632eef5..9167661b1f55 100644 --- a/src/Processors/Merges/Algorithms/MergedData.cpp +++ b/src/Processors/Merges/Algorithms/MergedData.cpp @@ -31,7 +31,7 @@ void MergedData::initialize(const Block & header, const IMergingAlgorithm::Input /// Sometimes header can contain Sparse columns, we don't support Sparse in merge algorithms. columns[i] = recursiveRemoveSparse(std::move(columns[i]))->assumeMutable(); if (columns[i]->hasDynamicStructure()) - columns[i]->takeDynamicStructureFromSourceColumns(source_columns[i]); + columns[i]->takeDynamicStructureFromSourceColumns(source_columns[i], max_dynamic_subcolumns); } } diff --git a/src/Processors/Merges/Algorithms/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h index 2561c1e19aea..59bca3262501 100644 --- a/src/Processors/Merges/Algorithms/MergedData.h +++ b/src/Processors/Merges/Algorithms/MergedData.h @@ -13,8 +13,8 @@ class Block; class MergedData { public: - explicit MergedData(bool use_average_block_size_, UInt64 max_block_size_, UInt64 max_block_size_bytes_) - : max_block_size(max_block_size_), max_block_size_bytes(max_block_size_bytes_), use_average_block_size(use_average_block_size_) + explicit MergedData(bool use_average_block_size_, UInt64 max_block_size_, UInt64 max_block_size_bytes_, std::optional max_dynamic_subcolumns_) + : max_block_size(max_block_size_), max_block_size_bytes(max_block_size_bytes_), use_average_block_size(use_average_block_size_), max_dynamic_subcolumns(max_dynamic_subcolumns_) { } @@ -55,6 +55,7 @@ class MergedData const UInt64 max_block_size = 0; const UInt64 max_block_size_bytes = 0; const bool use_average_block_size = false; + const std::optional max_dynamic_subcolumns; bool need_flush = false; }; diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 60116994df45..c9e869b18729 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -14,13 +14,14 @@ MergingSortedAlgorithm::MergingSortedAlgorithm( const SortDescription & description_, size_t max_block_size_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, bool apply_virtual_row_conversions_) : header(std::move(header_)) - , merged_data(use_average_block_sizes, max_block_size_, max_block_size_bytes_) + , merged_data(use_average_block_sizes, max_block_size_, max_block_size_bytes_, max_dynamic_subcolumns_) , description(description_) , limit(limit_) , out_row_sources_buf(out_row_sources_buf_) diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index d9e73472ac79..511dac4cfee9 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -20,6 +20,7 @@ class MergingSortedAlgorithm final : public IMergingAlgorithm const SortDescription & description_, size_t max_block_size_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, SortingQueueStrategy sorting_queue_strategy_, UInt64 limit_ = 0, WriteBuffer * out_row_sources_buf_ = nullptr, diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 4fcb74660e97..9dff38c02bf0 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -38,11 +38,12 @@ ReplacingSortedAlgorithm::ReplacingSortedAlgorithm( const String & version_column, size_t max_block_size_rows, size_t max_block_size_bytes, + std::optional max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, bool cleanup_, bool enable_vertical_final_) - : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs, std::make_unique(use_average_block_sizes, max_block_size_rows, max_block_size_bytes)) + : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs, std::make_unique(use_average_block_sizes, max_block_size_rows, max_block_size_bytes, max_dynamic_subcolumns_)) , cleanup(cleanup_), enable_vertical_final(enable_vertical_final_) { if (!is_deleted_column.empty()) diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index 40ef8eb7f7e0..7ab4d07ba0a2 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -44,6 +44,7 @@ class ReplacingSortedAlgorithm final : public IMergingAlgorithmWithSharedChunks const String & version_column, size_t max_block_size_rows, size_t max_block_size_bytes, + std::optional max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false, bool cleanup = false, diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 753f0a21230b..6598846d4ae7 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -566,8 +566,8 @@ static void setRow(Row & row, std::vector & row_columns, const Column } -SummingSortedAlgorithm::SummingMergedData::SummingMergedData(UInt64 max_block_size_rows_, UInt64 max_block_size_bytes_, ColumnsDefinition & def_) - : MergedData(false, max_block_size_rows_, max_block_size_bytes_) +SummingSortedAlgorithm::SummingMergedData::SummingMergedData(UInt64 max_block_size_rows_, UInt64 max_block_size_bytes_, std::optional max_dynamic_subcolumns_, ColumnsDefinition & def_) + : MergedData(false, max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_) , def(def_), current_row(def.column_names.size()), current_row_columns(def.column_names.size()) { } @@ -802,6 +802,7 @@ SummingSortedAlgorithm::SummingSortedAlgorithm( const Names & partition_and_sorting_required_columns, size_t max_block_size_rows, size_t max_block_size_bytes, + std::optional max_dynamic_subcolumns_, const String & sum_function_name, const String & sum_function_map_name, bool remove_default_values, @@ -809,7 +810,7 @@ SummingSortedAlgorithm::SummingSortedAlgorithm( : IMergingAlgorithmWithDelayedChunk(header_, num_inputs, std::move(description_)) , columns_definition( defineColumns(*header_, description, column_names_to_sum, partition_and_sorting_required_columns, sum_function_name, sum_function_map_name, remove_default_values, aggregate_all_columns)) - , merged_data(max_block_size_rows, max_block_size_bytes, columns_definition) + , merged_data(max_block_size_rows, max_block_size_bytes, max_dynamic_subcolumns_, columns_definition) { } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index 577dcba0fc2a..bbe68cf219a5 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -25,6 +25,7 @@ class SummingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk const Names & partition_and_sorting_required_columns, size_t max_block_size_rows, size_t max_block_size_bytes, + std::optional max_dynamic_subcolumns_, const String & sum_function_name, const String & sum_function_map_name, bool remove_default_values, @@ -77,7 +78,7 @@ class SummingSortedAlgorithm final : public IMergingAlgorithmWithDelayedChunk using MergedData::insertRow; public: - SummingMergedData(UInt64 max_block_size_rows, UInt64 max_block_size_bytes_, ColumnsDefinition & def_); + SummingMergedData(UInt64 max_block_size_rows, UInt64 max_block_size_bytes_, std::optional max_dynamic_subcolumns_, ColumnsDefinition & def_); void initialize(const Block & header, const IMergingAlgorithm::Inputs & inputs) override; diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp index 51be6347861d..3452b25a7a5a 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.cpp @@ -15,9 +15,10 @@ VersionedCollapsingAlgorithm::VersionedCollapsingAlgorithm( const String & sign_column_, size_t max_block_size_rows_, size_t max_block_size_bytes_, + std::optional max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes) - : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE, std::make_unique(use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_)) + : IMergingAlgorithmWithSharedChunks(header_, num_inputs, std::move(description_), out_row_sources_buf_, MAX_ROWS_IN_MULTIVERSION_QUEUE, std::make_unique(use_average_block_sizes, max_block_size_rows_, max_block_size_bytes_, max_dynamic_subcolumns_)) /// -1 for +1 in FixedSizeDequeWithGaps's internal buffer. 3 is a reasonable minimum size to collapse anything. , max_rows_in_queue(std::min(std::max(3, max_block_size_rows_), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 1) , current_keys(max_rows_in_queue) diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h index 0cf6315b9b51..1522b41697af 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h @@ -22,6 +22,7 @@ class VersionedCollapsingAlgorithm final : public IMergingAlgorithmWithSharedChu SortDescription description_, const String & sign_column_, size_t max_block_size_rows, size_t max_block_size_bytes, + std::optional max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false); diff --git a/src/Processors/Merges/CoalescingSortedTransform.h b/src/Processors/Merges/CoalescingSortedTransform.h index d37ba2984c41..c9bab156469b 100644 --- a/src/Processors/Merges/CoalescingSortedTransform.h +++ b/src/Processors/Merges/CoalescingSortedTransform.h @@ -21,7 +21,8 @@ class CoalescingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_ ) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, @@ -32,6 +33,7 @@ class CoalescingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( @@ -34,6 +35,7 @@ class CollapsingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, Graphite::Params params_, time_t time_of_merge_) : IMergingTransform( @@ -25,6 +26,7 @@ class GraphiteRollupSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_, bool always_read_till_end_, @@ -37,6 +38,7 @@ MergingSortedTransform::MergingSortedTransform( description_, max_block_size_rows, max_block_size_bytes, + max_dynamic_subcolumns_, sorting_queue_strategy, limit_, out_row_sources_buf_, diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index ee4be2177cab..7f2ccfab38bd 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -17,6 +17,7 @@ class MergingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, SortingQueueStrategy sorting_queue_strategy, UInt64 limit_ = 0, bool always_read_till_end_ = false, diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index 6e8eb4dff6c2..e9bf4aed7b6a 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -21,6 +21,7 @@ class ReplacingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false, bool cleanup = false, @@ -34,6 +35,7 @@ class ReplacingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_ ) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, @@ -34,6 +35,7 @@ class SummingSortedTransform final : public IMergingTransform max_dynamic_subcolumns_, WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( @@ -31,6 +32,7 @@ class VersionedCollapsingTransform final : public IMergingTransform(header, num_outputs, - sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch); + sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt, SortingQueueStrategy::Batch); case MergeTreeData::MergingParams::Collapsing: return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, true, max_block_size_rows, /*max_block_size_bytes=*/0); + sort_description, merging_params.sign_column, true, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt); case MergeTreeData::MergingParams::Summing: { auto required_columns = metadata_snapshot->getPartitionKey().expression->getRequiredColumns(); required_columns.append_range(metadata_snapshot->getSortingKey().expression->getRequiredColumns()); return std::make_shared(header, num_outputs, - sort_description, merging_params.columns_to_sum, required_columns, max_block_size_rows, /*max_block_size_bytes=*/0); + sort_description, merging_params.columns_to_sum, required_columns, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt); } case MergeTreeData::MergingParams::Aggregating: return std::make_shared(header, num_outputs, - sort_description, max_block_size_rows, /*max_block_size_bytes=*/0); + sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt); case MergeTreeData::MergingParams::Replacing: return std::make_shared(header, num_outputs, - sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size_rows, /*max_block_size_bytes=*/0, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty(), enable_vertical_final); + sort_description, merging_params.is_deleted_column, merging_params.version_column, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt, /*out_row_sources_buf_*/ nullptr, /*use_average_block_sizes*/ false, /*cleanup*/ !merging_params.is_deleted_column.empty(), enable_vertical_final); case MergeTreeData::MergingParams::VersionedCollapsing: return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, max_block_size_rows, /*max_block_size_bytes=*/0); + sort_description, merging_params.sign_column, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt); case MergeTreeData::MergingParams::Graphite: return std::make_shared(header, num_outputs, - sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now); + sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt, merging_params.graphite_params, now); case MergeTreeData::MergingParams::Coalescing: { auto required_columns = metadata_snapshot->getPartitionKey().expression->getRequiredColumns(); required_columns.append_range(metadata_snapshot->getSortingKey().expression->getRequiredColumns()); return std::make_shared(header, num_outputs, - sort_description, merging_params.columns_to_sum, required_columns, max_block_size_rows, /*max_block_size_bytes=*/0); + sort_description, merging_params.columns_to_sum, required_columns, max_block_size_rows, /*max_block_size_bytes=*/0, /*max_dynamic_subcolumns*/std::nullopt); } } }; diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 8c180d864a6c..13c22aa441ef 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -342,6 +342,7 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr result_sort_desc, sort_settings.max_block_size, /*max_block_size_bytes=*/0, + /*max_dynamic_subcolumns*/std::nullopt, SortingQueueStrategy::Batch, limit_, always_read_till_end, @@ -446,6 +447,7 @@ void SortingStep::fullSort( result_sort_desc, sort_settings.max_block_size, /*max_block_size_bytes=*/0, + /*max_dynamic_subcolumns*/std::nullopt, SortingQueueStrategy::Batch, limit_, always_read_till_end); diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index 321804652758..8a892867f0a7 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -35,11 +35,13 @@ ColumnGathererStream::ColumnGathererStream( ReadBuffer & row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, + std::optional max_dynamic_subcolumns_, bool is_result_sparse_) : sources(num_inputs) , row_sources_buf(row_sources_buf_) , block_preferred_size_rows(block_preferred_size_rows_) , block_preferred_size_bytes(block_preferred_size_bytes_) + , max_dynamic_subcolumns(max_dynamic_subcolumns_) , is_result_sparse(is_result_sparse_) { if (num_inputs == 0) @@ -77,7 +79,7 @@ void ColumnGathererStream::initialize(Inputs inputs) result_column = ColumnSparse::create(std::move(result_column)); if (result_column->hasDynamicStructure()) - result_column->takeDynamicStructureFromSourceColumns(source_columns); + result_column->takeDynamicStructureFromSourceColumns(source_columns, max_dynamic_subcolumns); } IMergingAlgorithm::Status ColumnGathererStream::merge() @@ -195,10 +197,11 @@ ColumnGathererTransform::ColumnGathererTransform( std::unique_ptr row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, + std::optional max_dynamic_subcolumns_, bool is_result_sparse_) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, - num_inputs, *row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_) + num_inputs, *row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, max_dynamic_subcolumns_, is_result_sparse_) , row_sources_buf_holder(std::move(row_sources_buf_)) , log(getLogger("ColumnGathererStream")) { diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index d3b3bca6c768..1b70212ea4e6 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -61,6 +61,7 @@ class ColumnGathererStream final : public IMergingAlgorithm ReadBuffer & row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, + std::optional max_dynamic_subcolumns_, bool is_result_sparse_); const char * getName() const override { return "ColumnGathererStream"; } @@ -94,6 +95,7 @@ class ColumnGathererStream final : public IMergingAlgorithm const size_t block_preferred_size_rows; const size_t block_preferred_size_bytes; + const std::optional max_dynamic_subcolumns; const bool is_result_sparse; Source * source_to_fully_copy = nullptr; @@ -113,6 +115,7 @@ class ColumnGathererTransform final : public IMergingTransform row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, + std::optional max_dynamic_subcolumns_, bool is_result_sparse_); String getName() const override { return "ColumnGathererTransform"; } diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 9fa4d503610d..d8f80640ea12 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -269,7 +269,8 @@ void MergeSortingTransform::consume(Chunk chunk) 0, description, max_merged_block_size, - /*max_merged_block_size_bytes*/0, + /*max_merged_block_size_bytes=*/0, + /*max_dynamic_subcolumns=*/std::nullopt, SortingQueueStrategy::Batch, limit, /*always_read_till_end_=*/ false, diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index 02e8d6614aa6..72db5049ebdf 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -82,8 +82,19 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) EXPECT_EQ(pipe.numOutputPorts(), 3); - auto transform = std::make_shared(pipe.getSharedHeader(), pipe.numOutputPorts(), sort_description, - 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, true); + auto transform = std::make_shared( + pipe.getSharedHeader(), + pipe.numOutputPorts(), + sort_description, + /*max_block_size_rows=*/ 8192, + /*max_block_size_bytes=*/ 0, + /*max_dynamic_subcolumns=*/ std::nullopt, + SortingQueueStrategy::Batch, + /*limit=*/ 0, + /*always_read_till_end=*/ false, + /*out_row_sources_buf=*/ nullptr, + /*filter_column_name=*/ std::nullopt, + /*use_average_block_sizes=*/ true); pipe.addTransform(std::move(transform)); @@ -124,8 +135,19 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) EXPECT_EQ(pipe.numOutputPorts(), 3); - auto transform = std::make_shared(pipe.getSharedHeader(), pipe.numOutputPorts(), sort_description, - 8192, /*max_block_size_bytes=*/0, SortingQueueStrategy::Batch, 0, false, nullptr, true); + auto transform = std::make_shared( + pipe.getSharedHeader(), + pipe.numOutputPorts(), + sort_description, + /*max_block_size_rows=*/ 8192, + /*max_block_size_bytes=*/ 0, + /*max_dynamic_subcolumns=*/ std::nullopt, + SortingQueueStrategy::Batch, + /*limit=*/ 0, + /*always_read_till_end=*/ false, + /*out_row_sources_buf=*/ nullptr, + /*filter_column_name=*/ std::nullopt, + /*use_average_block_sizes=*/ true); pipe.addTransform(std::move(transform)); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 961d1769f5b3..4f8964dca32d 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -119,6 +119,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool use_const_adaptive_granularity; extern const MergeTreeSettingsUInt64 max_merge_delayed_streams_for_parallel_write; extern const MergeTreeSettingsBool ttl_only_drop_parts; + extern const MergeTreeSettingsUInt64Auto merge_max_dynamic_subcolumns_in_wide_part; } namespace ErrorCodes @@ -1107,11 +1108,13 @@ class ColumnGathererStep : public ITransformingStep const String & rows_sources_temporary_file_name_, UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, + std::optional max_dynamic_subcolumns_, bool is_result_sparse_) : ITransformingStep(input_header_, input_header_, getTraits()) , rows_sources_temporary_file_name(rows_sources_temporary_file_name_) , merge_block_size_rows(merge_block_size_rows_) , merge_block_size_bytes(merge_block_size_bytes_) + , max_dynamic_subcolumns(max_dynamic_subcolumns_) , is_result_sparse(is_result_sparse_) {} @@ -1133,6 +1136,7 @@ class ColumnGathererStep : public ITransformingStep std::move(rows_sources_read_buf), merge_block_size_rows, merge_block_size_bytes, + max_dynamic_subcolumns, is_result_sparse); pipeline.addTransform(std::move(transform)); @@ -1163,6 +1167,7 @@ class ColumnGathererStep : public ITransformingStep const String rows_sources_temporary_file_name; const UInt64 merge_block_size_rows; const UInt64 merge_block_size_bytes; + const std::optional max_dynamic_subcolumns; const bool is_result_sparse; }; @@ -1213,11 +1218,17 @@ MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & { bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; const auto merge_tree_settings = global_ctx->data->getSettings(); + std::optional max_dynamic_subcolumns = std::nullopt; + if (global_ctx->future_part->part_format.part_type == MergeTreeDataPartType::Wide) + max_dynamic_subcolumns = (*merge_tree_settings)[MergeTreeSetting::merge_max_dynamic_subcolumns_in_wide_part].valueOrNullopt(); + + bool is_result_sparse = ISerialization::hasKind(global_ctx->new_data_part->getSerialization(column_name)->getKindStack(), ISerialization::Kind::SPARSE); auto merge_step = std::make_unique( merge_column_query_plan.getCurrentHeader(), RowsSourcesTemporaryFile::FILE_ID, (*merge_tree_settings)[MergeTreeSetting::merge_max_block_size], (*merge_tree_settings)[MergeTreeSetting::merge_max_block_size_bytes], + max_dynamic_subcolumns, is_result_sparse); merge_step->setStepDescription("Gather column"); merge_column_query_plan.addStep(std::move(merge_step)); @@ -1688,6 +1699,7 @@ class MergePartsStep : public ITransformingStep const String & rows_sources_temporary_file_name_, UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, + std::optional max_dynamic_subcolumns_, bool blocks_are_granules_size_, bool cleanup_, time_t time_of_merge_) @@ -1698,6 +1710,7 @@ class MergePartsStep : public ITransformingStep , rows_sources_temporary_file_name(rows_sources_temporary_file_name_) , merge_block_size_rows(merge_block_size_rows_) , merge_block_size_bytes(merge_block_size_bytes_) + , max_dynamic_subcolumns(max_dynamic_subcolumns_) , blocks_are_granules_size(blocks_are_granules_size_) , cleanup(cleanup_) , time_of_merge(time_of_merge_) @@ -1732,6 +1745,7 @@ class MergePartsStep : public ITransformingStep sort_description, merge_block_size_rows, merge_block_size_bytes, + max_dynamic_subcolumns, SortingQueueStrategy::Default, /* limit_= */0, /* always_read_till_end_= */false, @@ -1742,40 +1756,40 @@ class MergePartsStep : public ITransformingStep case MergeTreeData::MergingParams::Collapsing: merged_transform = std::make_shared( header, input_streams_count, sort_description, merging_params.sign_column, false, - merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns, rows_sources_write_buf, blocks_are_granules_size); break; case MergeTreeData::MergingParams::Summing: merged_transform = std::make_shared( - header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_and_sorting_required_columns, merge_block_size_rows, merge_block_size_bytes); + header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_and_sorting_required_columns, merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns); break; case MergeTreeData::MergingParams::Aggregating: - merged_transform = std::make_shared(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes); + merged_transform = std::make_shared(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns); break; case MergeTreeData::MergingParams::Replacing: merged_transform = std::make_shared( header, input_streams_count, sort_description, merging_params.is_deleted_column, merging_params.version_column, - merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size, + merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns, rows_sources_write_buf, blocks_are_granules_size, cleanup); break; case MergeTreeData::MergingParams::Coalescing: merged_transform = std::make_shared( - header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_and_sorting_required_columns, merge_block_size_rows, merge_block_size_bytes); + header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_and_sorting_required_columns, merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns); break; case MergeTreeData::MergingParams::Graphite: merged_transform = std::make_shared( - header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, + header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns, merging_params.graphite_params, time_of_merge); break; case MergeTreeData::MergingParams::VersionedCollapsing: merged_transform = std::make_shared( header, input_streams_count, sort_description, merging_params.sign_column, - merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + merge_block_size_rows, merge_block_size_bytes, max_dynamic_subcolumns, rows_sources_write_buf, blocks_are_granules_size); break; } @@ -1820,6 +1834,7 @@ class MergePartsStep : public ITransformingStep const String rows_sources_temporary_file_name; const UInt64 merge_block_size_rows; const UInt64 merge_block_size_bytes; + const std::optional max_dynamic_subcolumns; const bool blocks_are_granules_size; const bool cleanup{false}; const time_t time_of_merge{0}; @@ -2019,6 +2034,10 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const bool cleanup = global_ctx->cleanup && global_ctx->future_part->final; + std::optional max_dynamic_subcolumns = std::nullopt; + if (global_ctx->future_part->part_format.part_type == MergeTreeDataPartType::Wide) + max_dynamic_subcolumns = (*merge_tree_settings)[MergeTreeSetting::merge_max_dynamic_subcolumns_in_wide_part].valueOrNullopt(); + auto merge_step = std::make_unique( merge_parts_query_plan.getCurrentHeader(), sort_description, @@ -2027,6 +2046,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const (is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources' temporary file is used only for vertical merge (*merge_tree_settings)[MergeTreeSetting::merge_max_block_size], (*merge_tree_settings)[MergeTreeSetting::merge_max_block_size_bytes], + max_dynamic_subcolumns, ctx->blocks_are_granules_size, cleanup, global_ctx->time_of_merge); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 1cf7966dddc0..e680bed5f391 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -448,34 +448,34 @@ Block MergeTreeDataWriter::mergeBlock( return nullptr; case MergeTreeData::MergingParams::Replacing: return std::make_shared( - header, 1, sort_description, merging_params.is_deleted_column, merging_params.version_column, block_size + 1, /*block_size_bytes=*/0); + header, 1, sort_description, merging_params.is_deleted_column, merging_params.version_column, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt); case MergeTreeData::MergingParams::Collapsing: return std::make_shared( header, 1, sort_description, merging_params.sign_column, - false, block_size + 1, /*block_size_bytes=*/0, getLogger("MergeTreeDataWriter"), /*out_row_sources_buf_=*/ nullptr, + false, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt, getLogger("MergeTreeDataWriter"), /*out_row_sources_buf_=*/ nullptr, /*use_average_block_sizes=*/ false, /*throw_if_invalid_sign=*/ true); case MergeTreeData::MergingParams::Summing: { auto required_columns = metadata_snapshot->getPartitionKey().expression->getRequiredColumns(); required_columns.append_range(metadata_snapshot->getSortingKey().expression->getRequiredColumns()); return std::make_shared( header, 1, sort_description, merging_params.columns_to_sum, - required_columns, block_size + 1, /*block_size_bytes=*/0, "sumWithOverflow", "sumMapWithOverflow", true, false); + required_columns, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt, "sumWithOverflow", "sumMapWithOverflow", true, false); } case MergeTreeData::MergingParams::Aggregating: - return std::make_shared(header, 1, sort_description, block_size + 1, /*block_size_bytes=*/0); + return std::make_shared(header, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt); case MergeTreeData::MergingParams::VersionedCollapsing: return std::make_shared( - header, 1, sort_description, merging_params.sign_column, block_size + 1, /*block_size_bytes=*/0); + header, 1, sort_description, merging_params.sign_column, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt); case MergeTreeData::MergingParams::Graphite: return std::make_shared( - header, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, merging_params.graphite_params, time(nullptr)); + header, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt, merging_params.graphite_params, time(nullptr)); case MergeTreeData::MergingParams::Coalescing: { auto required_columns = metadata_snapshot->getPartitionKey().expression->getRequiredColumns(); required_columns.append_range(metadata_snapshot->getSortingKey().expression->getRequiredColumns()); return std::make_shared( header, 1, sort_description, merging_params.columns_to_sum, - required_columns, block_size + 1, /*block_size_bytes=*/0, "last_value", "last_value", false, true); + required_columns, block_size + 1, /*block_size_bytes=*/0, /*max_dynamic_subcolumns=*/std::nullopt, "last_value", "last_value", false, true); } } }; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index c73ca49f808c..d7d1136f1e4f 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -293,6 +293,13 @@ namespace ErrorCodes Enables writing marks per each substream instead of per each column in Compact parts. It allows to read individual subcolumns from the data part efficiently. )", 0) \ + DECLARE(UInt64Auto, merge_max_dynamic_subcolumns_in_wide_part, Field("auto"), R"( + The maximum number of dynamic subcolumns that can be created in every column in the Wide data part after merge. + It allows to reduce number of files created in Wide data part regardless of dynamic parameters specified in the data type. + + For example, if the table has a column with the JSON(max_dynamic_paths=1024) type and the setting merge_max_dynamic_subcolumns_in_wide_part is set to 128, + after merge into the Wide data part number of dynamic paths will be decreased to 128 in this part and only 128 paths will be written as dynamic subcolumns. + )", 0) \ \ /** Merge selector settings. */ \ DECLARE(UInt64, merge_selector_blurry_base_scale_factor, 0, R"( diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index fdefe6b8df92..c9580863b3bd 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -49,6 +49,7 @@ struct MutableColumnsAndConstraints; M(CLASS_NAME, Seconds) \ M(CLASS_NAME, String) \ M(CLASS_NAME, UInt64) \ + M(CLASS_NAME, UInt64Auto) \ M(CLASS_NAME, MergeTreeObjectSerializationVersion) \ M(CLASS_NAME, MergeTreeObjectSharedDataSerializationVersion) \ M(CLASS_NAME, MergeTreeDynamicSerializationVersion) \ diff --git a/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.reference b/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.reference new file mode 100644 index 000000000000..be6bf773c679 --- /dev/null +++ b/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.reference @@ -0,0 +1,4 @@ +['a','b','c','d'] ['e','f'] +['a','b'] ['c','d','e','f'] +['a','b','c','d'] ['e','f'] +['a','b'] ['c','d','e','f'] diff --git a/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.sql b/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.sql new file mode 100644 index 000000000000..ad39526501bc --- /dev/null +++ b/tests/queries/0_stateless/03638_merge_max_dynamic_subcolumns_in_wide_part.sql @@ -0,0 +1,16 @@ +drop table if exists test; +create table test (json JSON(max_dynamic_paths=4)) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, merge_max_dynamic_subcolumns_in_wide_part=2; +insert into test select '{"a" : 42, "b" : 42, "c" : 42, "d" : 42, "e" : 42, "f" : 42}'; +select JSONDynamicPaths(json), JSONSharedDataPaths(json) from test; +insert into test select '{"a" : 42, "b" : 42, "c" : 42, "d" : 42, "e" : 42, "f" : 42}'; +optimize table test final; +select JSONDynamicPaths(json), JSONSharedDataPaths(json) from test limit 1; +drop table test; + +create table test (x UInt32, json JSON(max_dynamic_paths=4)) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=1, min_rows_for_wide_part=1, merge_max_dynamic_subcolumns_in_wide_part=2, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1; +insert into test select 42, '{"a" : 42, "b" : 42, "c" : 42, "d" : 42, "e" : 42, "f" : 42}'; +select JSONDynamicPaths(json), JSONSharedDataPaths(json) from test; +insert into test select 42, '{"a" : 42, "b" : 42, "c" : 42, "d" : 42, "e" : 42, "f" : 42}'; +optimize table test final; +select JSONDynamicPaths(json), JSONSharedDataPaths(json) from test limit 1; +drop table test \ No newline at end of file From bbb95b42c4937bcf82197394e0529ea7ebdc0340 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Dec 2025 15:14:43 +0000 Subject: [PATCH 085/113] Backport #92376 to 25.8: Fix `count_distinct_optimization` pass over window functions and over multiple arguments --- src/Analyzer/Passes/CountDistinctPass.cpp | 4 ++-- .../03761_count_distinct_optimization_window.reference | 10 ++++++++++ .../03761_count_distinct_optimization_window.sql | 9 +++++++++ ...nt_distinct_optimization_multiple_columns.reference | 2 ++ ...62_count_distinct_optimization_multiple_columns.sql | 2 ++ 5 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03761_count_distinct_optimization_window.reference create mode 100644 tests/queries/0_stateless/03761_count_distinct_optimization_window.sql create mode 100644 tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.reference create mode 100644 tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.sql diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index b00e04859df9..027688d0cb0f 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -55,7 +55,7 @@ class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContextas(); - if (!function_node) + if (!function_node || function_node->hasWindow()) return; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); @@ -64,7 +64,7 @@ class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContextgetArguments().getNodes(); - if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) + if (count_distinct_arguments_nodes.size() != 1 || count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) return; auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; diff --git a/tests/queries/0_stateless/03761_count_distinct_optimization_window.reference b/tests/queries/0_stateless/03761_count_distinct_optimization_window.reference new file mode 100644 index 000000000000..f00c965d8307 --- /dev/null +++ b/tests/queries/0_stateless/03761_count_distinct_optimization_window.reference @@ -0,0 +1,10 @@ +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 diff --git a/tests/queries/0_stateless/03761_count_distinct_optimization_window.sql b/tests/queries/0_stateless/03761_count_distinct_optimization_window.sql new file mode 100644 index 000000000000..7ce3c95b887a --- /dev/null +++ b/tests/queries/0_stateless/03761_count_distinct_optimization_window.sql @@ -0,0 +1,9 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/86442 +-- The transformation shouldn't be applied if the aggregation has window parameters +SELECT uniqExact(c0) OVER (ORDER BY c0 DESC) +FROM +( + SELECT number AS c0 + FROM numbers(10) +) AS t0 +SETTINGS count_distinct_optimization = 1, enable_analyzer = 1; \ No newline at end of file diff --git a/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.reference b/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.reference new file mode 100644 index 000000000000..e862ecadf058 --- /dev/null +++ b/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.reference @@ -0,0 +1,2 @@ +143 +143 diff --git a/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.sql b/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.sql new file mode 100644 index 000000000000..e099379c79d2 --- /dev/null +++ b/tests/queries/0_stateless/03762_count_distinct_optimization_multiple_columns.sql @@ -0,0 +1,2 @@ +SELECT count(DISTINCT x, y) FROM (SELECT number % 11 AS x, number % 13 AS y FROM system.numbers LIMIT 1000) SETTINGS count_distinct_optimization = 0, enable_analyzer = 1; +SELECT count(DISTINCT x, y) FROM (SELECT number % 11 AS x, number % 13 AS y FROM system.numbers LIMIT 1000) SETTINGS count_distinct_optimization = 1, enable_analyzer = 1; \ No newline at end of file From 6ba491ef882a06506b9c9110983301d00f4b51fe Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 20 Dec 2025 15:14:39 +0000 Subject: [PATCH 086/113] Backport #92726 to 25.8: Fix incorrect granules pruning in`KeyCondition` for `match` function --- src/Storages/MergeTree/KeyCondition.cpp | 10 +++ ...condition_match_function_relaxed.reference | 86 +++++++++++++++++++ ...2_key_condition_match_function_relaxed.sql | 48 +++++++++++ 3 files changed, 144 insertions(+) create mode 100644 tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference create mode 100644 tests/queries/0_stateless/03772_key_condition_match_function_relaxed.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ca991937b68a..79b03554f63c 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -3300,6 +3300,16 @@ BoolMask KeyCondition::checkInHyperrectangle( rpn_stack.back().can_be_true = mayExistOnBloomFilter(*element.bloom_filter_data, column_index_to_column_bf); } + /// If the condition is relaxed, the `can_be_false` branch is no longer reliable; it may have false negatives. + /// If `element.range` is relaxed (and thus wider) and contains `key_range`, then `can_be_false` becomes false. + /// However, in reality `can_be_false` may be true, because the actual range of element may be stricter than `element.range`. + /// For example, for `match(...)`, a false negative here (i.e. `can_be_false` is false) would make + /// `not match(...)` set `can_be_true = false`, causing us to skip the granule, which would be incorrect. + /// Therefore, we must set `can_be_false = true` to be safe. + /// Additionally, when `KeyCondition::isRelaxed()` is true, the caller should ignore `can_be_false` anyway. + if (element.relaxed) + rpn_stack.back().can_be_false = true; + if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE) rpn_stack.back() = !rpn_stack.back(); } diff --git a/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference new file mode 100644 index 000000000000..1edaf8cde789 --- /dev/null +++ b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference @@ -0,0 +1,86 @@ +-- { echo } +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^https?://clickhouse[.]com/'); +1 +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^https?://clickhouse[.]com/'); +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.03772_table_match) + Indexes: + PrimaryKey + Keys: + url + Condition: not((url in [\'http\', \'httq\'))) + Parts: 1/1 + Granules: 1/1 + Search Algorithm: generic exclusion search + Ranges: 1 +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^abcd'); +1 +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^abcd'); +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.03772_table_match) + Indexes: + PrimaryKey + Keys: + url + Condition: not((url in [\'abcd\', \'abce\'))) + Parts: 1/1 + Granules: 1/1 + Search Algorithm: generic exclusion search + Ranges: 1 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^abcd'); +0 +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^abcd'); +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.03772_table_match) + Indexes: + PrimaryKey + Keys: + url + Condition: (url in [\'abcd\', \'abce\')) + Parts: 0/1 + Granules: 0/1 + Search Algorithm: binary search + Ranges: 0 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^https?://clickhouse[.]com/') = false; +1 +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^https?://clickhouse[.]com/') = false; +Expression ((Project names + Projection)) + Aggregating + Expression (Before GROUP BY) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.03772_table_match) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Ranges: 1 diff --git a/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.sql b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.sql new file mode 100644 index 000000000000..edb033b406ed --- /dev/null +++ b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.sql @@ -0,0 +1,48 @@ +-- Tags: no-replicated-database, no-parallel-replicas +-- no-replicated-database: EXPLAIN output differs for replicated database. +-- no-parallel-replicas: EXPLAIN output differs for parallel replicas. + + +DROP TABLE IF EXISTS 03772_table_match; + +CREATE TABLE 03772_table_match +ENGINE = MergeTree() +ORDER BY url AS +SELECT 'http://example1.com/' AS url; + +-- { echo } +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^https?://clickhouse[.]com/'); + +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^https?://clickhouse[.]com/'); + +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^abcd'); + +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE NOT match(url, '^abcd'); + +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^abcd'); + +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^abcd'); + +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^https?://clickhouse[.]com/') = false; + +EXPLAIN indexes = 1 +SELECT count(*) +FROM 03772_table_match +WHERE match(url, '^https?://clickhouse[.]com/') = false; From d62c349be6b276af9191cfce8440cefa7cff4196 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 21 Dec 2025 17:13:35 +0000 Subject: [PATCH 087/113] Backport #92748 to 25.8: Do not attempt to delete temporary directories at startup if a MergeTree table is created over a readonly disk --- src/Storages/StorageMergeTree.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 45331190bc1d..acfe2905f91d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -192,6 +192,10 @@ StorageMergeTree::StorageMergeTree( void StorageMergeTree::startup() { + /// Do not schedule any background jobs if current storage has static data files. + if (isStaticStorage()) + return; + clearEmptyParts(); /// Temporary directories contain incomplete results of merges (after forced restart) @@ -202,10 +206,6 @@ void StorageMergeTree::startup() time_after_previous_cleanup_parts.restart(); time_after_previous_cleanup_temporary_directories.restart(); - /// Do not schedule any background jobs if current storage has static data files. - if (isStaticStorage()) - return; - try { background_operations_assignee.start(); From cdd5d2ce92a2e7d249c2137a054ec1a05f7e374a Mon Sep 17 00:00:00 2001 From: "Nihal Z. Miaji" <81457724+nihalzp@users.noreply.github.com> Date: Mon, 22 Dec 2025 15:56:16 +0800 Subject: [PATCH 088/113] Make test compatible --- .../03772_key_condition_match_function_relaxed.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference index 1edaf8cde789..36bc49bead94 100644 --- a/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference +++ b/tests/queries/0_stateless/03772_key_condition_match_function_relaxed.reference @@ -20,7 +20,7 @@ Expression ((Project names + Projection)) Parts: 1/1 Granules: 1/1 Search Algorithm: generic exclusion search - Ranges: 1 + Ranges: 1 SELECT count(*) FROM 03772_table_match WHERE NOT match(url, '^abcd'); @@ -42,7 +42,7 @@ Expression ((Project names + Projection)) Parts: 1/1 Granules: 1/1 Search Algorithm: generic exclusion search - Ranges: 1 + Ranges: 1 SELECT count(*) FROM 03772_table_match WHERE match(url, '^abcd'); @@ -64,7 +64,7 @@ Expression ((Project names + Projection)) Parts: 0/1 Granules: 0/1 Search Algorithm: binary search - Ranges: 0 + Ranges: 0 SELECT count(*) FROM 03772_table_match WHERE match(url, '^https?://clickhouse[.]com/') = false; @@ -83,4 +83,4 @@ Expression ((Project names + Projection)) Condition: true Parts: 1/1 Granules: 1/1 - Ranges: 1 + Ranges: 1 From fc9a3ecd9d4a1c99137044ec9e8b81b9383b2c79 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 22 Dec 2025 15:05:14 +0100 Subject: [PATCH 089/113] Update MergeTask.cpp --- src/Storages/MergeTree/MergeTask.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 4f8964dca32d..900910989d32 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1222,7 +1222,6 @@ MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & if (global_ctx->future_part->part_format.part_type == MergeTreeDataPartType::Wide) max_dynamic_subcolumns = (*merge_tree_settings)[MergeTreeSetting::merge_max_dynamic_subcolumns_in_wide_part].valueOrNullopt(); - bool is_result_sparse = ISerialization::hasKind(global_ctx->new_data_part->getSerialization(column_name)->getKindStack(), ISerialization::Kind::SPARSE); auto merge_step = std::make_unique( merge_column_query_plan.getCurrentHeader(), RowsSourcesTemporaryFile::FILE_ID, From f7deba997d134097a8506c35c1cfc2ad4ca1a255 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Mon, 22 Dec 2025 15:07:27 +0100 Subject: [PATCH 090/113] Update StorageLog.cpp --- src/Storages/StorageLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index b17074b7eff5..6fa4c07a44d4 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -253,7 +253,7 @@ void LogSource::readPrefix(const NameAndTypePair & name_and_type, ISerialization if (cache.contains(ISerialization::getSubcolumnNameForStream(path))) return nullptr; - String data_file_name = ISerialization::getFileNameForStream(name_and_type, path, {}); + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); const auto & data_file_it = storage.data_files_by_names.find(data_file_name); if (data_file_it == storage.data_files_by_names.end()) From ddf38389dbd790df37b39c9d987b3ee35b6ad0da Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Dec 2025 15:13:54 +0000 Subject: [PATCH 091/113] Backport #92810 to 25.8: Change log level --- src/Databases/DataLake/RestCatalog.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Databases/DataLake/RestCatalog.cpp b/src/Databases/DataLake/RestCatalog.cpp index a321d3f1b325..482878aeb664 100644 --- a/src/Databases/DataLake/RestCatalog.cpp +++ b/src/Databases/DataLake/RestCatalog.cpp @@ -156,7 +156,7 @@ RestCatalog::Config RestCatalog::loadConfig() std::string json_str; readJSONObjectPossiblyInvalid(json_str, *buf); - LOG_TEST(log, "Received catalog configuration settings: {}", json_str); + LOG_DEBUG(log, "Received catalog configuration settings: {}", json_str); Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(json_str); @@ -170,7 +170,7 @@ RestCatalog::Config RestCatalog::loadConfig() auto overrides_object = object->get("overrides").extract(); parseCatalogConfigurationSettings(overrides_object, result); - LOG_TEST(log, "Parsed catalog configuration settings: {}", result.toString()); + LOG_DEBUG(log, "Parsed catalog configuration settings: {}", result.toString()); return result; } @@ -322,7 +322,7 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( .create(credentials); }; - LOG_TEST(log, "Requesting: {}", url.toString()); + LOG_DEBUG(log, "Requesting: {}", url.toString()); try { @@ -440,7 +440,7 @@ RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_name { auto buf = createReadBuffer(config.prefix / NAMESPACES_ENDPOINT, params); auto namespaces = parseNamespaces(*buf, base_namespace); - LOG_TEST(log, "Loaded {} namespaces in base namespace {}", namespaces.size(), base_namespace); + LOG_DEBUG(log, "Loaded {} namespaces in base namespace {}", namespaces.size(), base_namespace); return namespaces; } catch (const DB::HTTPException & e) @@ -468,7 +468,7 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const String json_str; readJSONObjectPossiblyInvalid(json_str, buf); - LOG_TEST(log, "Received response: {}", json_str); + LOG_DEBUG(log, "Received response: {}", json_str); try { @@ -594,7 +594,7 @@ bool RestCatalog::getTableMetadataImpl( const std::string & table_name, TableMetadata & result) const { - LOG_TEST(log, "Checking table {} in namespace {}", table_name, namespace_name); + LOG_DEBUG(log, "Checking table {} in namespace {}", table_name, namespace_name); DB::HTTPHeaderEntries headers; if (result.requiresCredentials()) @@ -613,7 +613,7 @@ bool RestCatalog::getTableMetadataImpl( if (buf->eof()) { - LOG_TEST(log, "Table doesn't exist (endpoint: {})", endpoint); + LOG_DEBUG(log, "Table doesn't exist (endpoint: {})", endpoint); return false; } @@ -623,7 +623,7 @@ bool RestCatalog::getTableMetadataImpl( #ifdef DEBUG_OR_SANITIZER_BUILD /// This log message might contain credentials, /// so log it only for debugging. - LOG_TEST(log, "Received metadata for table {}: {}", table_name, json_str); + LOG_DEBUG(log, "Received metadata for table {}: {}", table_name, json_str); #endif Poco::JSON::Parser parser; @@ -641,7 +641,7 @@ bool RestCatalog::getTableMetadataImpl( { location = metadata_object->get("location").extract(); result.setLocation(location); - LOG_TEST(log, "Location for table {}: {}", table_name, location); + LOG_DEBUG(log, "Location for table {}: {}", table_name, location); } else { From 13f3fa4156ea790fdf94ea490f6465a791449e34 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 22 Dec 2025 18:58:11 +0000 Subject: [PATCH 092/113] Update autogenerated version to 25.8.13.73 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 7bef112d31f1..fd878184ecfb 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54513) +SET(VERSION_REVISION 54514) SET(VERSION_MAJOR 25) SET(VERSION_MINOR 8) -SET(VERSION_PATCH 13) -SET(VERSION_GITHASH 25db09bd0a09eb1576ae0ba56f6e52d9f2c4651e) -SET(VERSION_DESCRIBE v25.8.13.1-lts) -SET(VERSION_STRING 25.8.13.1) +SET(VERSION_PATCH 14) +SET(VERSION_GITHASH 2dbc61a9bb88108b1e83b11113f24e68f7c211f7) +SET(VERSION_DESCRIBE v25.8.14.1-lts) +SET(VERSION_STRING 25.8.14.1) # end of autochange From 878003d53e08fff1b8dc5353c6f70c9de13132c0 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Dec 2025 14:10:23 +0100 Subject: [PATCH 093/113] Update gtest_blocks_size_merging_streams.cpp --- src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index 72db5049ebdf..9f8ac2320b2d 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -93,7 +93,6 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) /*limit=*/ 0, /*always_read_till_end=*/ false, /*out_row_sources_buf=*/ nullptr, - /*filter_column_name=*/ std::nullopt, /*use_average_block_sizes=*/ true); pipe.addTransform(std::move(transform)); @@ -146,7 +145,6 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) /*limit=*/ 0, /*always_read_till_end=*/ false, /*out_row_sources_buf=*/ nullptr, - /*filter_column_name=*/ std::nullopt, /*use_average_block_sizes=*/ true); pipe.addTransform(std::move(transform)); From 61e20819151119f3ccb541916182680e9e93b3ae Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Dec 2025 14:11:36 +0100 Subject: [PATCH 094/113] Delete tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql --- .../0_stateless/03753_log_engine_shared_prefixes_bug.sql | 7 ------- 1 file changed, 7 deletions(-) delete mode 100644 tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql diff --git a/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql b/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql deleted file mode 100644 index c4e3176aa286..000000000000 --- a/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.sql +++ /dev/null @@ -1,7 +0,0 @@ -DROP TABLE IF EXISTS test; -CREATE TABLE test (c String) ENGINE = Log; -INSERT INTO TABLE test SELECT randomString(10) from numbers(1000); -INSERT INTO TABLE test SELECT randomString(10) from numbers(1000); -SELECT * FROM test ORDER BY c, c.size FORMAT Null; -DROP TABLE test; - From 3adb2a6c9ed4f4cd4ba1e9ad9053c0df52e64669 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 23 Dec 2025 14:11:47 +0100 Subject: [PATCH 095/113] Delete tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference --- .../0_stateless/03753_log_engine_shared_prefixes_bug.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference diff --git a/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference b/tests/queries/0_stateless/03753_log_engine_shared_prefixes_bug.reference deleted file mode 100644 index e69de29bb2d1..000000000000 From 206a0f919cf35f2b51814953da9b8549eb907f53 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 23 Dec 2025 13:24:39 +0000 Subject: [PATCH 096/113] Backport #92453 to 25.8: Fix possible conflict in column and subcolumn with the same name --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 +++++-- ...olumn_conflicting_with_subcolumn.reference | 20 +++++++++++++++++++ ...3763_column_conflicting_with_subcolumn.sql | 7 +++++++ 3 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.reference create mode 100644 tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3d0570d0acb2..bdffdf4f2d26 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -570,9 +570,10 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const size_t pos = 0; for (const auto & column : columns) - { column_name_to_position.emplace(column.name, pos++); + for (const auto & column : columns) + { auto it = serialization_infos.find(column.name); auto serialization = it == serialization_infos.end() ? IDataType::getSerialization(column) @@ -583,7 +584,9 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const IDataType::forEachSubcolumn([&](const auto &, const auto & subname, const auto & subdata) { auto full_name = Nested::concatenateName(column.name, subname); - serializations.emplace(full_name, subdata.serialization); + /// Don't override the column serialization with subcolumn serialization if column with the same name exists. + if (!column_name_to_position.contains(full_name)) + serializations.emplace(full_name, subdata.serialization); }, ISerialization::SubstreamData(serialization)); } diff --git a/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.reference b/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.reference new file mode 100644 index 000000000000..93d50fd0a518 --- /dev/null +++ b/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.reference @@ -0,0 +1,20 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +[0] 0 +[1] 1 +[2] 2 +[3] 3 +[4] 4 +[5] 5 +[6] 6 +[7] 7 +[8] 8 +[9] 9 diff --git a/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.sql b/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.sql new file mode 100644 index 000000000000..b3d455dc713e --- /dev/null +++ b/tests/queries/0_stateless/03763_column_conflicting_with_subcolumn.sql @@ -0,0 +1,7 @@ +drop table if exists test; +create table test (a Array(UInt64), `a.size0` UInt64) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=1; +insert into test select [number], number from numbers(10); +select a.size0 from test; +select a, a.size0 from test; +drop table test; + From 295e8e0abc68ceefecdbd9a227eb423d6f0894fb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 23 Dec 2025 18:16:27 +0000 Subject: [PATCH 097/113] Backport #92867 to 25.8: Fix tuples in the old parquet reader --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 42 ++++++++++--------- .../03773_parquet_roundtrip_bug.reference | 2 + .../03773_parquet_roundtrip_bug.sql | 5 +++ 3 files changed, 30 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/03773_parquet_roundtrip_bug.reference create mode 100644 tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 06b69ca71e3f..9508fb199ba6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -64,7 +64,6 @@ namespace ErrorCodes extern const int MEMORY_LIMIT_EXCEEDED; extern const int CANNOT_READ_ALL_DATA; extern const int CANNOT_PARSE_NUMBER; - extern const int LOGICAL_ERROR; } namespace @@ -358,7 +357,7 @@ static KeyCondition::ColumnIndexToBloomFilter buildColumnIndexToBF( } // Complex / nested types contain more than one index. We don't support those. - if (parquet_indexes.size() > 1) + if (parquet_indexes.size() != 1) { continue; } @@ -573,18 +572,13 @@ const parquet::ColumnDescriptor * getColumnDescriptorIfBloomFilterIsPresent( const auto & parquet_indexes = clickhouse_column_index_to_parquet_index[clickhouse_column_index].parquet_indexes; - // complex types like structs, tuples and maps will have more than one index. - // we don't support those for now - if (parquet_indexes.size() > 1) + /// Complex types like structs, tuples and maps will have more than one index; we don't support those for now. + /// Empty tuples are also not supported. + if (parquet_indexes.size() != 1) { return nullptr; } - if (parquet_indexes.empty()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column maps to 0 parquet leaf columns, raise an issue and try the query with `input_format_parquet_bloom_filter_push_down=false`"); - } - auto parquet_column_index = parquet_indexes[0]; const auto * parquet_column_descriptor = parquet_rg_metadata->schema()->Column(parquet_column_index); @@ -1275,23 +1269,33 @@ NamesAndTypesList ArrowParquetSchemaReader::readSchema() THROW_ARROW_NOT_OK(parquet::arrow::FromParquetSchema(metadata->schema(), &schema)); /// When Parquet's schema is converted to Arrow's schema, logical types are lost (at least in - /// the currently used Arrow 11 version). Therefore, we manually add the logical types as metadata - /// to Arrow's schema. Logical types are useful for determining which ClickHouse column type to convert to. + /// the currently used Arrow 11 version). This is only a problem for JSON columns, which are + /// string columns with a piece of metadata saying that they should be converted to JSON. + /// Here's a hack to propagate logical types in simple cases. std::vector> new_fields; new_fields.reserve(schema->num_fields()); + ArrowFieldIndexUtil field_util( + format_settings.parquet.case_insensitive_column_matching, + format_settings.parquet.allow_missing_columns); + const auto field_indices = field_util.calculateFieldIndices(*schema); + for (int i = 0; i < schema->num_fields(); ++i) { auto field = schema->field(i); - const auto * parquet_node = metadata->schema()->Column(i); - const auto * lt = parquet_node->logical_type().get(); - - if (lt and !lt->is_invalid()) + auto it = field_indices.find(field->name()); + if (it != field_indices.end() && it->second.second == 1) { - std::shared_ptr kv = field->HasMetadata() ? field->metadata()->Copy() : arrow::key_value_metadata({}, {}); - THROW_ARROW_NOT_OK(kv->Set("PARQUET:logical_type", lt->ToString())); + const auto * parquet_node = metadata->schema()->Column(it->second.first); + const auto * lt = parquet_node->logical_type().get(); - field = field->WithMetadata(std::move(kv)); + if (lt && !lt->is_invalid()) + { + std::shared_ptr kv = field->HasMetadata() ? field->metadata()->Copy() : arrow::key_value_metadata({}, {}); + THROW_ARROW_NOT_OK(kv->Set("PARQUET:logical_type", lt->ToString())); + + field = field->WithMetadata(std::move(kv)); + } } new_fields.emplace_back(std::move(field)); } diff --git a/tests/queries/0_stateless/03773_parquet_roundtrip_bug.reference b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.reference new file mode 100644 index 000000000000..bc03cc2d26b4 --- /dev/null +++ b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.reference @@ -0,0 +1,2 @@ +JSON Tuple(\n `1` UInt8,\n `2` String) JSON +JSON Tuple(\n `1` UInt8,\n `2` String) JSON diff --git a/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql new file mode 100644 index 000000000000..582961d49f80 --- /dev/null +++ b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql @@ -0,0 +1,5 @@ +-- Tags: no-fasttest + +insert into function file(currentDatabase() || '_03773.parquet') select '{"a": 42}'::JSON as j1, tuple(42, '{}') as t, '{"b": 1337}'::JSON as j2 settings engine_file_truncate_on_insert=1; +select toTypeName(j1), toTypeName(t), toTypeName(j2) from file(currentDatabase() || '_03773.parquet') settings input_format_parquet_use_native_reader_v3=0; +select toTypeName(j1), toTypeName(t), toTypeName(j2) from file(currentDatabase() || '_03773.parquet') settings input_format_parquet_use_native_reader_v3=1; From e48441d16f4caf788df41f9c487d0cb05809cf83 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 23 Dec 2025 18:18:45 +0000 Subject: [PATCH 098/113] Backport #92838 to 25.8: Fix usage of index with subqueries and `IN` clause in lightweight updates --- src/Interpreters/MutationsInterpreter.cpp | 44 ++++++++++++------- src/Interpreters/MutationsInterpreter.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 15 +++++-- src/Processors/QueryPlan/CreatingSetsStep.h | 2 +- ...ightweight_update_index_subquery.reference | 3 ++ ...3773_lightweight_update_index_subquery.sql | 21 +++++++++ 6 files changed, 65 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/03773_lightweight_update_index_subquery.reference create mode 100644 tests/queries/0_stateless/03773_lightweight_update_index_subquery.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e055697f0326..825e993f2e3a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1339,6 +1339,19 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s } } +std::optional MutationsInterpreter::createFilterDAGForStage(const Stage & stage) +{ + const auto & names = stage.filter_column_names; + if (names.empty()) + return std::nullopt; + + ActionsDAG::NodeRawConstPtrs nodes(names.size()); + for (size_t i = 0; i < names.size(); ++i) + nodes[i] = &stage.expressions_chain.steps[i]->actions()->dag.findInOutputs(names[i]); + + return ActionsDAG::buildFilterActionsDAG(nodes); +} + void MutationsInterpreter::Source::read( Stage & first_stage, QueryPlan & plan, @@ -1366,20 +1379,6 @@ void MutationsInterpreter::Source::read( if (data) { - const auto & steps = first_stage.expressions_chain.steps; - const auto & names = first_stage.filter_column_names; - size_t num_filters = names.size(); - - std::optional filter; - if (!first_stage.filter_column_names.empty()) - { - ActionsDAG::NodeRawConstPtrs nodes(num_filters); - for (size_t i = 0; i < num_filters; ++i) - nodes[i] = &steps[i]->actions()->dag.findInOutputs(names[i]); - - filter = ActionsDAG::buildFilterActionsDAG(nodes); - } - createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, plan, @@ -1391,7 +1390,7 @@ void MutationsInterpreter::Source::read( required_columns, nullptr, mutation_settings.apply_deleted_mask, - std::move(filter), + createFilterDAGForStage(first_stage), false, false, context_, @@ -1400,6 +1399,7 @@ void MutationsInterpreter::Source::read( else { auto select = std::make_shared(); + std::shared_ptr filter_actions_dag; select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); for (const auto & column_name : first_stage.output_columns) @@ -1412,8 +1412,11 @@ void MutationsInterpreter::Source::read( if (!first_stage.filters.empty()) { ASTPtr where_expression; + if (first_stage.filters.size() == 1) + { where_expression = first_stage.filters[0]; + } else { auto coalesced_predicates = std::make_shared(); @@ -1423,11 +1426,18 @@ void MutationsInterpreter::Source::read( coalesced_predicates->arguments->children = first_stage.filters; where_expression = std::move(coalesced_predicates); } + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression)); + + if (auto filter = createFilterDAGForStage(first_stage)) + filter_actions_dag = std::make_shared(std::move(*filter)); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to create filter DAG for stage with non-empty filters"); } SelectQueryInfo query_info; query_info.query = std::move(select); + query_info.filter_actions_dag = std::move(filter_actions_dag); size_t max_block_size = context_->getSettingsRef()[Setting::max_block_size]; Names extracted_column_names; @@ -1467,7 +1477,7 @@ void MutationsInterpreter::initQueryPlan(Stage & first_stage, QueryPlan & plan) plan.setConcurrencyControl(false); source.read(first_stage, plan, metadata_snapshot, context, settings); - addCreatingSetsStep(plan, first_stage.analyzer->getPreparedSets(), context); + addDelayedCreatingSetsStep(plan, first_stage.analyzer->getPreparedSets(), context); } QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const @@ -1498,7 +1508,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } } - addCreatingSetsStep(plan, stage.analyzer->getPreparedSets(), context); + addDelayedCreatingSetsStep(plan, stage.analyzer->getPreparedSets(), context); } QueryPlanOptimizationSettings do_not_optimize_plan_settings(context); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 67e762f2b5b7..ad3b44398fa2 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -176,8 +176,8 @@ class MutationsInterpreter void initQueryPlan(Stage & first_stage, QueryPlan & query_plan); void prepareMutationStages(std::vector &prepared_stages, bool dry_run); QueryPipelineBuilder addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; - std::optional getStorageSortDescriptionIfPossible(const Block & header) const; + static std::optional createFilterDAGForStage(const Stage & stage); ASTPtr getPartitionAndPredicateExpressionForMutationCommand(const MutationCommand & command) const; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 51ecef51a9b2..2f2c5a6bf85b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -218,14 +218,13 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet continue; plan->optimize(optimization_settings); - plans.emplace_back(std::move(plan)); } return plans; } -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) +void addDelayedCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { if (!prepared_sets) return; @@ -234,7 +233,17 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (subqueries.empty()) return; - addCreatingSetsStep(query_plan, std::move(subqueries), context); + const auto & settings = context->getSettingsRef(); + SizeLimits network_transfer_limits(settings[Setting::max_rows_to_transfer], settings[Setting::max_bytes_to_transfer], settings[Setting::transfer_overflow_mode]); + auto prepared_sets_cache = context->getPreparedSetsCache(); + + auto step = std::make_unique( + query_plan.getCurrentHeader(), + std::move(subqueries), + network_transfer_limits, + prepared_sets_cache); + + query_plan.addStep(std::move(step)); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 3730545d8125..35dd7515759e 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -92,7 +92,7 @@ class DelayedCreatingSetsStep final : public IQueryPlanStep void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context); -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); +void addDelayedCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipeline, PreparedSets::Subqueries subqueries, ContextPtr context); diff --git a/tests/queries/0_stateless/03773_lightweight_update_index_subquery.reference b/tests/queries/0_stateless/03773_lightweight_update_index_subquery.reference new file mode 100644 index 000000000000..8975c726eb76 --- /dev/null +++ b/tests/queries/0_stateless/03773_lightweight_update_index_subquery.reference @@ -0,0 +1,3 @@ +0 +0 +3 diff --git a/tests/queries/0_stateless/03773_lightweight_update_index_subquery.sql b/tests/queries/0_stateless/03773_lightweight_update_index_subquery.sql new file mode 100644 index 000000000000..9ec34be0e280 --- /dev/null +++ b/tests/queries/0_stateless/03773_lightweight_update_index_subquery.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_lwd; + +CREATE TABLE test_lwd (id UInt64) +ENGINE MergeTree() ORDER BY id +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192; + +SET lightweight_delete_mode = 'lightweight_update_force'; + +INSERT INTO test_lwd SELECT number FROM numbers(100000); + +SET max_rows_to_read = 10000; +SET force_primary_key = 1; + +DELETE FROM test_lwd WHERE id IN (10000, 10001, 10002); +DELETE FROM test_lwd WHERE id IN (SELECT number FROM numbers(10003, 3)); + +SELECT count() FROM test_lwd WHERE id IN (SELECT number FROM numbers(10000, 3)); +SELECT count() FROM test_lwd WHERE id IN (SELECT number FROM numbers(10003, 3)); +SELECT count() FROM test_lwd WHERE id IN (SELECT number FROM numbers(10006, 3)); + +DROP TABLE IF EXISTS test_lwd; From cc8695ab04d16e277937fa943dcb9cdf6660af23 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 23 Dec 2025 22:15:05 +0000 Subject: [PATCH 099/113] Backport #87020 to 25.8: Fix table name encoding in data lake rest catalog --- src/Databases/DataLake/RestCatalog.cpp | 13 +++- .../integration/test_database_iceberg/test.py | 67 +++++++++++++++++++ 2 files changed, 77 insertions(+), 3 deletions(-) diff --git a/src/Databases/DataLake/RestCatalog.cpp b/src/Databases/DataLake/RestCatalog.cpp index 482878aeb664..6c0e771452d2 100644 --- a/src/Databases/DataLake/RestCatalog.cpp +++ b/src/Databases/DataLake/RestCatalog.cpp @@ -302,7 +302,8 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( { const auto & context = getContext(); - Poco::URI url(base_url / endpoint); + /// enable_url_encoding=false to allow use tables with encoded sequences in names like 'foo%2Fbar' + Poco::URI url(base_url / endpoint, /* enable_url_encoding */ false); if (!params.empty()) url.setQueryParameters(params); @@ -542,7 +543,12 @@ DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & bas for (size_t i = 0; i < identifiers_object->size(); ++i) { const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); - const auto table_name = current_table_json->get("name").extract(); + /// If table has encoded sequence (like 'foo%2Fbar') + /// catalog returns decoded character instead of sequence ('foo/bar') + /// Here name encoded back to 'foo%2Fbar' format + const auto table_name_raw = current_table_json->get("name").extract(); + std::string table_name; + Poco::URI::encode(table_name_raw, "/", table_name); tables.push_back(base_namespace + "." + table_name); if (limit && tables.size() >= limit) @@ -731,7 +737,8 @@ void RestCatalog::sendRequest(const String & endpoint, Poco::JSON::Object::Ptr r }; } - Poco::URI url(endpoint); + /// enable_url_encoding=false to allow use tables with encoded sequences in names like 'foo%2Fbar' + Poco::URI url(endpoint, /* enable_url_encoding */ false); auto wb = DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) .withMethod(method) diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 662b51e0f687..895a90e5b42b 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -560,6 +560,7 @@ def test_create(started_cluster): node.query(f"INSERT INTO {CATALOG_NAME}.`{root_namespace}.{table_name}` VALUES ('AAPL');", settings={"allow_experimental_insert_into_iceberg": 1, 'write_full_path_in_iceberg_metadata': 1}) assert node.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_name}`") == "AAPL\n" + def test_drop_table(started_cluster): node = started_cluster.instances["node1"] @@ -575,3 +576,69 @@ def test_drop_table(started_cluster): drop_clickhouse_iceberg_table(node, root_namespace, table_name) assert len(catalog.list_tables(root_namespace)) == 0 + + +def test_table_with_slash(started_cluster): + node = started_cluster.instances["node1"] + + # pyiceberg at current moment (version 0.9.1) has a bug with table names with slashes + # see https://github.com/apache/iceberg-python/issues/2462 + # so we need to encode it manually + table_raw_suffix = "table/foo" + table_encoded_suffix = "table%2Ffoo" + + test_ref = f"test_list_tables_{uuid.uuid4()}" + table_name = f"{test_ref}_{table_raw_suffix}" + table_encoded_name = f"{test_ref}_{table_encoded_suffix}" + root_namespace = f"{test_ref}_namespace" + + catalog = load_catalog_impl(started_cluster) + catalog.create_namespace(root_namespace) + + create_table(catalog, root_namespace, table_name, DEFAULT_SCHEMA, PartitionSpec(), DEFAULT_SORT_ORDER) + + create_clickhouse_iceberg_database(started_cluster, node, CATALOG_NAME) + node.query(f"INSERT INTO {CATALOG_NAME}.`{root_namespace}.{table_encoded_name}` VALUES (NULL, 'AAPL', 193.24, 193.31, tuple('bot'));", settings={"allow_experimental_insert_into_iceberg": 1, 'write_full_path_in_iceberg_metadata': 1}) + assert node.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_encoded_name}`") == "\\N\tAAPL\t193.24\t193.31\t('bot')\n" + + +def test_cluster_select(started_cluster): + node1 = started_cluster.instances["node1"] + node2 = started_cluster.instances["node2"] + + test_ref = f"test_list_tables_{uuid.uuid4()}" + table_name = f"{test_ref}_table" + root_namespace = f"{test_ref}_namespace" + + catalog = load_catalog_impl(started_cluster) + create_clickhouse_iceberg_database(started_cluster, node1, CATALOG_NAME) + create_clickhouse_iceberg_database(started_cluster, node2, CATALOG_NAME) + create_clickhouse_iceberg_table(started_cluster, node1, root_namespace, table_name, "(x String)") + node1.query(f"INSERT INTO {CATALOG_NAME}.`{root_namespace}.{table_name}` VALUES ('pablo');", settings={"allow_experimental_insert_into_iceberg": 1, 'write_full_path_in_iceberg_metadata': 1}) + + query_id = uuid.uuid4().hex + assert node1.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_name}` SETTINGS parallel_replicas_for_cluster_engines=1, enable_parallel_replicas=2, cluster_for_parallel_replicas='cluster_simple'", query_id=query_id) == 'pablo\n' + + node1.query("SYSTEM FLUSH LOGS system.query_log") + node2.query("SYSTEM FLUSH LOGS system.query_log") + + assert node1.query(f"SELECT Settings['parallel_replicas_for_cluster_engines'] AS parallel_replicas_for_cluster_engines FROM system.query_log WHERE query_id = '{query_id}' LIMIT 1;") == '1\n' + + for replica in [node1, node2]: + cluster_secondary_queries = ( + replica.query( + f""" + SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log + WHERE + type = 'QueryStart' AND + positionCaseInsensitive(query, 's3Cluster') != 0 AND + position(query, 'system.query_log') = 0 AND + NOT is_initial_query + """ + ) + .strip() + .split("\n") + ) + assert len(cluster_secondary_queries) == 1 + + assert node2.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_name}`", settings={"parallel_replicas_for_cluster_engines":1, 'enable_parallel_replicas': 2, 'cluster_for_parallel_replicas': 'cluster_simple', 'parallel_replicas_for_cluster_engines' : 1}) == 'pablo\n' From ee67ca6054aa2d6367031707b4da07d3d0b8f89d Mon Sep 17 00:00:00 2001 From: Konstantin Vedernikov <75157521+scanhex12@users.noreply.github.com> Date: Tue, 23 Dec 2025 23:16:50 +0100 Subject: [PATCH 100/113] Update test.py --- .../integration/test_database_iceberg/test.py | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 895a90e5b42b..1fbc012b9056 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -601,44 +601,3 @@ def test_table_with_slash(started_cluster): node.query(f"INSERT INTO {CATALOG_NAME}.`{root_namespace}.{table_encoded_name}` VALUES (NULL, 'AAPL', 193.24, 193.31, tuple('bot'));", settings={"allow_experimental_insert_into_iceberg": 1, 'write_full_path_in_iceberg_metadata': 1}) assert node.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_encoded_name}`") == "\\N\tAAPL\t193.24\t193.31\t('bot')\n" - -def test_cluster_select(started_cluster): - node1 = started_cluster.instances["node1"] - node2 = started_cluster.instances["node2"] - - test_ref = f"test_list_tables_{uuid.uuid4()}" - table_name = f"{test_ref}_table" - root_namespace = f"{test_ref}_namespace" - - catalog = load_catalog_impl(started_cluster) - create_clickhouse_iceberg_database(started_cluster, node1, CATALOG_NAME) - create_clickhouse_iceberg_database(started_cluster, node2, CATALOG_NAME) - create_clickhouse_iceberg_table(started_cluster, node1, root_namespace, table_name, "(x String)") - node1.query(f"INSERT INTO {CATALOG_NAME}.`{root_namespace}.{table_name}` VALUES ('pablo');", settings={"allow_experimental_insert_into_iceberg": 1, 'write_full_path_in_iceberg_metadata': 1}) - - query_id = uuid.uuid4().hex - assert node1.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_name}` SETTINGS parallel_replicas_for_cluster_engines=1, enable_parallel_replicas=2, cluster_for_parallel_replicas='cluster_simple'", query_id=query_id) == 'pablo\n' - - node1.query("SYSTEM FLUSH LOGS system.query_log") - node2.query("SYSTEM FLUSH LOGS system.query_log") - - assert node1.query(f"SELECT Settings['parallel_replicas_for_cluster_engines'] AS parallel_replicas_for_cluster_engines FROM system.query_log WHERE query_id = '{query_id}' LIMIT 1;") == '1\n' - - for replica in [node1, node2]: - cluster_secondary_queries = ( - replica.query( - f""" - SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log - WHERE - type = 'QueryStart' AND - positionCaseInsensitive(query, 's3Cluster') != 0 AND - position(query, 'system.query_log') = 0 AND - NOT is_initial_query - """ - ) - .strip() - .split("\n") - ) - assert len(cluster_secondary_queries) == 1 - - assert node2.query(f"SELECT * FROM {CATALOG_NAME}.`{root_namespace}.{table_name}`", settings={"parallel_replicas_for_cluster_engines":1, 'enable_parallel_replicas': 2, 'cluster_for_parallel_replicas': 'cluster_simple', 'parallel_replicas_for_cluster_engines' : 1}) == 'pablo\n' From b0813d1d3daedb19a5f0b7e5229dd8e2544afc24 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 23 Dec 2025 15:59:53 -0800 Subject: [PATCH 101/113] Retrofit --- tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql index 582961d49f80..7a8eb738812a 100644 --- a/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql +++ b/tests/queries/0_stateless/03773_parquet_roundtrip_bug.sql @@ -1,5 +1,6 @@ -- Tags: no-fasttest +set schema_inference_make_columns_nullable = 0; insert into function file(currentDatabase() || '_03773.parquet') select '{"a": 42}'::JSON as j1, tuple(42, '{}') as t, '{"b": 1337}'::JSON as j2 settings engine_file_truncate_on_insert=1; select toTypeName(j1), toTypeName(t), toTypeName(j2) from file(currentDatabase() || '_03773.parquet') settings input_format_parquet_use_native_reader_v3=0; select toTypeName(j1), toTypeName(t), toTypeName(j2) from file(currentDatabase() || '_03773.parquet') settings input_format_parquet_use_native_reader_v3=1; From 8b06e227076fe580411db5038b444078949fcaf2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Dec 2025 11:13:24 +0000 Subject: [PATCH 102/113] Backport #92807 to 25.8: Fix crash when receiving from disconnected `Connection` --- src/Processors/Sources/RemoteSource.cpp | 8 ++++---- src/Processors/Sources/RemoteSource.h | 1 - src/QueryPipeline/RemoteQueryExecutor.cpp | 3 +++ src/QueryPipeline/RemoteQueryExecutor.h | 2 ++ 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 8a1224b47bd9..e69b785fd2ad 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -93,8 +93,11 @@ ISource::Status RemoteSource::prepare() if (is_async_state) return Status::Async; - if (executor_finished) + if (query_executor->isFinished()) + { + getPort().finish(); return Status::Finished; + } Status status = ISource::prepare(); /// To avoid resetting the connection (because of "unfinished" query) in the @@ -126,7 +129,6 @@ void RemoteSource::work() if (need_drain) { query_executor->finish(); - executor_finished = true; return; } @@ -247,9 +249,7 @@ void RemoteSource::onCancel() noexcept void RemoteSource::onUpdatePorts() { if (getPort().isFinished()) - { query_executor->finish(); - } } diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 7f414ea6d9de..f7c414b7e6f9 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -48,7 +48,6 @@ class RemoteSource final : public ISource private: std::atomic_bool was_query_sent = false; bool need_drain = false; - bool executor_finished = false; bool add_aggregation_info = false; RemoteQueryExecutorPtr query_executor; RowsBeforeStepCounterPtr rows_before_limit; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index fa4794555ee4..e4291f6b93b4 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -778,6 +778,9 @@ void RemoteQueryExecutor::finish() { LockAndBlocker guard(was_cancelled_mutex); + /// To make sure finish is only called once + SCOPE_EXIT({ finished = true; }); + /** If one of: * - nothing started to do; * - received all packets before EndOfStream; diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index e3fb64bb0f7b..d309027d17ff 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -222,6 +222,8 @@ class RemoteQueryExecutor /// return true if parallel replica packet was processed bool processParallelReplicaPacketIfAny(); + bool isFinished() const { return finished; } + private: RemoteQueryExecutor( const String & query_, From 2d4567b16a159e0576c0102a85d46de0c9379b31 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 24 Dec 2025 13:22:32 +0000 Subject: [PATCH 103/113] Backport #92993 to 25.8: More logs in rest catalogs --- src/Databases/DataLake/RestCatalog.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Databases/DataLake/RestCatalog.cpp b/src/Databases/DataLake/RestCatalog.cpp index 6c0e771452d2..19e9ad39a0ca 100644 --- a/src/Databases/DataLake/RestCatalog.cpp +++ b/src/Databases/DataLake/RestCatalog.cpp @@ -529,6 +529,8 @@ DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & bas String json_str; readJSONObjectPossiblyInvalid(json_str, buf); + LOG_DEBUG(log, "Received tables response: {}", json_str); + try { Poco::JSON::Parser parser; @@ -579,9 +581,9 @@ bool RestCatalog::tryGetTableMetadata( { return getTableMetadataImpl(namespace_name, table_name, result); } - catch (...) + catch (const DB::Exception & ex) { - DB::tryLogCurrentException(log); + LOG_DEBUG(log, "tryGetTableMetadata response: {}", ex.what()); return false; } } @@ -625,6 +627,7 @@ bool RestCatalog::getTableMetadataImpl( String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); + LOG_DEBUG(log, "Receiving table metadata {} {}", table_name, json_str); #ifdef DEBUG_OR_SANITIZER_BUILD /// This log message might contain credentials, From 10719e9a7bfa33390231a1a99bf6ffb72caaa42a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 25 Dec 2025 06:16:16 +0000 Subject: [PATCH 104/113] Backport #93017 to 25.8: Fix incorrect granules pruning in `KeyCondition` for `NOT materialize(...)` and `NOT CAST(...)` --- src/Storages/MergeTree/KeyCondition.cpp | 8 +++++ ...ndition_not_materialize_not_cast.reference | 32 ++++++++++++++++++ ...key_condition_not_materialize_not_cast.sql | 33 +++++++++++++++++++ 3 files changed, 73 insertions(+) create mode 100644 tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference create mode 100644 tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 79b03554f63c..a1282c4b2ab7 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -721,11 +721,19 @@ static const ActionsDAG::Node & cloneDAGWithInversionPushDown( { /// Remove "materialize" from index analysis. res = &cloneDAGWithInversionPushDown(*node.children.front(), inverted_dag, inputs_mapping, context, need_inversion); + + /// `need_inversion` was already pushed into the child; avoid adding an extra `not()` wrapper + /// Without this, we could add an extra `not()` here (double inversion), e.g. `NOT materialize(x = 0)` -> `not(notEquals(x, 0))`. + handled_inversion = true; } else if (isTrivialCast(node)) { /// Remove trivial cast and keep its first argument. res = &cloneDAGWithInversionPushDown(*node.children.front(), inverted_dag, inputs_mapping, context, need_inversion); + + /// `need_inversion` was already pushed into the child; avoid adding an extra `not()` wrapper + /// Without this, we could add an extra `not()` here (double inversion), e.g. `NOT CAST(x = 0, 'UInt8')` -> `not(notEquals(x, 0))`. + handled_inversion = true; } else if (need_inversion && (name == "and" || name == "or")) { diff --git a/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference new file mode 100644 index 000000000000..440e3b37c833 --- /dev/null +++ b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference @@ -0,0 +1,32 @@ +1 +2 +Expression (Project names) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + Projection)) + Filter ((WHERE + Change column names to column identifiers)) + ReadFromMergeTree (default.t_cast_bug) + Indexes: + PrimaryKey + Keys: + val + Condition: (val not in [0, 0]) + Parts: 2/3 + Granules: 2/3 + Search Algorithm: generic exclusion search + Ranges: 2 +1 +2 +Expression (Project names) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + (WHERE + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))))))))) + Filter (((WHERE + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))[split]) + ReadFromMergeTree (default.t_materialize_bug) + Indexes: + PrimaryKey + Keys: + val + Condition: (val not in [0, 0]) + Parts: 2/3 + Granules: 2/3 + Search Algorithm: generic exclusion search + Ranges: 2 diff --git a/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.sql b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.sql new file mode 100644 index 000000000000..af7b1466fc30 --- /dev/null +++ b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.sql @@ -0,0 +1,33 @@ +-- Tags: no-replicated-database, no-parallel-replicas, no-parallel, no-random-merge-tree-settings +-- EXPLAIN output may differ + +DROP TABLE IF EXISTS t_cast_bug; + +CREATE TABLE t_cast_bug (val UInt8) ENGINE = MergeTree ORDER BY val; +SYSTEM STOP MERGES t_cast_bug; + +INSERT INTO t_cast_bug VALUES (1); +INSERT INTO t_cast_bug VALUES (0); +INSERT INTO t_cast_bug VALUES (0), (2); + +SELECT val FROM t_cast_bug WHERE NOT CAST(val = 0, 'UInt8') ORDER BY val; + +EXPLAIN indexes=1 +SELECT val FROM t_cast_bug WHERE NOT CAST(val = 0, 'UInt8') ORDER BY val; + + +DROP TABLE IF EXISTS t_materialize_bug; + +CREATE TABLE t_materialize_bug (val UInt8) ORDER BY (val); +CREATE VIEW v AS SELECT val, val = 0 AS is_zero FROM t_materialize_bug; +SYSTEM STOP MERGES t_materialize_bug; + +INSERT INTO t_materialize_bug VALUES (1); +INSERT INTO t_materialize_bug VALUES (0); +INSERT INTO t_materialize_bug VALUES (0), (2); + +SELECT val FROM v WHERE NOT is_zero ORDER BY val; + +EXPLAIN indexes=1 +SELECT val FROM v WHERE NOT is_zero ORDER BY val; + From 0885283de59e7b84fc02542052ab9457add23983 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 25 Dec 2025 08:15:31 +0000 Subject: [PATCH 105/113] Backport #93029 to 25.8: Fix leaving connection in a broken state after preliminary cancellation distributed queries --- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index e4291f6b93b4..bf8d24531fbc 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -778,9 +778,6 @@ void RemoteQueryExecutor::finish() { LockAndBlocker guard(was_cancelled_mutex); - /// To make sure finish is only called once - SCOPE_EXIT({ finished = true; }); - /** If one of: * - nothing started to do; * - received all packets before EndOfStream; @@ -791,6 +788,9 @@ void RemoteQueryExecutor::finish() if (!isQueryPending() || hasThrownException()) return; + /// To make sure finish is only called once + SCOPE_EXIT({ finished = true; }); + /** If you have not read all the data yet, but they are no longer needed. * This may be due to the fact that the data is sufficient (for example, when using LIMIT). */ From 2703844126d823a7456d4c30f8f93d4e11446efd Mon Sep 17 00:00:00 2001 From: "Nihal Z. Miaji" <81457724+nihalzp@users.noreply.github.com> Date: Thu, 25 Dec 2025 17:15:58 +0800 Subject: [PATCH 106/113] Make test compatible --- .../03773_key_condition_not_materialize_not_cast.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference index 440e3b37c833..14044fda5108 100644 --- a/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference +++ b/tests/queries/0_stateless/03773_key_condition_not_materialize_not_cast.reference @@ -13,7 +13,7 @@ Expression (Project names) Parts: 2/3 Granules: 2/3 Search Algorithm: generic exclusion search - Ranges: 2 + Ranges: 2 1 2 Expression (Project names) @@ -29,4 +29,4 @@ Expression (Project names) Parts: 2/3 Granules: 2/3 Search Algorithm: generic exclusion search - Ranges: 2 + Ranges: 2 From 469193f615ce3616ea725c432206480b2a49435c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 29 Dec 2025 02:56:48 +0000 Subject: [PATCH 107/113] Backport #92725 to 25.8: Rework for grant check. Fix check access for wildcard grants. --- src/Access/AccessRights.cpp | 141 ++++++++++++------- src/Access/tests/gtest_access_rights_ops.cpp | 92 ++++++++++++ 2 files changed, 182 insertions(+), 51 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index 558478f97953..312a3ce77f8c 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -366,12 +366,14 @@ struct AccessRights::Node AccessFlags flags; /// flags = (inherited_flags - partial_revokes) | explicit_grants AccessFlags min_flags_with_children; /// min_flags_with_children = access & child[0].min_flags_with_children & ... & child[N-1].min_flags_with_children AccessFlags max_flags_with_children; /// max_flags_with_children = access | child[0].max_flags_with_children | ... | child[N-1].max_flags_with_children - std::unique_ptr children; + std::shared_ptr children; bool wildcard_grant = false; Node() = default; Node(const Node & src) { *this = src; } + Node(Node && src) noexcept = default; + Node & operator=(Node && src) noexcept = default; Iterator begin() const { return ++Iterator{*this}; } Iterator end() const { return Iterator{*this}; } @@ -388,12 +390,25 @@ struct AccessRights::Node min_flags_with_children = src.min_flags_with_children; max_flags_with_children = src.max_flags_with_children; if (src.children) - children = std::make_unique(*src.children); + children = std::make_shared(*src.children); else children = nullptr; return *this; } + Node makeLightCopy() const + { + Node res; + res.node_name = node_name; + res.level = level; + res.flags = flags; + res.wildcard_grant = wildcard_grant; + res.min_flags_with_children = min_flags_with_children; + res.max_flags_with_children = max_flags_with_children; + res.children = children; + return res; + } + bool isLeaf() const { return node_name.empty(); } template @@ -480,6 +495,8 @@ struct AccessRights::Node template bool isGranted(const AccessFlags & flags_, std::string_view name, const Args &... subnames) const { + const auto next_level = static_cast(level + 1); + AccessFlags flags_to_check = flags_ - min_flags_with_children; if (!flags_to_check) return true; @@ -498,16 +515,12 @@ struct AccessRights::Node /// "foo" (SELECT) /// / \ /// "" (leaf, USAGE) "bar" (SELECT) - const auto & [node, _] = tryGetLeafOrPrefix(name, /* return_parent_node= */ true); - /// Check min_flags_with_children because wildcard allows to grant for all children. - return node.min_flags_with_children.contains(flags_to_check); + const auto node = tryGetLeaf(name, next_level, /* return_parent_node= */ true); + return node.isGranted(flags_to_check); } - const auto & [node, final] = tryGetLeafOrPrefix(name); - if (final) - return node.isGranted(flags_to_check, subnames...); - - return node.flags.contains(flags_to_check); + const auto node = tryGetLeaf(name, next_level); + return node.isGranted(flags_to_check, subnames...); } template @@ -521,17 +534,9 @@ struct AccessRights::Node for (const auto & name : names) { - const Node * child = tryGetLeaf(name); - if (child) - { - if (!child->isGranted(flags_to_check, name)) - return false; - } - else - { - if (!flags.contains(flags_to_check)) - return false; - } + Node child = tryGetLeaf(name); + if (!child.isGranted(flags_to_check, name)) + return false; } return true; } @@ -671,7 +676,7 @@ struct AccessRights::Node return *this; if (!children) - children = std::make_unique(); + children = std::make_shared(); auto find_possible_prefix = [path](const Node & n) { @@ -710,7 +715,9 @@ struct AccessRights::Node auto & new_parent = getChildNode(prefix, level_); child.node_name = child.node_name.substr(i); - new_parent.children = std::make_unique(); + new_parent.children = std::make_shared(); + new_parent.min_flags_with_children = it->min_flags_with_children & flags; + new_parent.max_flags_with_children = it->max_flags_with_children | flags; new_parent.children->splice(new_parent.children->begin(), *children, it); return new_parent.getLeaf(new_path, level_, return_parent_node); @@ -720,44 +727,68 @@ struct AccessRights::Node auto & child = getChildNode(path, level_); /// Child is a leaf. - if (path.empty()) + if (path.empty() || return_parent_node) return child; /// Creates a leaf in child. return child.getLeaf("", level_, return_parent_node); } - /// Returns a pair [node, is_leaf]. - std::pair tryGetLeafOrPrefix(std::string_view path, bool return_parent_node = false) const + /// Same logic as `getLeaf`, but const. + /// Using non-move copy on the returned node will make a full copy of all underlying children. + Node tryGetLeaf(std::string_view path, Level level_ = Level::GLOBAL_LEVEL, bool return_parent_node = false) const { - if (!children) - return {*this, false}; + /// The implementation of this function is almost the same as `getLeaf`. + /// Instead of creating new nodes in-place, this function creates ephemeral nodes without modifying the tree structure. + /// If a node already exists, this function returns a lightweight copy of it (without the deep copy of the children) + /// + /// The nodes, returned by this function, don't have ownership of their children (!) + /// Attempting to read or modify `.children` may break the grants tree logic (!) + /// The returned ephemeral nodes shouldn't be used longer than a function scope (!) + if (path.empty() && return_parent_node) + return this->makeLightCopy(); - for (auto & child : *children) + auto find_possible_prefix = [path](const Node & n) { - if (path.empty() && child.isLeaf()) - return {child, true}; + if (path.empty()) + return n.isLeaf(); - if (!child.isLeaf() && path.starts_with(child.node_name)) + return n.node_name[0] == path[0]; + }; + + if (children) + { + if (auto it = std::find_if(children->begin(), children->end(), find_possible_prefix); it != children->end()) { - if (return_parent_node && path == child.node_name) - return {child, true}; + if (path.empty()) + return it->makeLightCopy(); + + auto child = it->makeLightCopy(); + const auto & [left, right] = std::mismatch(path.begin(), path.end(), child.node_name.begin(), child.node_name.end()); + + if (right == child.node_name.end()) + return child.tryGetLeaf(path.substr(child.node_name.size()), level_, return_parent_node); + + /// See `getLeaf` + size_t i = std::distance(path.begin(), left); + std::string_view prefix = path.substr(0, i); + std::string_view new_path = path.substr(i); + + auto new_parent = createChildNode(prefix, level_); + child.node_name = child.node_name.substr(i); + new_parent.children = std::make_shared(); + new_parent.min_flags_with_children = child.min_flags_with_children & flags; + new_parent.max_flags_with_children = child.max_flags_with_children | flags; + new_parent.children->push_back(std::move(child)); - return child.tryGetLeafOrPrefix(path.substr(child.node_name.size())); + return new_parent.tryGetLeaf(new_path, level_, return_parent_node); } } - return {*this, false}; - } - - /// Similar to `getLeaf`, but returns nullptr if no leaf was found. - const Node * tryGetLeaf(std::string_view path, bool return_parent_node = false) const - { - const auto & [node, final] = tryGetLeafOrPrefix(path, return_parent_node); - if (!final) - return nullptr; + if (path.empty() || return_parent_node) + return createChildNode(path, level_); - return &node; + return createChildNode(path, level_).tryGetLeaf("", level_, return_parent_node); } /// Returns a child node with the given name. If no child was found, creates a new one with specified *level_*. @@ -768,16 +799,24 @@ struct AccessRights::Node return *child; if (!children) - children = std::make_unique(); - - auto & new_child = children->emplace_back(); - new_child.node_name = name; - new_child.level = level_; - new_child.flags = flags & new_child.getAllGrantableFlags(); + children = std::make_shared(); + auto & new_child = children->emplace_back(createChildNode(name, level_)); return new_child; } + Node createChildNode(const std::string_view name, const Level level_ = Level::GLOBAL_LEVEL) const + { + Node child; + child.node_name = name; + child.level = level_; + child.flags = flags & child.getAllGrantableFlags(); + child.min_flags_with_children = flags; + child.max_flags_with_children = flags; + + return child; + } + /// Similar to `getChildNode`, but returns nullptr if no child node was found. Node * tryGetChildNode(std::string_view name) const { diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index 86b622cdda85..8b1084b7b852 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -1030,3 +1030,95 @@ TEST(AccessRights, PartialRevokePropagation) ASSERT_TRUE(root.isGranted(AccessType::SELECT, "writeonly")); ASSERT_FALSE(root.isGranted(AccessType::INSERT, "writeonly")); } + +TEST(AccessRights, PartialRevokeIsGrantedWildcard) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.revoke(AccessType::SELECT, "system", "zookeeper"); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "normal")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "system", "query_log")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system", "zookeeper")); + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "system", "zookeeper")); + + // system.zookeeper is revoked, but system.zookeeper2 is not + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "system", "zookeeper2")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "system", "zookeeper2")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "system", "query_log")); + + root = {}; + root.grant(AccessType::SELECT); + root.revokeWildcard(AccessType::SELECT, "system", "zookeeper"); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system", "zookeeper")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "system", "zookeeper2")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "system", "query_log")); +} + +TEST(AccessRights, SamePrefixDifferentLevels) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.revoke(AccessType::SELECT, "test"); + root.revoke(AccessType::SELECT, "prod", "test"); + root.revoke(AccessType::SELECT, "dev", "data", "test"); + + // "test" is revoked + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "test")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "test", "anytable")); + + // "prod.test" is revoked + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "prod")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "prod", "test")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod", "foo")); + + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "prod", "other")); + + // "dev.data.test" is revoked + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "dev", "data", "other")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "dev", "data", "test")); +} + +TEST(AccessRights, IsGrantedWildcardDatabaseLevel) +{ + AccessRights root; + root.grant(AccessType::SELECT, "mydb"); + + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "mydb")); + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "mydb2")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "mydb", "anytable")); + + root = {}; + root.grantWildcard(AccessType::SELECT, "test"); + + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "test")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "testing")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "test123")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "tes")); + + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "test")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::SELECT, "testing")); + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "te")); +} + +TEST(AccessRights, MultipleAccessTypesPartialRevoke) +{ + AccessRights root; + root.grant(AccessType::SELECT); + root.grant(AccessType::INSERT); + root.revoke(AccessType::SELECT, "readonly"); + root.revoke(AccessType::INSERT, "readwrite"); + + // SELECT + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "normal")); + ASSERT_FALSE(root.isGranted(AccessType::SELECT, "readonly")); + ASSERT_TRUE(root.isGranted(AccessType::SELECT, "readwrite")); + + // INSERT + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "normal")); + ASSERT_TRUE(root.isGranted(AccessType::INSERT, "readonly")); + ASSERT_FALSE(root.isGranted(AccessType::INSERT, "readwrite")); + + // Wildcard checks + ASSERT_FALSE(root.isGrantedWildcard(AccessType::SELECT, "readonl")); + ASSERT_TRUE(root.isGrantedWildcard(AccessType::INSERT, "readonl")); +} From f3a7202e348818a2feb21c3e991a1bf44b03bc1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 29 Dec 2025 15:13:21 +0000 Subject: [PATCH 108/113] Backport #93130 to 25.8: tests: fix test_distributed_insert_select_to_rmt_where flakiness (due to deduplication) --- tests/integration/test_s3_cluster_insert_select/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_s3_cluster_insert_select/test.py b/tests/integration/test_s3_cluster_insert_select/test.py index 7535640f57a4..dd53e93e6f4c 100644 --- a/tests/integration/test_s3_cluster_insert_select/test.py +++ b/tests/integration/test_s3_cluster_insert_select/test.py @@ -208,7 +208,11 @@ def test_distributed_insert_select_to_rmt_where(started_cluster): INSERT INTO {table} SELECT * FROM s3Cluster( '{cluster_name}', 'http://minio1:9001/root/data/generated/*.csv', 'minio', '{minio_secret_key}', 'CSV','a String, b UInt64' - ) WHERE b = 100 SETTINGS parallel_distributed_insert_select=2; + ) WHERE b = 100 + SETTINGS + parallel_distributed_insert_select=2, + -- disable deduplication since all rows identical, and if the batch size will be the same size on different nodes it will be deduplicated + insert_deduplicate=0; """ ) From 3d1edb07b60d87e732f988d3aad58d6118b4f470 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 29 Dec 2025 19:12:09 +0000 Subject: [PATCH 109/113] Backport #93016 to 25.8: Fix possible error FILE_DOESNT_EXIST after sparse column mutation --- src/Storages/MergeTree/MutateTask.cpp | 12 ++++++++++-- ..._bad_sparse_cration_with_altered_ration.reference | 10 ++++++++++ ...tation_bad_sparse_cration_with_altered_ration.sql | 9 +++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.reference create mode 100644 tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.sql diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 7ca173420a23..ba505319c03a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -336,12 +336,20 @@ static void splitAndModifyMutationCommands( || command.type == MutationCommand::Type::MATERIALIZE_PROJECTION || command.type == MutationCommand::Type::MATERIALIZE_TTL || command.type == MutationCommand::Type::DELETE - || command.type == MutationCommand::Type::UPDATE || command.type == MutationCommand::Type::APPLY_DELETED_MASK || command.type == MutationCommand::Type::APPLY_PATCHES) { for_interpreter.push_back(command); } + else if (command.type == MutationCommand::Type::UPDATE) + { + for_interpreter.push_back(command); + + /// Update column can change the set of substreams for column if it + /// changes serialization (for example from Sparse to not Sparse). + /// We add it "for renames" because these set of commands also removes redundant files + for_file_renames.push_back(command); + } else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION || command.type == MutationCommand::Type::DROP_STATISTICS) @@ -980,7 +988,7 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX)) add_rename(STATS_FILE_PREFIX + command.column_name + STATS_FILE_SUFFIX, STATS_FILE_PREFIX + command.rename_to + STATS_FILE_SUFFIX); } - else if (command.type == MutationCommand::Type::READ_COLUMN || command.type == MutationCommand::Type::MATERIALIZE_COLUMN) + else if (command.type == MutationCommand::Type::UPDATE || command.type == MutationCommand::Type::READ_COLUMN || command.type == MutationCommand::Type::MATERIALIZE_COLUMN) { /// Remove files for streams that exist in source_part, /// but were removed in new_part by MODIFY COLUMN or MATERIALIZE COLUMN from diff --git a/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.reference b/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.reference new file mode 100644 index 000000000000..7f1bc308d222 --- /dev/null +++ b/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.reference @@ -0,0 +1,10 @@ +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.sql b/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.sql new file mode 100644 index 000000000000..3bdd73631fed --- /dev/null +++ b/tests/queries/0_stateless/03778_mutation_bad_sparse_cration_with_altered_ration.sql @@ -0,0 +1,9 @@ +drop table if exists test; +create table test (a UInt32, b UInt32) engine=MergeTree order by tuple() settings min_bytes_for_wide_part=0, ratio_of_defaults_for_sparse_serialization=0.0, max_suspicious_broken_parts=0, max_suspicious_broken_parts_bytes=0; +insert into test select number, number from numbers(10); +alter table test modify setting ratio_of_defaults_for_sparse_serialization=1.0; +alter table test update b = 0 where 1 settings mutations_sync=2; +detach table test; +attach table test; +select * from test; + From 738d848fcabd312082cbec1b4630412e14594e12 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 30 Dec 2025 13:24:49 +0000 Subject: [PATCH 110/113] Backport #92842 to 25.8: Fix creating type hint for path 'skip' in JSON --- src/DataTypes/DataTypeObject.cpp | 22 +++++++++++---- src/Parsers/ASTObjectTypeArgument.cpp | 28 +++++++++++++++++++ src/Parsers/ASTObjectTypeArgument.h | 17 +++++++++++ src/Parsers/ParserDataType.cpp | 4 +-- .../03773_json_with_typed_path_skip.reference | 1 + .../03773_json_with_typed_path_skip.sql | 2 ++ 6 files changed, 67 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03773_json_with_typed_path_skip.reference create mode 100644 tests/queries/0_stateless/03773_json_with_typed_path_skip.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 923ee858e1e2..58929b99daa6 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include "config.h" @@ -202,7 +203,11 @@ String DataTypeObject::doGetName() const for (const auto & path : sorted_typed_paths) { write_separator(); - out << backQuoteIfNeed(path) << " " << typed_paths.at(path)->getName(); + /// We must quote path "SKIP" to avoid its confusion with SKIP keyword. + if (boost::to_upper_copy(path) == "SKIP") + out << backQuote(path) << " " << typed_paths.at(path)->getName(); + else + out << backQuoteIfNeed(path) << " " << typed_paths.at(path)->getName(); } std::vector sorted_skip_paths; @@ -489,11 +494,18 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject:: } else if (object_type_argument->path_with_type) { - const auto * path_with_type = object_type_argument->path_with_type->as(); + const auto * path_with_type = object_type_argument->path_with_type->as(); auto data_type = DataTypeFactory::instance().get(path_with_type->type); - if (typed_paths.contains(path_with_type->name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found duplicated path with type: {}", path_with_type->name); - typed_paths.emplace(path_with_type->name, data_type); + if (typed_paths.contains(path_with_type->path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found duplicated path with type: {}", path_with_type->path); + + for (const auto & [path, _] : typed_paths) + { + if (path.starts_with(path_with_type->path + ".") || path_with_type->path.starts_with(path + ".")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found incompatible typed paths: {} and {}. One of them is a prefix of the other", path, path_with_type->path); + } + + typed_paths.emplace(path_with_type->path, data_type); } else if (object_type_argument->skip_path) { diff --git a/src/Parsers/ASTObjectTypeArgument.cpp b/src/Parsers/ASTObjectTypeArgument.cpp index c8aaf950deb1..c33e8050dcfc 100644 --- a/src/Parsers/ASTObjectTypeArgument.cpp +++ b/src/Parsers/ASTObjectTypeArgument.cpp @@ -1,11 +1,39 @@ #include #include #include +#include +#include namespace DB { +ASTPtr ASTObjectTypedPathArgument::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (type) + { + res->type = type->clone(); + res->children.push_back(res->type); + } + + return res; +} + +void ASTObjectTypedPathArgument::formatImpl( + WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + /// We must quote path "SKIP" to avoid its confusion with SKIP keyword in Object arguments. + if (boost::to_upper_copy(path) == "SKIP") + ostr << backQuote(path) << ' '; + else + ostr << backQuoteIfNeed(path) << ' '; + + type->format(ostr, settings, state, frame); +} + ASTPtr ASTObjectTypeArgument::clone() const { auto res = std::make_shared(*this); diff --git a/src/Parsers/ASTObjectTypeArgument.h b/src/Parsers/ASTObjectTypeArgument.h index 19db28b03989..0ddd353c3abb 100644 --- a/src/Parsers/ASTObjectTypeArgument.h +++ b/src/Parsers/ASTObjectTypeArgument.h @@ -6,6 +6,23 @@ namespace DB { +/// A pair of Object path and its data type. For example: a.b.c String. +class ASTObjectTypedPathArgument : public IAST +{ +public: + /// path + String path; + /// type + ASTPtr type; + + /** Get the text that identifies this element. */ + String getID(char delim) const override { return "ObjectTypedPath" + (delim + path); } + ASTPtr clone() const override; + +protected: + void formatImpl(WriteBuffer & ostr, const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + /** An argument of Object data type declaration (for example for JSON). Can contain one of: * - pair (path, data type) * - path that should be skipped diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index cea7d14e0c06..3b351073d68b 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -110,8 +110,8 @@ class ObjectArgumentParser : public IParserBase if (!type_parser.parse(pos, type, expected)) return false; - auto name_and_type = std::make_shared(); - name_and_type->name = getIdentifierName(identifier); + auto name_and_type = std::make_shared(); + name_and_type->path = getIdentifierName(identifier); name_and_type->type = type; name_and_type->children.push_back(name_and_type->type); argument->path_with_type = name_and_type; diff --git a/tests/queries/0_stateless/03773_json_with_typed_path_skip.reference b/tests/queries/0_stateless/03773_json_with_typed_path_skip.reference new file mode 100644 index 000000000000..eac58cb4f413 --- /dev/null +++ b/tests/queries/0_stateless/03773_json_with_typed_path_skip.reference @@ -0,0 +1 @@ +{"skip":"Hello"} diff --git a/tests/queries/0_stateless/03773_json_with_typed_path_skip.sql b/tests/queries/0_stateless/03773_json_with_typed_path_skip.sql new file mode 100644 index 000000000000..a0ad48e00d56 --- /dev/null +++ b/tests/queries/0_stateless/03773_json_with_typed_path_skip.sql @@ -0,0 +1,2 @@ +select '{"skip" : "Hello"}'::JSON(`skip` Nullable(String)); + From f632419848ff53345d78a01c8e942896a2a0a2a6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 2 Jan 2026 11:13:07 +0000 Subject: [PATCH 111/113] Backport #93233 to 25.8: Remove unused columns when rebuilding projection on merge. --- src/Storages/MergeTree/MergeTask.cpp | 6 +++- .../__init__.py | 0 .../configs/insert_limits.xml | 8 +++++ .../test.py | 30 +++++++++++++++++++ 4 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_projection_rebuild_with_required_columns/__init__.py create mode 100644 tests/integration/test_projection_rebuild_with_required_columns/configs/insert_limits.xml create mode 100644 tests/integration/test_projection_rebuild_with_required_columns/test.py diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 900910989d32..af064aa83ddd 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -892,7 +892,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::calculateProjections(const Blo for (size_t i = 0, size = global_ctx->projections_to_rebuild.size(); i < size; ++i) { const auto & projection = *global_ctx->projections_to_rebuild[i]; - Block block_to_squash = projection.calculate(block, global_ctx->context); + Block block_with_required_columns; + for (const auto & name : projection.getRequiredColumns()) + if (name != "_part_offset") + block_with_required_columns.insert(block.getByName(name)); + Block block_to_squash = projection.calculate(block_with_required_columns, global_ctx->context); /// Avoid replacing the projection squash header if nothing was generated (it used to return an empty block) if (block_to_squash.rows() == 0) return; diff --git a/tests/integration/test_projection_rebuild_with_required_columns/__init__.py b/tests/integration/test_projection_rebuild_with_required_columns/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_projection_rebuild_with_required_columns/configs/insert_limits.xml b/tests/integration/test_projection_rebuild_with_required_columns/configs/insert_limits.xml new file mode 100644 index 000000000000..18c9e18d779b --- /dev/null +++ b/tests/integration/test_projection_rebuild_with_required_columns/configs/insert_limits.xml @@ -0,0 +1,8 @@ + + + + 1048576 + 10000 + + + diff --git a/tests/integration/test_projection_rebuild_with_required_columns/test.py b/tests/integration/test_projection_rebuild_with_required_columns/test.py new file mode 100644 index 000000000000..27728c75b7b7 --- /dev/null +++ b/tests/integration/test_projection_rebuild_with_required_columns/test.py @@ -0,0 +1,30 @@ +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance("node1", user_configs=["configs/insert_limits.xml"]) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_projection_rebuild_uses_only_required_columns(started_cluster): + node1.query("create table tab (x UInt64, y UInt64, data String codec(NONE), v UInt8, projection p (select _part_offset order by y)) engine = ReplacingMergeTree(v) order by x settings allow_part_offset_column_in_projections=1, deduplicate_merge_projection_mode='rebuild';") + node1.query("insert into tab select number, number, rightPad('', 100, 'a'), 0 from numbers(30000);") + node1.query("optimize table tab final settings mutations_sync=2, alter_sync=2;") + node1.query("system flush logs;") + + uuid = node1.query("select uuid from system.tables where table = 'tab';").strip() + cnt = node1.query("select count() from system.text_log where query_id = '{}::all_1_1_2' and message like '%Reading%from part p_%from the beginning of the part%'".format(uuid)) + assert (cnt == '2\n') + From 9e7dfe695b614560494231e5adacceace7acbe02 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 4 Jan 2026 12:19:14 +0000 Subject: [PATCH 112/113] Backport #92888 to 25.8: Fallback to read & write for native copy fails --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 7e584ab59e14..d319b1550ee1 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -431,6 +431,12 @@ void copyAzureBlobStorageFile( "Will attempt to copy using read & write. source container = {} blob = {} and destination container = {} blob = {}", src_container_for_logging, src_blob, dest_container_for_logging, dest_blob); } + else if (e.StatusCode == Azure::Core::Http::HttpStatusCode::BadRequest) + { + LOG_TRACE(log, "Copy operation has thrown bad argument error. e.what = {}. " + "Will attempt to copy using read & write. source container = {} blob = {} and destination container = {} blob = {}", + e.what(), src_container_for_logging, src_blob, dest_container_for_logging, dest_blob); + } else throw; } From fc3dda70eef7fd4c7279d9d76b5877762fcd79a1 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Wed, 14 Jan 2026 18:32:56 +0100 Subject: [PATCH 113/113] fix dt64 inference --- src/IO/ReadHelpers.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 5b7779d23be9..fc2864545fa1 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1156,8 +1156,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } else { - auto ok = readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow); - if (!ok && (buf.eof() || *buf.position() != '.')) + if (!readDateTimeTextImpl(whole, buf, date_lut, allowed_date_delimiters, allowed_time_delimiters, saturate_on_overflow)) return ReturnType(false); } }