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 settinggeotoh3_lon_lat_input_order = true
. #78852 (Pratima Patel). - Indexes of type
full_text
were renamed togin
. This follows the more familiar terminology of PostgreSQL and other databases. Existing indexes of typefull_text
remain loadable but they will throw an exception (suggestinggin
indexes instead) when one tries to use them in searches. #79024 (Robert Schulze). - Add a filesystem cache setting
allow_dynamic_cache_resize
, by defaultfalse
, 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
andusearch
. Both have been stubs for a long time, i.e. every attempt to use the legacy indexes returned an error anyways. If you still haveannoy
andusearch
indexes, please drop them. #79802 (Robert Schulze). #* Removeformat_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 theCREATE
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 nametoTime
totoTimeWithFixedDate
because the functiontoTime
is required for the cast function. #75735 (Yarik Briukhovetskyi). - Support correlated subqueries as an argument of
EXISTS
expression in theWHERE
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 parallelizeINSERT
s 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
andgetMergeTreeSetting
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 leverageversion-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 implicitFROM 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
andicebergBucketTransform
functions. Support data files pruning inIceberg
tables partitioned withbucket transfom
. #79262 (Daniil Ivanik). - Add a support for Coalescing Merge Tree. This closes #78869. #79344 (Konstantin Vedernikov).
- Add
stringBytesUniq
andstringBytesEntropy
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 forpointInPolygon
. #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
andBech32m
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-likedelta
,rate
,idelta
andirate
. #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 inarrayFirst
,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 optionalmax_rows_per_postings_list
andngram_size
. #80262 (Elmi Ahmadov). - Experimental indexes of type
gin
were renamed totext
. Existing indexes of typegin
remain loadable but they will throw an exception (suggestingtext
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 thehash
algorithm when the size of the right table is below the threshold. #76185 (Nikita Taranov). - The existing implementation of
Pipe::resize
creates a singleResize
orStrictResize
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 theExecutingGraph::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 allowingINSERT
s data in parallel (can be enabled withparallel_inserts=true
queue setting). PreviouslyS3Queue
/AzureQueue
can only do the first part of the pipeline in parallel (downloading, parsing), INSERT was single-threaded. AndINSERT
s are almost always the bottleneck. Now it will scale almost linear withprocessing_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 setting
use_skip_indexes_in_final_exact_mode
. If a query on aReplacingMergeTree
table has theFINAL
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). #* ReplaceDELETE FROM ... WHERE 1
queries toTRUNCATE
. (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))
toconvertDecimalsImpl
. #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 optionquery_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
andsource
) forsystem.named_collections
. Closes #78179. #78582 (MikhailBurdukov). - Added field
condition
to system tablesystem.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
(hereall_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 theCREATE 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 inS3Queue
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
NULL
s 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 thecache
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 withparam_<name>
(underscore). This closes #63093. #79429 (Engel Danila). #* Add filesystem cache settingmax_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
reconfigurereconnect
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: whenthrow_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
andquery_id
to thesystem.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 accountis_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 theEXCEPT
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
inhttp_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 toPREWHERE
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 toFalse
. #80088 (Shankar Iyer). - This tiny patch resolves #75817: allows get
auxiliary_zookeepers
data fromsystem.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 supportsstring
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
andany_respect_nulls
insimpleAggregateFunction
. #80219 (Diskein). - Remove unnecessary call
adjustCreateQueryForBackup()
for replicated databases. #80282 (Vitaly Baranov). #* Allow extra options (that go after--
like-- --config.value='abc'
) inclickhouse-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 inclickhouse-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. Beforeconnect_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
andIndexGenericExclusionSearchAlgorithm
. #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 toSYSTEM 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 withindexes = 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 renamedtable_name
anddatabase_name
columns totable
anddatabase
insystem.iceberg_history
for consistency. #80975 (alesapin). - Allow read-only mode for the
merge
table function, so theCREATE 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 incompletesystem.asynchronouse_metrics
). Add in-memory caches size (in bytes) intodashboard.html
.VectorSimilarityIndexCacheSize
/IcebergMetadataFilesCacheSize
has been renamed toVectorSimilarityIndexCacheBytes
/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
andnamed_collections
in theclickhouse-local
configuration file. #81105 (Alexey Milovidov). - Fix highlighting of
PARTITION BY
inINSERT
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 asCREATE
,INSERT
(until recently, these queries resulted in an infinite spinner); - when double clicking on a table, scroll to the top. #81131 (Alexey Milovidov). #* Updatec-ares
tov1.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
anduse_skip_indexes_if_final_exact_mode
now default toTrue
. Queries withFINAL
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 setuse_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 journalmkdir
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, aConstraint::POINT
may needs to be converted to aConstraint::RANGE
. For example:toDate(event_time) = '2025-06-03'
implies a range forevent_time
: ['2025-06-03 00:00:00', '2025-06-04 00:00:00'). This PR fixes this behavior. #81400 (zoomxi). #* Usepostgres
16.9. #81437 (Konstantin Bogdanov). #* Useopenssl
3.2.4. #81438 (Konstantin Bogdanov). #* Useabseil-cpp
2025-01-27. #81440 (Konstantin Bogdanov). #* Usemongo-c-driver
1.30.4. #81449 (Konstantin Bogdanov). #* Usekrb5
1.21.3-final. #81453 (Konstantin Bogdanov). #* Useorc
2.1.2. #81455 (Konstantin Bogdanov). #* Add support for the--database
argument inclickhouse-local
. You can switch to a previously created database. This closes #44115. #81465 (Alexey Milovidov). #*clickhouse
/ch
aliases will invokeclickhouse-client
instead ofclickhouse-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). #* Usedelta-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 (withalter_sync = 0
)RENAME COLUMN
andADD COLUMN
queries. #76346 (Anton Popov). - Fix
SELECT
query rewriting duringVIEW
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 (viaapply_settings_from_server
) (previously leads toUnknown 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
whenJOIN ... USING
involvesALIAS
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 ofINSERT SELECT
into table a function with schema inference ifSELECT
has scalar subqueries. #78677 (Pervakov Grigorii). - Fix analyzer: with
prefer_global_in_and_join=1
for Distributed table in SELECT queryin
function should be replaced byglobalIn
. #78749 (Yakov Olkhovskiy). - Fixed several types of
SELECT
queries that read from tables withMongoDB
engine ormongodb
table function: queries with implicit conversion of const value inWHERE
clause (e.g.WHERE datetime = '2025-03-10 00:00:00'
) ; queries withLIMIT
andGROUP 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 negativeblock_size
andwrite_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 inCONSTRAINT
sections forON 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 inOpenSSLInitializer
. 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 typesDynamic/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 inIN
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
andallow_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. weekdayMonday
Tuesday
, etc.) is followed by a compound formatter (a formatter that prints multiple components at once, e.g.%D
aka. the American date05/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 theGROUPING
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 inazure_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 functionparseDateTime
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 settingparsedatetime_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 thesystem.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 warningsCannot 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 inStripeLog
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 caseON
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 withjoin_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 withIN 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 bypredicate-push-down
optimization. Fixes #80443. #80834 (Nikolai Kochetov). - Fix logical error when resolving star (*) matcher in table function in
JOIN
withUSING
. #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 inJSONExtract
andJSON
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 columnsused_executable_user_defined_functions
orused_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 omittedEnum
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
fordistributed_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 inDataLakeConfiguration
when usingCluster
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 theURL
engine. This closes #81356. #81502 (Alexey Milovidov). - There is a chance that Web UI displays
NaN%
(typical JavaScript problems). #81507 (Alexey Milovidov). - Fix
DatabaseReplicated
fordatabase_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
duringATTACH
toReplicatedMergeTree
. #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 stuckDROP VIEW
queries. #81705 (Bharat Nallan). - Fix StorageRedis join in some cases. #81736 (Pervakov Grigorii).
- Fix crash in
ConcurrentHashJoin
with emptyUSING ()
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).