Skip to main content
Skip to main content

v25.6 Changelog for Cloud

Backward incompatible change

  • Function geoToH3() now accepts the input in the order (lat, lon,res) (which is the standard order for geometric functions). Users who wish to retain the legacy result order (lon, lat,res) can set setting geotoh3_lon_lat_input_order = true. #78852 (Pratima Patel).
  • Indexes of type full_text were renamed to gin. This follows the more familiar terminology of PostgreSQL and other databases. Existing indexes of type full_text remain loadable but they will throw an exception (suggesting gin indexes instead) when one tries to use them in searches. #79024 (Robert Schulze).
  • Add a filesystem cache setting allow_dynamic_cache_resize, by default false, to allow dynamic resize of filesystem cache. Why: in certain environments (ClickHouse Cloud) all the scaling events happen through the restart of the process and we would love this feature to be explicitly disabled to have more control over the behavior + as a safety measure. This PR is marked as backward incompatible, because in older versions dynamic cache resize worked by default without special setting. #79148 (Kseniia Sumarokova).
  • Removed support for legacy index types annoy and usearch. Both have been stubs for a long time, i.e. every attempt to use the legacy indexes returned an error anyways. If you still have annoy and usearch indexes, please drop them. #79802 (Robert Schulze). #* Remove format_alter_commands_with_parentheses server setting. The setting was introduced and disabled by default in 24.2. It was enabled by default in 25.2. As there are no LTS versions that don't support the new format, we can remove the setting. #79970 (János Benjamin Antal). #* Minor: Force backup_threads and restore_threads server settings to be non zero. #80224 (Raúl Marín).
  • Fix bitNot() for String to return a zero-terminated string. #80791 (Azat Khuzhin).

New feature

  • Add a new option to MergeTree SETTINGS which specifies a default compression codec in case the CREATE query does not explicitly define one for the given columns. This closes #42005. #66394 (gvoelfin).
  • Follow up for https://github.com/ClickHouse/ClickHouse/pull/71943. This PR implements Time/Time64 data types. Implements new data types: Time (HHH:MM:SS) and Time64 (HHH:MM:SS.<fractional>), some basic cast functions and functions to interact with other data types. Also, changed the existing function's name toTime to toTimeWithFixedDate because the function toTime is required for the cast function. #75735 (Yarik Briukhovetskyi).
  • Support correlated subqueries as an argument of EXISTS expression in the WHERE clause. Closes #72459. #76078 (Dmitry Novik).
  • Allow writing to Merge table engines. #77484 (Anton Ivashkin).
  • Distributed INSERT SELECT for replicated MergeTree tables now efficiently uses parallel replicas to parallelize INSERTs by selecting different data on different nodes and inserting them independently. #78041 (Igor Nikonov).
  • Add mapContainsValuesLike/mapContainsValues/mapExtractValuesLike functions to filter on map values and their support in bloomfilter based indexes. #78171 (UnamedRus).
  • Add system.iceberg_history table. #78244 (Smita Kulkarni).
  • Added query slot scheduling for workloads, see https://clickhouse.com/docs/operations/workload-scheduling#query_scheduling for details. #78415 (Sergei Trifonov).
  • Add getServerSetting and getMergeTreeSetting function. Closes https://github.com/clickhouse/clickhouse/issues/78318. #78439 (NamNguyenHoai).
  • Support disallowed values under settings constraints. #78499 (Bharat Nallan).
  • Add new iceberg_enable_version_hint setting to leverage version-hint.text file. #78594 (Arnaud Briche).
  • Gives the possibility to truncate specific tables from a database, filtered with the LIKE keyword. #78597 (Yarik Briukhovetskyi).
  • clickhouse-local (and its shorthand alias, ch) now use an implicit FROM table when there is input data for processing. This closes #65023. Also enabled format inference in clickhouse-local if --input-format is not specified and it processes a regular file. #79085 (Alexey Milovidov).
  • Add icebergHash and icebergBucketTransform functions. Support data files pruning in Iceberg tables partitioned with bucket transfom. #79262 (Daniil Ivanik).
  • Add a support for Coalescing Merge Tree. This closes #78869. #79344 (Konstantin Vedernikov).
  • Add stringBytesUniq and stringBytesEntropy functions to search for possibly random or encrypted data. #79350 (Sachin Kumar Singh).
  • Support _part_starting_offset virtual column in MergeTree-family tables. This column represents the cumulative row count of all preceding parts, calculated at query time based on the current part list. The cumulative values are retained throughout query execution and remain effective even after part pruning. Related internal logic has been refactored to support this behavior. #79417 (Amos Bird).
  • Added a setting enable_shared_storage_snapshot_in_query to enable sharing the same storage snapshot across all subqueries in a single query. This ensures consistent reads from the same table, even when the table is referenced multiple times within a query. #79471 (Amos Bird).
  • Support writing CH JSON columns to Parquet and reading Parquet JSON columns directly as CH JSON columns. #79649 (Nihal Z. Miaji).
  • Bundle chdig - TUI interface for ClickHouse (top like) as part of ClickHouse. #79666 (Azat Khuzhin).
  • Add MultiPolygon support for pointInPolygon. #79773 (Nihal Z. Miaji).
  • Support geo parquet. This closes #75317. #79777 (Konstantin Vedernikov).
  • Add support for querying local filesystem-mounted delta tables via deltaLakeLocal table function. #79781 (roykim98).
  • Added base32 encode/decode functionality. #79809 (Joanna Hulboj).
  • Clickhouse vector search now supports both pre-filtering and post-filtering and provides related settings for finer control. (issue #78161). #79854 (Shankar Iyer).
  • Support functions to read WKB format. This partially closes #43941. #80139 (Konstantin Vedernikov).
  • Add new setting cast_string_to_date_time_mode that allows to choose DateTime parsing mode during cast from String. #80210 (Pavel Kruglov).
  • Added Bech32 and Bech32m encoding and decoding functions (issue #40381). #80239 (George Larionov).
  • Support disk setting for Atomic and Ordinary DB engines, specifying the disk to store table metadata files. #80546 (Tuan Pham Anh).
  • Support functions to unpack and compare merge tree parts. #80573 (Mikhail Artemenko).
  • timeSeries* helper functions to speed up some scenarios when working with time series data: - re-sample the data to the time grid with specified start timestamp, end timestamp and step - calculate PromQL-like delta, rate, idelta and irate. #80590 (Alexander Gololobov).
  • Allow filtering of parts selected for querying by the disk they reside on. #80650 (tanner-bruce).
  • Add a landing page with the list of embedded web tools. It will open when requested by a browser-like user agent. #81129 (Alexey Milovidov).
  • Allow for filtering NULL values in arrayFirst, arrayFirstIndex, arrayLast & arrayLastIndex. Fixes #81113. #81197 (Lennard Eijsackers).

Experimental feature

  • Hive metastore catalog for iceberg datalake. #77677 (Konstantin Vedernikov).
  • The explicit parameters are supported via key-value pairs. Currently, supported parameters are a mandatory tokenizer and two optional max_rows_per_postings_list and ngram_size. #80262 (Elmi Ahmadov).
  • Experimental indexes of type gin were renamed to text. Existing indexes of type gin remain loadable but they will throw an exception (suggesting text indexes instead) when one tries to use them in searches. #80855 (Robert Schulze).

Performance improvement

  • Speed up secondary indices by evaluating their expressions on multiple granules at once. #64109 (Alexey Milovidov).
  • Introduced threshold (regulated by setting parallel_hash_join_threshold) to fall back to the hash algorithm when the size of the right table is below the threshold. #76185 (Nikita Taranov).
  • The existing implementation of Pipe::resize creates a single Resize or StrictResize node by inserting it into the pipeline topology, which then acts as a central hub connecting all input streams (upstream nodes) to a unified set of output streams (downstream nodes). This design leads to contention for the ExecutingGraph::Node::status_mutex during pipeline graph execution, especially in high-core-count environments. When pipelines scale to tens or hundreds of streams, this contention results in:. #77562 (Zhiguo Zhou).
  • Improve performance of S3Queue/AzureQueue by allowing INSERTs data in parallel (can be enabled with parallel_inserts=true queue setting). Previously S3Queue/AzureQueue can only do the first part of the pipeline in parallel (downloading, parsing), INSERT was single-threaded. And INSERTs are almost always the bottleneck. Now it will scale almost linear with processing_threads_num. #77671 (Azat Khuzhin).
  • Change the compact part format to save marks for each substream to be able to read individual sub-columns. The old compact format is still supported for reads and can be enabled for writes using MergeTree setting write_marks_for_substreams_in_compact_parts. It's disabled by default for safer upgrades as it changes the compact parts storage. It will be enabled by default in one of the next releases. #77940 (Pavel Kruglov).
  • Introduced new settinguse_skip_indexes_in_final_exact_mode. If a query on a ReplacingMergeTree table has the FINAL clause, reading only table ranges based on skip indexes may produce incorrect result. This setting can ensure that correct results are returned by scanning newer parts that have overlap with primary key ranges returned by the skip index. Set to 0 to disable, 1 to enable. #78350 (Shankar Iyer).
  • Now we use the number of replicas to determine the task size for reading with parallel replicas enabled. This provides better work distribution between replicas when the amount of data to read is not very large. #78695 (Nikita Taranov).
  • Allow parallel merging of uniqExact states during the final stage of distributed aggregation. #78703 (Nikita Taranov).
  • Fix possible performance degradation of the parallel merging of uniqExact states for aggregation with key. #78724 (Nikita Taranov). #* Replace DELETE FROM ... WHERE 1 queries to TRUNCATE. (Reverted). #78739 (Konstantin Vedernikov).
  • Reduce the number of List Blobs API calls to Azure storage. #78860 (Julia Kartseva).
  • Merge equality conditions from filter query plan step into JOIN condition if possible to allow using them as hash table keys. #78877 (Dmitry Novik).
  • Improve performance of hive path parsing by using extractKeyValuePairs instead of regex. #79067 (Arthur Passos).
  • Fix performance of the distributed INSERT SELECT with parallel replicas. #79441 (Azat Khuzhin).
  • Allow moving conditions with sub-columns to PREWHERE. #79489 (Pavel Kruglov).
  • Performance improvements to all bloom filter types. #79800 (Delyan Kratunov).
  • Prevent LogSeriesLimiter from doing cleanup on every construction, avoiding lock contention and performance regressions in high-concurrency scenarios. #79864 (filimonov).
  • Enable compile_expressions (JIT compiler for fragments of ordinary expressions) by default. This closes #51264 and #56386 and #66486. #79907 (Alexey Milovidov).
  • Speedup queries with trivial count optimization. #79945 (Raúl Marín).
  • Introduced a happy path in UniqExactSet::merge when one of the sets is empty. Also, now if the LHS set is two-level and the RHS is single-level, we won't do the conversion to two-level for the RHS. #79971 (Nikita Taranov).
  • Add __attribute__((always_inline)) to convertDecimalsImpl. #79999 (Konstantin Bogdanov).
  • Set input_format_parquet_bloom_filter_push_down to true by default. Also, fix a mistake in the settings changes history. #80058 (Alexey Milovidov). #* Make logging asynchronous by default. You can disable this by setting <async>false</async> under <logger>. #80125 (Raúl Marín).
  • Improve memory reuse efficiency and reduce page faults when using the two-level hash tables. #80245 (Jiebin Sun).
  • Avoid unnecessary update and reduce lock contention in QueryConditionCache. #80247 (Jiebin Sun).
  • Small optimization of concatenateBlocks, which could be good for parallel Hash join. #80328 (李扬).
  • When selecting mark ranges from the primary key range, binary search cannot be used if the primary key is wrapped with functions. This PR improves this limitation: binary search can still be applied when the primary key is wrapped with an always monotonic function chain, or when the RPN contains an element that is always true. This PR closes #45536. #80597 (zoomxi).
  • Improve the shutdown speed of Kafka engine (remove an extra 3 seconds delay in case of multiple Kafka tables). #80796 (Azat Khuzhin).
  • Reduce memory usage of async inserts and improve the performance of insert queries. #80972 (Raúl Marín).
  • Don't profile processors if the log table is disabled. #81256 (Raúl Marín).
  • Speed up toFixedString when the source is exactly what's requested. #81257 (Raúl Marín).
  • Don't process quota values if the user is not limited. #81549 (Raúl Marín).
  • Make ProcfsMetricsProvider thread_local to keep files open between tasks. #81576 (Raúl Marín).
  • Fixed performance regression in memory tracking. #81694 (Michael Kolupaev).

Improvement

#* clickhouse-local will retain its databases after restart if you specify the --path command line argument. This closes #50647. This closes #49947. #71722 (Alexey Milovidov).

  • EXPLAIN SYNTAX now uses a new analyzer. It returns an abstract syntax tree (AST) built from the query tree. Added option query_tree_passes to control the number of passes to be executed before converting the query tree to the AST. #74536 (Vladimir Cherkasov). #* Use SLRU cache policy in filesystem cache by default. #75072 (Kseniia Sumarokova).
  • Refactored the logic of pushing to views. #77309 (Sema Checherinda).
  • Object storage cluster table functions (e.g. s3Cluster) will now assign files to replicas for reading based on consistent hash to improve cache locality. #77326 (Andrej Hoos).
  • Refresh S3 credentials after error AuthenticationRequired. #77353 (Vitaly Baranov).
  • Embed proxy configuration in some HTTP buffers with the help of builders. #77693 (Arthur Passos).
  • Added dictionary metrics to system.asynchronous_metrics - DictionaryMaxUpdateDelay - The maximum delay(in seconds) of dictionary update. - DictionaryTotalFailedUpdates - Number of errors since last successful loading in all dictionaries. #78175 (Vlad).
  • Add functions divideOrNull,moduloOrNull, intDivOrNull,positiveModuloOrNull to return NULL when right argument is zero. #78276 (kevinyhzou).
  • Extend the isIPAddressInRange function to String, IPv4, IPv6, Nullable(String) Nullable(IPv4) and Nullable(IPv6) data types. #78364 (YjyJeff).
  • Change PostgreSQL engine connection pooler settings dynamically. #78414 (Samay Sharma).
  • Allow to specify _part_offset in a normal projection. This is the first step to build a projection index. It can be used with #58224 and can help improve https://github.com/ClickHouse/ClickHouse/pull/63207. #78429 (Amos Bird).
  • Improve the sharding key optimization on distributed queries. #78452 (fhw12345).
  • Add new columns(create_query and source) for system.named_collections. Closes #78179. #78582 (MikhailBurdukov).
  • Added field condition to system table system.query_condition_cache. It stores the plaintext condition whose hash is used as a key in the query condition cache. #78671 (Robert Schulze).
  • Implement Kafka re-balance like logic for StorageKafka2 using ClickHouse Keeper. For each replica we support two types of partition locks: permanent locks and temporary locks. The replica tries to hold permanent locks as long as possible, at any given time there are no more than all_topic_partitions / active_replicas_count (here all_topic_partitions is the number of all partitions, active_replicas_count is the number of active replicas) permanent locks on the replica, if there are more, then the replica releases some partitions. Some partitions are temporarily held by the replica. The maximum number of temporary locks on a replica changes dynamically to give other replicas a chance to take some partitions into permanent locks. When updating temporary locks, the replica releases them all and tries to take some others again. #78726 (Daria Fomina).
  • Add table settings for SASL configuration and credentials to the Kafka table engine. This allows configuring SASL-based authentication to Kafka and Kafka-compatible systems directly in the CREATE TABLE statement rather than having to use configuration files or named collections. #78810 (Christoph Wurm).
  • Add a warning about databases that were potentially created to save broken tables. #78841 (János Benjamin Antal).
  • Vector similarity indexes can now be created on top of BFloat16 columns. #78850 (Robert Schulze).
  • Support unix timestamps with a fractional part in best-effort DateTime64 parsing. #78908 (Pavel Kruglov).
  • In storage DeltaLake delta-kernel implementation fix for columnMappingMode.name, add tests for schema evolution. #78921 (Kseniia Sumarokova).
  • Improve insert into Variant column in Values format by better conversion of values. #78923 (Pavel Kruglov).
  • Add _time virtual column in S3Queue engine. #78926 (Anton Ivashkin).
  • The tokens function was extended to accept an additional "tokenizer" argument plus further tokenizer-specific arguments. #79001 (Elmi Ahmadov).
  • The SHOW CLUSTER statement now expands macros (if any) in its argument. #79006 (arf42).
  • Hash functions now support NULLs inside arrays, tuples, and maps. (issues #48365 and #48623). #79008 (Michael Kolupaev).
  • Support for a refresh in read-only MergeTree tables. #79033 (Alexey Milovidov). #* Update cctz to 2025a. #79043 (Raúl Marín).
  • Make settings controlling connection drop on overloaded CPU hot-reloadable. #79052 (Alexey Katsman).
  • It's better for usability. #79066 (Alexey Milovidov).
  • Enable the query condition cache by default. #79080 (Alexey Milovidov). #* Make tabs undo-able in the Web UI. This closes #71284. #79084 (Alexey Milovidov).
  • Remove settings during recoverLostReplica the same as was done in https://github.com/ClickHouse/ClickHouse/pull/78637. #79113 (Nikita Mikhaylov).
  • Add ProfileEvents ParquetReadRowGroups and ParquetPrunedRowGroups to profile parquet index prune. #79180 (flynn).
  • Add container prefix to data paths reported in system.tables for plain disks in Azure blob storage, making reporting consistent with S3 and GCP. #79241 (Julia Kartseva).
  • Support altering a database on cluster. #79242 (Tuan Pham Anh).
  • Explicitly skip missed runs of statistics collection for QueryMetricLog, otherwise the log will take a long time to catch up with the current time. #79257 (Mikhail Artemenko).
  • Added an ability to apply lightweight deletes on the fly (with settings lightweight_deletes_sync = 0, apply_mutations_on_fly = 1. #79281 (Anton Popov).
  • Optimized ALTER ... DELETE mutations for parts in which all rows should be deleted. Now, in such cases an empty part is created instead of original without executing a mutation. #79307 (Anton Popov).
  • Some small optimizations to CHColumnToArrowColumn. #79308 (Bharat Nallan).
  • The setting allow_archive_path_syntax was marked as experimental by mistake. Add a test to prevent having experimental settings enabled by default. #79320 (Alexey Milovidov).
  • Made page cache settings adjustable on a per-query level. This is needed for faster experimentation and for the possibility of fine-tuning for high-throughput and low-latency queries. #79337 (Alexey Milovidov).
  • Do not print number tips in pretty formats for numbers that look like most of the 64-bit hashes. This closes #79334. #79338 (Alexey Milovidov).
  • If data in the pretty format is displayed in the terminal, and a subsequent block has the same column widths, it can continue from the previous block, glue it to the previous block by moving the cursor up. This closes #79333. The feature is controlled by the new setting, output_format_pretty_glue_chunks. #79339 (Alexey Milovidov).
  • Colors of graphs on the advanced dashboards will be calculated from the hash of the corresponding query. This makes it easier to remember and locate a graph while scrolling the dashboard. #79341 (Alexey Milovidov).
  • Add asynchronous metric, FilesystemCacheCapacity - total capacity in the cache virtual filesystem. This is useful for global infrastructure monitoring. #79348 (Alexey Milovidov).
  • Optimize access to system.parts (read columns/indexes size only when requested). #79352 (Azat Khuzhin).
  • Select important fields for query 'SHOW CLUSTER <name>' instead of all fields. #79368 (Tuan Pham Anh).
  • Allow to specify storage settings for DatabaseCatalog. #79407 (Kseniia Sumarokova).
  • Support local storage in delta kernel. #79416 (Kseniia Sumarokova).
  • Add a query level setting to enable delta-kernel-rs: allow_experimental_delta_kernel_rs. #79418 (Kseniia Sumarokova).
  • Fix possible endless loop when listing blobs from Azure/S3 blob storage. #79425 (Alexander Gololobov).
  • Now, ClickHouse also accepts query parameters as param-<name> (dash) along with param_<name> (underscore). This closes #63093. #79429 (Engel Danila). #* Add filesystem cache setting max_size_ratio_to_total_space. #79460 (Kseniia Sumarokova).
  • Detailed warning msg for bandwidth discount when copying data from local to remote S3 with checksum enabled. #79464 (VicoWu).
  • For clickhouse-benchmark reconfigure reconnect option to take 0, 1 or N as values for reconnecting accordingly. #79465 (Sachin Kumar Singh).
  • Add setting input_format_max_block_size_bytes to limit blocks created in input formats in bytes. It can help to avoid high memory usage during data import when rows contains large values. #79495 (Pavel Kruglov).
  • Enhance sparseGrams speed and memory usage. #79517 (Konstantin Vedernikov).
  • Avoid extra copying of the block during insertion into Compact part when possible. #79536 (Pavel Kruglov).
  • Enable DeltaLake storage delta-kernel implementation by default. #79541 (Kseniia Sumarokova).
  • If reading from an URL involves multiple redirects, setting enable_url_encoding is correctly applied across all redirects in the chain. #79563 (Shankar Iyer).
  • Allow ALTER TABLE ... MOVE|REPLACE PARTITION for tables on different plain_rewritable disks. #79566 (Julia Kartseva).
  • Support scalar correlated subqueries in the WHERE clause. Closes #6697. #79600 (Dmitry Novik).
  • Previously when input_format_parquet_max_block_size = 0 ClickHouse would stuck. Now this behaviour is fixed. This closes #79394. #79601 (abashkeev).
  • Add throw_on_error setting for startup_scripts: when throw_on_error is true, the server will not start unless all queries complete successfully. By default, throw_on_error is false, preserving the previous behavior. #79732 (Aleksandr Musorin).
  • The vector similarity index is now also used if the reference vector is of type Array(BFloat16). #79745 (Shankar Iyer).
  • Add last_error_message, last_error_trace and query_id to the system.error_log table. Related ticket #75816. #79836 (Andrei Tinikov). #* Enable sending crash reports by default. This can be turned off in the server's configuration file. #79838 (Alexey Milovidov).
  • System table system.functions now shows in which ClickHouse version functions were first introduced. #79839 (Robert Schulze).
  • Added access_control_improvements.enable_user_name_access_type setting. This setting allows enabling/disabling of precise grants for users/roles, introduced in https://github.com/ClickHouse/ClickHouse/pull/72246. You may want to turn this setting off in case you have a cluster with the replicas older than 25.1. #79842 (pufit).
  • Proper implementation of ASTSelectWithUnionQuery::clone() method now takes into account is_normalized field as well. This might help with #77569. #79909 (Nikita Mikhaylov).
  • Support correlated subqueries in the projection list in simple cases. #79925 (Dmitry Novik).
  • Fix the inconsistent formatting of certain queries with the EXCEPT operator. If the left-hand side of the EXCEPT operator ends with *, the formatted query loses parentheses and is then parsed as a * with the EXCEPT modifier. These queries are found by the fuzzer and are unlikely to be found in practice. This closes #79950. #79952 (Alexey Milovidov).
  • Allow to add http_response_headers in http_handlers of any kind. #79975 (Andrey Zvonov).
  • Small improvement in JSON type parsing by using cache of variants deserialization order. #79984 (Pavel Kruglov).
  • Allow moving GLOBAL [NOT] IN predicate to PREWHERE clause if applicable. #79996 (Eduard Karacharov).
  • Add setting s3_slow_all_threads_after_network_error. #80035 (Vitaly Baranov).
  • The logging level about the selected parts to merge was wrong (Information). Closes #80061. #80062 (Alexey Milovidov).
  • Function reverse support Tuple data type. Closes #80053. #80083 (flynn).
  • Setting enble_url_encoding default value is now set to False. #80088 (Shankar Iyer).
  • This tiny patch resolves #75817: allows get auxiliary_zookeepers data from system.zookeeper table. #80146 (Nikolay Govorov).
  • Vector search using the vector similarity index is now beta (previously experimental). #80164 (Robert Schulze).
  • Add asynchronous metrics about the server's TCP sockets. This improves the observability. Closes #80187. #80188 (Alexey Milovidov).
  • Function tokens now supports string as a tokenizer. #80195 (Robert Schulze).
  • Parallel replicas: avoid waiting for slow unused replicas if all read tasks have been assigned to other replicas. #80199 (Igor Nikonov).
  • Support anylast_respect_nulls and any_respect_nulls in simpleAggregateFunction. #80219 (Diskein).
  • Remove unnecessary call adjustCreateQueryForBackup() for replicated databases. #80282 (Vitaly Baranov). #* Allow extra options (that go after -- like -- --config.value='abc') in clickhouse-local without the equality sign. Closes #80292. #80293 (Alexey Milovidov).
  • Highlight metacharacters in SHOW ... LIKE queries. This closes #80275. #80297 (Alexey Milovidov). #* Make SQL UDF persistent in clickhouse-local. The previously created function will be loaded at startup. This closes #80085. #80300 (Alexey Milovidov).
  • Support comparison between Time/Time64. #80327 (Yarik Briukhovetskyi).
  • Fix description in explain plan for a preliminary distinct step. #80330 (UnamedRus).
  • Allow use of named collections in ODBC/JDBC. #80334 (Andrey Zvonov).
  • Enable multiple-projection filtering support, allowing to use more than one projection for part-level filtering. This addresses #55525. This is the second step to implement projection index, following #78429. #80343 (Amos Bird).
  • Metrics for the number of readonly and broken disks. Indicator logs when DiskLocalCheckThread is started. #80391 (VicoWu).
  • Implement support for s3_plain_rewritable storage with projections. In previous versions, metadata objects in S3 referencing projections would not get updated when moved. Closes #70258. #80393 (Sav).
  • Parallel replicas uses separate connection timeout, see parallel_replicas_connect_timeout_ms setting. Before connect_timeout_with_failover_ms/connect_timeout_with_failover_secure_ms settings were used as connection timeout values for parallel replicas queries (1 second by default). #80421 (Igor Nikonov).
  • The SYSTEM UNFREEZE command will not try to look up parts in readonly and write-once disks. This closes #80430. #80432 (Alexey Milovidov).
  • Changed log level of a merged parts message from INFO to TRACE. #80476 (Hans Krutzer).
  • Implement flattened serialization for Dynamic and JSON in Native format that allows to serialize/deserialize Dynamic and JSON data without special structures like shared variant for Dynamic and shared data for JSON. This serialization can be enabled by setting output_format_native_use_flattened_dynamic_and_json_serialization. This serialization can be used for easier support for Dynamic and JSON in TCP protocol in clients in different languages. #80499 (Pavel Kruglov).
  • Changes the default behavior of partition pruning for Iceberg table. #80583 (Melvyn Peignon).
  • Add two new ProfileEvents for index search algorithm observability: IndexBinarySearchAlgorithm and IndexGenericExclusionSearchAlgorithm. #80679 (Pablo Marcos).
  • Do not complain about unsupported MADV_POPULATE_WRITE for older kernels in logs (to avoid logs polluting). #80704 (Robert Schulze).
  • Added support for Date32, DateTime64 in TTL. #80710 (Andrey Zvonov).
  • Adjust compatibility values for max_merge_delayed_streams_for_parallel_write. #80760 (Azat Khuzhin).
  • Fix a crash: if an exception is thrown in an attempt to remove a temporary file (they are used for spilling temporary data on disk) in the destructor, the program can terminate. #80776 (Alexey Milovidov).
  • Add IF EXISTS modifier to SYSTEM SYNC REPLICA. #80810 (Raúl Marín).
  • Extend the exception message about "Having zero bytes, but read range is not finished...", add finished_download_time column to system.filesystem_cache'. #80849 (Kseniia Sumarokova).
  • Previously, packed storage was not supported for the full-text index, because the segment id was updated on-fly by reading and writing (.gin_sid) file on disk. In case of packed storage, reading a value from the uncommited file is not supported and this led to an issue. #80852 (Elmi Ahmadov).
  • Add a search algorithm section to EXPLAIN output when using it with indexes = 1. It shows either "binary search" or "generic exclusion search". #80881 (Pablo Marcos).
  • At the beginning of 2024, prefer_column_name_to_alias was hardcoded to True for MySQL handler because the new analyzer was not enabled by default. Now, it has been unhardcoded. #80916 (Yarik Briukhovetskyi).
  • Now system.iceberg_history shows history for catalogs databases like glue or iceberg rest. Also renamed table_name and database_name columns to table and database in system.iceberg_history for consistency. #80975 (alesapin).
  • Allow read-only mode for the merge table function, so the CREATE TEMPORARY TABLE grant is not required for using it. #80981 (Miсhael Stetsyuk).
  • Better introspection of in-memory caches (expose information about caches in system.metrics over incomplete system.asynchronouse_metrics). Add in-memory caches size (in bytes) into dashboard.html. VectorSimilarityIndexCacheSize/IcebergMetadataFilesCacheSize has been renamed to VectorSimilarityIndexCacheBytes/IcebergMetadataFilesCacheBytes. #81023 (Azat Khuzhin).
  • Ignore databases with engines that can't contain RocksDB tables while reading from system.rocksdb. #81083 (Pervakov Grigorii).
  • Allow filesystem_caches and named_collections in the clickhouse-local configuration file. #81105 (Alexey Milovidov).
  • Fix highlighting of PARTITION BY in INSERT queries. In previous versions, PARTITION BY was not highlighted as a keyword. #81106 (Alexey Milovidov). #* Two mini improvements in Web UI: correctly handle queries without output, such as CREATE, INSERT (until recently, these queries resulted in an infinite spinner); - when double clicking on a table, scroll to the top. #81131 (Alexey Milovidov). #* Update c-ares to v1.34.5. #81159 (Konstantin Bogdanov). #* Upgrade curl to 8.14 to address CVE-2025-5025 and CVE-2025-4947. #81171 (larryluogit). #* Upgrade libarchive to 3.7.9 to address: CVE-2024-20696 CVE-2025-25724 CVE-2024-48958 CVE-2024-57970 CVE-2025-1632 CVE-2024-48957 CVE-2024-48615. #81174 (larryluogit). #* Upgrade libxml2 to 2.14.3. #81187 (larryluogit).
  • MemoryResidentWithoutPageCache provides the amount of physical memory used by the server process, excluding userspace page cache, in bytes. This provides a more accurate view of actual memory usage when userspace page cache is utilized. When userspace page cache is disabled, this value equals MemoryResident. #81233 (Jayme Bird).
  • Mark manually logged exceptions in client, local server, keeper client and disks app as logged, so that they are not logged twice. #81271 (Miсhael Stetsyuk).
  • Setting use_skip_indexes_if_final and use_skip_indexes_if_final_exact_mode now default to True. Queries with FINAL clause will now use skip indexes (if applicable) to shortlist granules and also read any additional granules corresponding to matching primary key ranges. Users needing earlier behaviour of approximate/imprecise results can set use_skip_indexes_if_final_exact_mode to FALSE after careful evaluation. #81331 (Shankar Iyer). #* When you have multiple queries in the web UI, it will run the one under the cursor. Continuation of #80977. #81354 (Alexey Milovidov).
  • This PR addresses issues with the implementation of is_strict in the monotonicity checks for conversion functions. Currently, some conversion functions, such as toFloat64(UInt32) and toDate(UInt8), incorrectly return is_strict as false when they should return true. #81359 (zoomxi). #* In filesystem with journal mkdir is written to the journal of filesystem which is persisted to disk. In case of slow disk this can take long time. Definitely make sense to move out from reserve lock scope. #81371 (Kseniia Sumarokova).
  • When checking if a KeyCondition matches a continuous range, if the key is wrapped with a non-strict function chain, a Constraint::POINT may needs to be converted to aConstraint::RANGE. For example: toDate(event_time) = '2025-06-03' implies a range for event_time: ['2025-06-03 00:00:00', '2025-06-04 00:00:00'). This PR fixes this behavior. #81400 (zoomxi). #* Use postgres 16.9. #81437 (Konstantin Bogdanov). #* Use openssl 3.2.4. #81438 (Konstantin Bogdanov). #* Use abseil-cpp 2025-01-27. #81440 (Konstantin Bogdanov). #* Use mongo-c-driver 1.30.4. #81449 (Konstantin Bogdanov). #* Use krb5 1.21.3-final. #81453 (Konstantin Bogdanov). #* Use orc 2.1.2. #81455 (Konstantin Bogdanov). #* Add support for the --database argument in clickhouse-local. You can switch to a previously created database. This closes #44115. #81465 (Alexey Milovidov). #* clickhouse/ch aliases will invoke clickhouse-client instead of clickhouse-local if --host or --port are specified. Continuation of #79422. Closes #65252. #81509 (Alexey Milovidov).
  • Now that we have the keeper response time distribution data, we can tune the histogram buckets. #81516 (Miсhael Stetsyuk).
  • Postpone reading of Iceberg manifest files until the first reading of a query. #81619 (Daniil Ivanik). #* Use grpc 1.73.0. #81629 (Konstantin Bogdanov). #* Use delta-kernel-rs v0.12.1. #81707 (Konstantin Bogdanov).
  • Add profile event PageCacheReadBytes. #81742 (Kseniia Sumarokova).

Bug fix (user-visible misbehavior in an official stable release)

  • Fix parameterized view with SELECT EXCEPT query. Closes #49447. #57380 (Nikolay Degterinsky).
  • Analyzer: fix column projection name after column type promotion in join. Closes #63345. #63519 (Dmitry Novik).
  • A materialized view can start too late, e.g. after the Kafka table that streams to it. #72123 (Ilya Golshtein).
  • Fixed a logical error in cases of column name clashes when analyzer_compatibility_join_using_top_level_identifier is enabled. #75676 (Vladimir Cherkasov).
  • Fixed rare crashes while reading from MergeTree table after multiple asynchronous (with alter_sync = 0) RENAME COLUMN and ADD COLUMN queries. #76346 (Anton Popov).
  • Fix SELECT query rewriting during VIEW creation with enabled analyzer. closes #75956. #76356 (Dmitry Novik).
  • Fix CTE usage in pushed-down predicates when allow_push_predicate_ast_for_distributed_subqueries is enabled. Fixes #75647. Fixes #79672. #77316 (Dmitry Novik).
  • Fix applying async_insert from server (via apply_settings_from_server) (previously leads to Unknown packet 11 from server errors on the client). #77578 (Azat Khuzhin).
  • Fixed refreshable materialized view in replicated databases not working on newly added replicas. #77774 (Michael Kolupaev).
  • Fixed refreshable materialized views breaking backups. #77893 (Michael Kolupaev).
  • Fix old firing logical error for transform. #78247 (Yarik Briukhovetskyi).
  • Fixes an issue where SYSTEM SYNC REPLICA LIGHTWEIGHT 'foo' would report success even when the specified replica didn't exist. The command now properly validates that the replica exists in Keeper before attempting synchronization. #78405 (Jayme Bird).
  • Fix some cases where secondary index was not applied with analyzer. Fixes #65607 , fixes #69373. #78485 (Nikolai Kochetov).
  • Fix dumping profile events (NetworkSendElapsedMicroseconds/NetworkSendBytes) for HTTP protocol with compression enabled (the error should not be more then the buffer size, usually around 1MiB). #78516 (Azat Khuzhin). #* ```sql CREATE TABLE t0 ( key Int32, value Int32 ) ENGINE=MergeTree() PRIMARY KEY key PARTITION BY key % 2;. #78593 (Vlad).
  • Fix analyzer producing LOGICAL_ERROR when JOIN ... USING involves ALIAS column - should produce an appropriate error. #78618 (Yakov Olkhovskiy).
  • Fix analyzer: CREATE VIEW ... ON CLUSTER fails if SELECT contains positional arguments. #78663 (Yakov Olkhovskiy).
  • Fix Block structure mismatch error in case of INSERT SELECT into table a function with schema inference if SELECT has scalar subqueries. #78677 (Pervakov Grigorii).
  • Fix analyzer: with prefer_global_in_and_join=1 for Distributed table in SELECT query in function should be replaced by globalIn. #78749 (Yakov Olkhovskiy).
  • Fixed several types of SELECT queries that read from tables with MongoDB engine or mongodb table function: queries with implicit conversion of const value in WHERE clause (e.g. WHERE datetime = '2025-03-10 00:00:00') ; queries with LIMIT and GROUP BY. Previously, they could return the wrong result. #78777 (Anton Popov).
  • Fix conversion between different JSON types. Now it's performed by simple cast through convertion to/from String. It's less effective but 100% accurate. #78807 (Pavel Kruglov).
  • Fix logical error during convertion of Dynamic type to Interval. #78813 (Pavel Kruglov).
  • Fix column rollback on JSON parsing error. #78836 (Pavel Kruglov).
  • Fix 'bad cast' error when join using constant alias column. #78848 (Vladimir Cherkasov).
  • Don't allow PREWHERE in materialized views on columns with different types in view and target table. #78889 (Pavel Kruglov).
  • Fix logical error during parsing of bad binary data of Variant column. #78982 (Pavel Kruglov).
  • Throw an exception when the parquet batch size is set to 0. Previously when output_format_parquet_batch_size = 0 ClickHouse would hang. Now this behavior is fixed. #78991 (daryawessely).
  • Fix deserialization of variant discriminators with basic format in compact parts. It was introduced in https://github.com/ClickHouse/ClickHouse/pull/55518. #79000 (Pavel Kruglov).
  • Dictionaries of type complex_key_ssd_cache now reject zero or negative block_size and write_buffer_size parameters (issue #78314). #79028 (Elmi Ahmadov).
  • Avoid using Field for non-aggregated columns in SummingMergeTree. It could lead to unexpected errors with Dynamic/Variant types used in SummingMergeTree. #79051 (Pavel Kruglov).
  • Fix read from materialized view with distributed destination table and different header in analyzer. #79059 (Pavel Kruglov).
  • Fix crash for a very specific situation when the currentDatabase function was used in CONSTRAINT sections for ON CLUSTER queries Closes #78100. #79070 (pufit).
  • Fixes a bug where arrayUnion() returned extra (incorrect) values on tables that had batch inserts. Fixes #75057. #79079 (Peter Nguyen). #* Fix segfault in OpenSSLInitializer. Closes #79092. #79097 (Konstantin Bogdanov).
  • Fix passing of external roles in inter-server queries. #79099 (Andrey Zvonov).
  • Always set prefix for S3 ListObject. #79114 (Azat Khuzhin).
  • Fixes a bug where arrayUnion() returned extra (incorrect) values on tables that had batch inserts. Fixes #79157. #79158 (Peter Nguyen).
  • Fix logical error after filter pushdown. #79164 (Pervakov Grigorii).
  • Try to use IColumn instead of Field in SingleValueDataGeneric. It fixes the incorrect return values for some aggregate functions like argMax for types Dynamic/Variant/JSON. #79166 (Pavel Kruglov).
  • Fix DeltaLake table engine with delta-kernel implementation being used with http based endpoints, fix NOSIGN. Closes #78124. #79203 (Kseniia Sumarokova).
  • Keeper fix: Avoid triggering watches on failed multi requests. #79247 (Antonio Andelic).
  • Forbid Dynamic and JSON types in IN. With current implementation of IN it can lead to incorrect results. Proper support of this types in IN is complicated and can be done in future. #79282 (Pavel Kruglov).
  • Fix check for duplicate paths in JSON type parsing. #79317 (Pavel Kruglov).
  • Fix SecureStreamSocket connection issues. #79383 (Konstantin Bogdanov).
  • Fix loading of plain_rewritable disks containing data. #79439 (Julia Kartseva).
  • Fix crash in dynamic subcolumns discovery in Wide parts in MergeTree. #79466 (Pavel Kruglov).
  • Verify the table name's length only for initial create queries. Do not verify this for secondary creates to avoid backward compatibility issues. #79488 (Miсhael Stetsyuk).
  • Fixed error Block structure mismatch in several cases with tables with sparse columns. #79491 (Anton Popov).
  • Fixes two cases of Logical Error: Can't set alias of * of Asterisk on alias. #79505 (Raúl Marín).
  • Fix applying use_native_copy and allow_azure_native_copy setting for azure blob storage and updated to use native copy only when credentials match resolves #78964. #79561 (Smita Kulkarni).
  • Fix using incorrect paths when renaming an Atomic database. #79569 (Tuan Pham Anh).
  • Fix order by JSON column with other columns. #79591 (Pavel Kruglov).
  • Fix result duplication when reading from remote with both use_hedged_requests and allow_experimental_parallel_reading_from_replicas disabled. #79599 (Eduard Karacharov).
  • Fix crash in delta-kernel implementation when using unity catalog. #79677 (Kseniia Sumarokova).
  • Resolve macros for autodiscovery clusters. #79696 (Anton Ivashkin).
  • Fix logical errors about a column's unknown origin scope produced while checking if this column is correlated. Fixes #78183. Fixes #79451. #79727 (Dmitry Novik).
  • Fix wrong results for grouping sets with ColumnConst and Analyzer. #79743 (Andrey Zvonov).
  • Fix local shard result duplication when reading from distributed table with local replica being stale. #79761 (Eduard Karacharov).
  • Handle incorrectly configured page_cache_limits suitably. #79805 (Bharat Nallan).
  • Fixes the result of SQL function formatDateTime if a variable-size formatter (e.g. %W aka. weekday Monday Tuesday, etc.) is followed by a compound formatter (a formatter that prints multiple components at once, e.g. %D aka. the American date 05/04/25). #79835 (Robert Schulze).
  • IcebergS3 supports count optimization, but IcebergS3Cluster does not. As a result, the count() result returned in cluster mode may be a multiple of the number of replicas. #79844 (wxybear).
  • Fix the sorting order of the NaNs with a negative sign bit. #79847 (Pervakov Grigorii).
  • Now GROUP BY ALL doesn't take into account the GROUPING part. #79915 (Yarik Briukhovetskyi).
  • Fixes AMBIGUOUS_COLUMN_NAME error with lazy materialization when no columns are used for query execution until projection. Example, SELECT * FROM t ORDER BY rand() LIMIT 5. #79926 (Igor Nikonov).
  • Fixed incorrect state merging for TopK / TopKWeighted functions that would cause excessive error values even when capacity was not exhausted. #79939 (Joel Höner).
  • Hide password for query CREATE DATABASE datalake ENGINE = DataLakeCatalog(\'http://catalog:8181\', \'admin\', \'password\'). #79941 (Han Fei).
  • Allow specifying an alias in JOIN USING. Specify this alias in case the column was renamed (e.g., because of `ARRAY JOIN). Fixes #73707. #79942 (Nikolai Kochetov).
  • Respect readonly setting in azure_blob_storage object storage. #79954 (Julia Kartseva).
  • Fixed incorrect query results and out-of-memory crashes when using match(column, '^…') with backslash-escaped characters. #79969 (filimonov).
  • Disabling hive partitioning for datalakes Partially addresses https://github.com/issues/assigned?issue=ClickHouse%7CClickHouse%7C79937. #80005 (Daniil Ivanik).
  • Skip indexes with lambda expressions could not be applied. Fix the case when high-level functions in the index definition exactly match the one in the query. #80025 (Nikolai Kochetov).
  • Allow materialized views with UNIONs to work correctly on new replicas. #80037 (Samay Sharma).
  • Fix metadata version during attach part on the replica executing ATTACH_PART command from replication log. #80038 (Aleksei Filatov).
  • Format specifier %e in SQL function parseDateTime now recognizes single-digit days (e.g. 3), whereas it previously required space padding (e.g. 3). This makes its behavior compatible with MySQL. To retain the previous behaviour, set setting parsedatetime_e_requires_space_padding = 1. (issue #78243). #80057 (Robert Schulze).
  • Executable User Defined Functions (eUDF) names are not added to the used_functions column of the system.query_log table, unlike other functions. This PR implements the addition of the eUDF name if the eUDF was used in the request. #80073 (Kyamran). #* Fix warnings Cannot find 'kernel' in '[...]/memory.stat' in ClickHouse's log (issue #77410). #80129 (Robert Schulze).
  • Fix logical error in Arrow format with LowCardinality(FixedString). #80156 (Pavel Kruglov).
  • Fix reading subcolumns from Merge engine. #80158 (Pavel Kruglov).
  • Fix a bug about the comparison between numeric types in KeyCondition. #80207 (Yarik Briukhovetskyi).
  • Fix AMBIGUOUS_COLUMN_NAME when lazy materialization applied to table with projections. #80251 (Igor Nikonov).
  • Fix incorrect count optimization for string prefix filters like LIKE 'ab_c%' when using implicit projections. This fixes #80250. #80261 (Amos Bird).
  • Fix improper serialization of nested numeric fields as strings in MongoDB documents. Remove maximum depth limit for documents from MongoDB. #80289 (Kirill Nikiforov).
  • Perform less strict metadata checks for RMT in the Replicated database. Closes #80296. #80298 (Nikolay Degterinsky).
  • Fix text representation of DateTime and DateTime64 for PostgreSQL storage. #80301 (Yakov Olkhovskiy).
  • Allow DateTime with timezone in StripeLog tables. This closes #44120. #80304 (Alexey Milovidov).
  • Disable filter-push-down for the predicate with a non-deterministic function in case the query plan step changes the number of rows. Fixes #40273. #80329 (Nikolai Kochetov).
  • Fix possible logical errors and crashes in projections with subcolumns. #80333 (Pavel Kruglov).
  • Fix NOT_FOUND_COLUMN_IN_BLOCK error caused by filter-push-down optimization of the logical JOIN sep in case ON expression is not a trivial equality. Fixes #79647 Fixes #77848. #80360 (Nikolai Kochetov).
  • Fix incorrect results when reading reverse-ordered keys in partitioned tables. This fixes #79987. #80448 (Amos Bird).
  • Fixed wrong sorting in tables with a nullable key and enabled optimize_read_in_order. #80515 (Pervakov Grigorii).
  • Fixed refreshable materialized view DROP getting stuck if the view was paused using SYSTEM STOP REPLICATED VIEW. #80543 (Michael Kolupaev).
  • Fix 'Cannot find column' with constant tuple in distributed query. #80596 (Yakov Olkhovskiy).
  • Fix shardNum function in Distributed tables with join_use_nulls. #80612 (János Benjamin Antal).
  • Fix incorrect results during reading column that exists in subset of tables in Merge engine. #80643 (Pavel Kruglov).
  • The timestamp in the iceberg_history table should now be correct. #80711 (Melvyn Peignon).
  • Fix handling of enum globs of a single element in object storage table functions. #80716 (Konstantin Bogdanov).
  • Fix wrong result types of comparison functions with Tuple(Dynamic) and String that led to logical error. #80728 (Pavel Kruglov).
  • Add missing support data type timestamp_ntz for unity catalog. Fixes #79535, Fixes #79875. #80740 (alesapin).
  • Fix THERE_IS_NO_COLUMN error for distributed queries with IN cte. Fixes #75032. #80757 (Nikolai Kochetov).
  • Fix excessive number of files (leads to excessive memory usage) for external ORDER BY. #80777 (Azat Khuzhin). #* This PR might close #80742. #80783 (zoomxi). #* Fix crash in Kafka due to get_member_id() was creating std::string from NULL (it was likely an issue only in case of connection to broker had been failed). #80793 (Azat Khuzhin).
  • Properly await consumers before shutting down Kafka engine (active consumers after shutdown can trigger various debug assertions and also may read data from brokers in background after table has been dropped/detached). #80795 (Azat Khuzhin).
  • Fix NOT_FOUND_COLUMN_IN_BLOCK, which is caused by predicate-push-down optimization. Fixes #80443. #80834 (Nikolai Kochetov).
  • Fix logical error when resolving star (*) matcher in table function in JOIN with USING. #80894 (Vladimir Cherkasov).
  • Fix memory accounting for Iceberg metadata files cache. #80904 (Azat Khuzhin).
  • Fix wrong partitioning with nullable partition key. #80913 (Pervakov Grigorii).
  • Fix Table does not exist error for distributed queries with pushed-down predicate (allow_push_predicate_ast_for_distributed_subqueries=1) when the source table does not exist on the initialtor. Fixes #77281. #80915 (Nikolai Kochetov).
  • Fix the logical error in the nested functions with named windows. #80926 (Pervakov Grigorii).
  • Fix extremes for nullable and floating-point columns. #80970 (Pervakov Grigorii).
  • Fix possible crash while querying from system.tables (likely the case under memory pressure). #80976 (Azat Khuzhin).
  • Fix atomic rename with truncate for files which compression is inferred from their file extension. #80979 (Pablo Marcos). #* Fix ErrorCodes::getName. #81032 (RinChanNOW).
  • Fix bug when user cannot list tables in Unity Catalog without permissions for all of them. Now all tables are listed properly, attempt to read from restricted table will throw an exception. #81044 (alesapin).
  • Now ClickHouse will ignore errors and unexpected responses from data lake catalogs in SHOW TABLES query. Fixes #79725. #81046 (alesapin).
  • Fix parsing of DateTime64 from integers in JSONExtract and JSON type parsing. #81050 (Pavel Kruglov).
  • Reflect date_time_input_format setting in schema inference cache. #81052 (Pavel Kruglov).
  • Fix crash on INSERT if table was DROPed after query started but before columns sent. #81053 (Azat Khuzhin).
  • Fix use-of-uninitialized-value in quantileDeterministic. #81062 (Azat Khuzhin).
  • Fix hardlinks count management for metadatastoragefromdisk disk transactions. add tests. #81066 (Sema Checherinda).
  • User Defined Functions (UDF) names are not added to the system.query_log table, unlike other functions. This PR implements the addition of the UDF name to one of the two columns used_executable_user_defined_functions or used_sql_user_defined_functions if the UDF was used in the request. #81101 (Kyamran).
  • Fixed Too large size ... passed to allocator errors or possible crashes on inserts via http protocol with text formats (JSON, Values, ...) and omitted Enum fields. #81145 (Anton Popov).
  • Fix LOGICAL_ERROR in case of Sparse column in INSERT block pushed to non-MT MV. #81161 (Azat Khuzhin).
  • Fix Unknown table expression identifier for distributed_product_mode_local=local with cross-replication. #81162 (Nikolai Kochetov).
  • Fixed incorrectly caching number of rows in parquet files after filtering. #81184 (Michael Kolupaev).
  • Fix fs cache max_size_to_total_space setting when used with relative cache path. #81237 (Kseniia Sumarokova).
  • Fixed clickhouse-local crashing when outputting const tuples or maps in Parquet format. #81249 (Michael Kolupaev).
  • Verify array offsets received over network. #81269 (Azat Khuzhin).
  • Fix some corner case in query that joins empty tables and uses window functions. The bug leads to exploding number of parallel streams which leads to OOMs. #81299 (Alexander Gololobov).
  • Fixes for datalake Cluster functions (deltaLakeCluster, icebergCluster, etc): (1) fix potential segfault in DataLakeConfiguration when using Cluster function with old analyzer; (2) remove duplicating data lake metadata updates (extra object storage requests); (3) fix redundant listing in object storage when format is not explicitly specified (which was already done for non-cluster data lake engines). #81300 (Kseniia Sumarokova).
  • Make force_restore_data flag recover lost keeper metadata. #81324 (Raúl Marín).
  • Fix region error in delta-kernel. Fixes #79914. #81353 (Kseniia Sumarokova).
  • Disable incorrect JIT for divideOrNull. #81370 (Raúl Marín).
  • Fix insert error when MergeTree table has a long partition column name. #81390 (hy123q).
  • Don't store content of several manifest files in memory. #81470 (Daniil Ivanik).
  • Fix possible crash during shutting down background pools (background_.*pool_size). #81473 (Azat Khuzhin).
  • Fix out-of-bounds read in the Npy format happening when writing to a table with the URL engine. This closes #81356. #81502 (Alexey Milovidov).
  • There is a chance that Web UI displays NaN% (typical JavaScript problems). #81507 (Alexey Milovidov).
  • Fix DatabaseReplicated for database_replicated_enforce_synchronous_settings=1. #81564 (Azat Khuzhin).
  • Fix sorting order for LowCardinality(Nullable(...)) types. #81583 (Pervakov Grigorii).
  • Server should not preserve a HTTP connection if the request has not been fully read from the socket. #81595 (Sema Checherinda).
  • Make scalar correlated subqueries return a nullable result of the projection expression. Fix the case when a correlated subquery produces an empty result set. #81632 (Dmitry Novik).
  • Fix Unexpected relative path for a deduplicated part during ATTACH to ReplicatedMergeTree. #81647 (Azat Khuzhin).
  • Query settings use_iceberg_partition_pruning will not take effect for iceberg storage, because it uses global context rather than query context. it's not critical because its default value is true. this pr can fix it. #81673 (Han Fei).
  • Add validation for merge tree setting merge_max_block_size to ensure that it's non zero. #81693 (Bharat Nallan).
  • Fix issues with clickhouse-local involving stuck DROP VIEW queries. #81705 (Bharat Nallan).
  • Fix StorageRedis join in some cases. #81736 (Pervakov Grigorii).
  • Fix crash in ConcurrentHashJoin with empty USING () and old analyzer enabled. #81754 (Nikita Taranov).
  • Keeper fix: block commits of new logs if there is invalid entry in the logs. Previously, if the leader applied some logs incorrectly, it would continue to commit new logs, even though the follower would detect digest mismatch and abort. #81780 (Antonio Andelic).