Skip to main content
Skip to main content

v25.4 Changelog for Cloud

Backward Incompatible Changes

  • Parquet output format converts Date and DateTime columns to date/time types supported by Parquet, instead of writing them as raw numbers. DateTime becomes DateTime64(3) (was: UInt32); setting output_format_parquet_datetime_as_uint32 brings back the old behavior. Date becomes Date32 (was: UInt16). #70950 (Michael Kolupaev).
  • Don't allow comparable types (like JSON/Object/AggregateFunction) in ORDER BY and comparison functions less/greater/equal/etc by default. #73276 (Pavel Kruglov).
  • JSONEachRowWithProgress will write the progress whenever the progress happens. In previous versions, the progress was shown only after each block of the result, which made it useless. Change the way how the progress is displayed: it will not show zero values. Keep in mind that the progress is sent even if it happens frequently. It can generate a significant volume of traffic. Keep in mind that the progress is not flushed when the output is compressed. This closes #70800. #73834 (Alexey Milovidov).
  • The mysql dictionary source no longer does SHOW TABLE STATUS query, because it does not provide any value for InnoDB tables, as long as for any recent MySQL versions. This closes #72636. This change is backward compatible, but put in this category, so you have a chance to notice it. #73914 (Alexey Milovidov).
  • Merge tables will unify the structure of underlying tables by using a union of their columns and deriving common types. This closes #64864. This closes #35307. In certain cases, this change could be backward incompatible. One example is when there is no common type between tables, but conversion to the type of the first table is still possible, such as in the case of UInt64 and Int64 or any numeric type and String. If you want to return to the old behavior, set merge_table_max_tables_to_look_for_schema_inference to 1 or set compatibility to 24.12 or earlier. #73956 (Alexey Milovidov).
  • CHECK TABLE queries now require a separate, CHECK grant. In previous versions, it was enough to have SHOW TABLES grant to run these queries. But a CHECK TABLE query can be heavy, and usual query complexity limits for SELECT queries don't apply to it. It led to the potential of DoS. #74471 (Alexey Milovidov).
  • Check all columns in a materialized view match the target table if allow_materialized_view_with_bad_select is false. #74481 (Christoph Wurm).
  • Function h3ToGeo() now returns the results in the order (lat, lon) (which is the standard order for geometric functions). Users who wish to retain the legacy result order (lon, lat) can set setting h3togeo_lon_lat_result_order = true. #74719 (Manish Gill).
  • Add JSONCompactEachRowWithProgress and JSONCompactStringsEachRowWithProgress formats. Continuation of #69989. The JSONCompactWithNames and JSONCompactWithNamesAndTypes no longer output "totals" - apparently, it was a mistake in the implementation. #75037 (Alexey Milovidov).
  • Change format_alter_operations_with_parentheses default to true to make alter commands list unambiguous (see https://github.com/ClickHouse/ClickHouse/pull/59532). This breaks replication with clusters prior to 24.3. If you are upgrading a cluster using older releases, turn off the setting in the server config or upgrade to 24.3 first. #75302 (Raúl Marín).
  • Disallow truncate database for replicated databases. #76651 (Bharat Nallan).
  • Disable parallel replicas by default when analyzer is disabled regardless compatibility setting. It's still possible to change this behavior by explicitly setting parallel_replicas_only_with_analyzer to false. #77115 (Igor Nikonov).
  • It's no longer possible to use NaN or inf for float values as settings. #77546 (Yarik Briukhovetskyi).
  • Fixes cases where dateTrunc is used with negative date/datetime arguments. #77622 (Yarik Briukhovetskyi).
  • The legacy MongoDB integration has been removed. Server setting use_legacy_mongodb_integration became obsolete and now does nothing. #77895 (Robert Schulze).
  • Enhance SummingMergeTree validation to skip aggregation for columns used in partition or sort keys. #78022 (Pervakov Grigorii).

New Features

  • Added an in-memory cache for deserialized skipping index granules. This should make repeated queries that use skipping indexes faster. The size of the new cache is controlled by server settings skipping_index_cache_size and skipping_index_cache_max_entries. The original motivation for the cache were vector similarity indexes which became a lot faster now. #70102 (Robert Schulze).
  • A new implementation of the Userspace Page Cache, which allows caching data in the in-process memory instead of relying on the OS page cache. It is useful when the data is stored on a remote virtual filesystem without backing with the local filesystem cache. #70509 (Michael Kolupaev).
  • Add setting to query Iceberg tables as of a specific timestamp. #71072 (Brett Hoerner).
  • Implement Iceberg tables partition pruning for time-related transform partition operations in Iceberg. #72044 (Daniil Ivanik).
  • Add the ability to create min-max (skipping) indices by default for columns managed by MergeTree using settings enable_minmax_index_for_all_numeric_columns (for numeric columns) and enable_minmax_index_for_all_string_columns (for string columns). For now, both settings are disabled, so there is no behavior change yet. #72090 (Smita Kulkarni).
  • Added aggregation function sequenceMatchEvents which return timestamps of matched events for longest chain of events in pattern. #72349 (UnamedRus).
  • SELECT and VIEW statements now support aliases, e.g. SELECT b FROM (SELECT number, number*2 FROM numbers(2)) AS x (a, b);. This enables TPC-H query 15 to run without modifications. #72480 (Yarik Briukhovetskyi).
  • Added a new setting enable_adaptive_memory_spill_scheduler that allows multiple grace JOINs in the same query to monitor their combined memory footprint and trigger spilling into an external storage adaptively to prevent MEMORY_LIMIT_EXCEEDED. #72728 (lgbo).
  • Added function arrayNormalizedGini. #72823 (flynn).
  • Support low cardinality decimal data types, fix #72256. #72833 (zhanglistar).
  • When min_age_to_force_merge_seconds and min_age_to_force_merge_on_partition_only are both enabled, the part merging will ignore the max bytes limit. #73656 (Kai Zhu).
  • Support reading HALF_FLOAT values from Apache Arrow/Parquet/ORC (they are read into Float32). This closes #72960. Keep in mind that IEEE-754 half float is not the same as BFloat16. Closes #73835. #73836 (Alexey Milovidov).
  • The system.trace_log table will contain two new columns, symbols and lines containing symbolized stack trace. It allows for easy collection and export of profile information. This is controlled by the server configuration value symbolize inside trace_log and is enabled by default. #73896 (Alexey Milovidov).
  • Add a new function, generateSerialID, which can be used to generate auto-incremental numbers in tables. Continuation of #64310 by kazalika. This closes #62485. #73950 (Alexey Milovidov).
  • Add syntax query1 PARALLEL WITH query2 PARALLEL WITH query3 ... PARALLEL WITH queryN That means subqueries {query1, query2, ... queryN} are allowed to run in parallel with each other (and it's preferable). #73983 (Vitaly Baranov).
  • Now, Play UI has a progress bar during query runtime. It allows cancelling queries. It displays the total number of records and the extended information about the speed. The table can be rendered incrementally as soon as data arrives. Enable HTTP compression. Rendering of the table became faster. The table header became sticky. It allows selecting cells and navigating them by arrow keys. Fix the issue when the outline of the selected cell makes it smaller. Cells no longer expand on mouse hover but only on selection. The moment to stop rendering the incoming data is decided on the client rather than on the server side. Highlight digit groups for numbers. The overall design was refreshed and became bolder. It checks if the server is reachable and the correctness of credentials and displays the server version and uptime. The cloud icon is contoured in every font, even in Safari. Big integers inside nested data types will be rendered better. It will display inf/nan correctly. It will display data types when the mouse is over a column header. #74204 (Alexey Milovidov).
  • Add the ability to create min-max (skipping) indices by default for columns managed by MergeTree using settings add_minmax_index_for_numeric_columns (for numeric columns) and add_minmax_index_for_string_columns (for string columns). For now, both settings are disabled, so there is no behavior change yet. #74266 (Smita Kulkarni).
  • Add script_query_number and script_line_number fields to system.query_log, to the ClientInfo in the native protocol, and to server logs. This closes #67542. Credits to pinsvin00 for kicking off this feature earlier in #68133. #74477 (Alexey Milovidov).
  • Add minus operator support for DateTime64, to allow subtraction between DateTime64 values, as well as DateTime. #74482 (Li Yin).
  • Support DeltaLake table engine for AzureBlobStorage. Fixes #68043. #74541 (Smita Kulkarni).
  • Add bind_host setting to set the source IP address for clickhouse client connections. #74741 (Todd Yocum).
  • Added an ability to apply non-finished (not materialized by background process) mutations during the execution of SELECT queries immediately after submitting. It can be enabled by setting apply_mutations_on_fly. #74877 (Anton Popov).
  • Fixed some previously unexpected cases when toStartOfInterval datetime arguments are negative. Done by implementing a new function called toStartOfIntervalAllowNegative, which does pretty much the same but returns only Date32/DateTime64. #74933 (Yarik Briukhovetskyi).
  • A new function initialQueryStartTime has been added. It returns the start time of the current query. The value is the same across all shards during a distributed query. #75087 (Roman Lomonosov).
  • Introduce parametrized_view_parameters in system.tables. Closes https://github.com/clickhouse/clickhouse/issues/66756. #75112 (NamNguyenHoai).
  • Allow altering a database comment. Closes #73351 ### Documentation entry for user-facing changes. #75622 (NamNguyenHoai).
  • Add ability to ATTACH tables without database layer (avoids UUID hack). #75788 (Azat Khuzhin).
  • Added concurrent_threads_scheduler server setting that governs how CPU slots are distributed among concurrent queries. Could be set to round_robin (previous behavior) or fair_round_robin to address the issue of unfair CPU distribution between INSERTs and SELECTs. #75949 (Sergei Trifonov).
  • Restore QPL codec which has been removed in v24.10 due to licensing issues. #76021 (Konstantin Bogdanov).
  • Added function arraySymmetricDifference. It returns all elements from multiple array arguments which do not occur in all arguments. Example: SELECT arraySymmetricDifference([1, 2], [2, 3]) returns [1, 3]. (issue #61673). #76231 (Filipp Abapolov).
  • Add estimatecompressionratio aggregate function- see #70801. #76661 (Tariq Almawash).
  • FilterTransformPassedRows and FilterTransformPassedBytes profile events will show the number of rows and number of bytes filtered during the query execution. #76662 (Onkar Deshpande).
  • Added the keccak256 hash function, commonly used in blockchain implementations, especially in EVM-based systems. #76669 (Arnaud Briche).
  • Scram SHA256 & update postgres wire auth. #76839 (scanhex12).
  • The functionality adds the ability to define a list of headers that are forwarded from the headers of the client request to the external http authenticator. #77054 (inv2004).
  • Support IcebergMetadataFilesCache, which will store manifest files/list and metadata.json in one cache. #77156 (Han Fei).
  • Add functions arrayLevenshteinDistance, arrayLevenshteinDistanceWeighted, and arraySimilarity. #77187 (Mikhail f. Shiryaev).
  • Add three new functions. icebergTruncate according to specification. https://iceberg.apache.org/spec/#truncate-transform-details, toYearNumSinceEpoch and toMonthNumSinceEpoch. Support truncate transform in partition pruning for Iceberg engine. #77403 (alesapin).
  • Allows a user to query the state of an Iceberg table as it existed at a previous point in time. #77439 (Daniil Ivanik).
  • Added CPU slot scheduling for workloads, see https://clickhouse.com/docs/operations/workload-scheduling#cpu_scheduling for details. #77595 (Sergei Trifonov).
  • The hasAll() function can now take advantage of the tokenbf_v1, ngrambf_v1 full-text skipping indices. #77662 (UnamedRus).
  • JSON data type is production-ready. See https://jsonbench.com/. Dynamic and Varaint data types are production ready. #77785 (Alexey Milovidov).
  • Added an in-memory cache for deserialized vector similarity indexes. This should make repeated approximate nearest neighbor (ANN) search queries faster. The size of the new cache is controlled by server settings vector_similarity_index_cache_size and vector_similarity_index_cache_max_entries. This feature supersedes the skipping index cache feature of earlier releases. #77905 (Shankar Iyer).
  • Functions sparseGrams and sparseGramsHashes with UTF8 versions added. Author: scanhex12. #78176 (Pervakov Grigorii).
  • Introduce toInterval function. This function accepts 2 arguments (value and unit), and converts the value to a specific Interval type. #78723 (Andrew Davis).

Experimental features

  • Allow automatic cleanup merges of entire partitions after a configurable timeout with a new setting enable_replacing_merge_with_cleanup_for_min_age_to_force_merge. #76440 (Christoph Wurm).
  • Add support for Unity Catalog for DeltaLake tables on top of AWS S3 and local filesystem. #76988 (alesapin).
  • Introduce experimental integration with AWS Glue service catalog for Iceberg tables. #77257 (alesapin).

Performance improvements

  • Optimize performance with lazy projection to avoid reading unused columns. #55518 (Xiaozhe Yu).
  • Start to compare rows from most likely unequal columns first. #63780 (UnamedRus).
  • Optimize RowBinary input format. Closes #63805. #65059 (Pavel Kruglov).
  • Speedup string deserialization by some low-level optimisation. #65948 (Nikita Taranov).
  • Apply preserve_most attribute at some places in code. #67778 (Nikita Taranov).
  • Implement query condition cache to improve query performance using repeated conditions. The range of the portion of data that does not meet the condition is remembered as a temporary index in memory. Subsequent queries will use this index. close #67768 ### Documentation entry for user-facing changes. #69236 (zhongyuankai).
  • Support async io prefetch for NativeORCBlockInputFormat, which improves overall performance by hiding remote io latency. Speedup ratio could reach 1.47x in my test case. #70534 (李扬).
  • Improve grace hash join performance by rerange the right join table by keys. #72237 (kevinyhzou).
  • Reintroduce respect ttl_only_drop_parts on materialize ttl; only read the necessary columns to recalculate TTL and drop parts by replacing them with an empty one. #72751 (Andrey Zvonov).
  • Allow arrayROCAUC and arrayAUCPR to compute partial area of the whole curve, so that its calculation can be parallelized over huge datasets. #72904 (Emmanuel).
  • Avoid spawn too many idle threads. #72920 (Guo Wangyang).
  • Splitting of left table blocks by hash was removed from the probe phase of the parallel_hash JOIN algorithm. #73089 (Nikita Taranov).
  • Don't list blob storage keys if we only have curly brackets expansion in table function. Closes #73333. #73518 (Konstantin Bogdanov).
  • Replace Int256 and UInt256 with clang builtin i256 in arithmetic calculation according to tests in #70502. #73658 (李扬).
  • Adds a fast path for functions with all argument types is numeric. Fix performance issues in https://github.com/ClickHouse/ClickHouse/pull/72258. #73820 (李扬).
  • Do not apply maskedExecute on non-function columns, improve the performance of short circuit execution. #73965 (lgbo).
  • Disable header detection for Kafka/NATS/RabbitMQ/FileLog to improve performance. #74006 (Azat Khuzhin).
  • Use log wrappers by value and don't allocate them in a heap. #74034 (Mikhail Artemenko).
  • Execute a pipeline with a higher degree of parallelism after aggregation with grouping sets. #74082 (Nikita Taranov).
  • Reduce critical section in MergeTreeReadPool. #74202 (Guo Wangyang).
  • Optimized function indexHint. Now, columns that are only used as arguments of function indexHint are not read from the table. #74314 (Anton Popov).
  • Parallel replicas performance improvement. Packets deserialization on query initiator, for packets not related to parallel replicas protocol, now always happens in pipeline thread. Before, it could happen in a thread responsible for pipeline scheduling, which could make initiator less responsive and delay pipeline execution. #74398 (Igor Nikonov).
  • Fixed calculation of size in memory for LowCardinality columns. #74688 (Nikita Taranov).
  • Improves the performance of whole JSON column reading in Wide parts from S3. It's done by adding prefetches for sub column prefixes deserialization, cache of deserialized prefixes and parallel deserialization of subcolumn prefixes. It improves reading of the JSON column from S3 4 times in query like SELECT data FROM table and about 10 times in query like SELECT data FROM table LIMIT 10. #74827 (Pavel Kruglov).
  • Preallocate memory used by async inserts to improve performance. #74945 (Ilya Golshtein).
  • Fixed double pre-allocation in ConcurrentHashJoin in case join sides are swapped by the optimizer. #75149 (Nikita Taranov).
  • Fixed unnecessary contention in parallel_hash when max_rows_in_join = max_bytes_in_join = 0. #75155 (Nikita Taranov).
  • Slight improvement in some join scenarios: precalculate number of output rows and reserve memory for them. #75376 (Alexander Gololobov).
  • plain_rewritable metadata files are small and do not need a large default buffer. Use a write buffer sized appropriately to fit the given path, improving memory utilization for a large number of active parts. ### Documentation entry for user-facing changes. #75758 (Julia Kartseva).
  • In some cases (e.g., empty array column) data parts can contain empty files. We can skip writing empty blobs to ObjectStorage and only store metadata for such files when the table resides on disk with separated metadata and object storages. #75860 (Alexander Gololobov).
  • It was discovered that concurrency control could lead to unfair CPU distribution between INSERTs and SELECTs. When all CPU slots are allocated unconditionally (w/o competition) to INSERTs with max_threads = 1 while SELECTs with high max_threads values suffer from poor performance due to using only a single thread. #75941 (Sergei Trifonov).
  • Trivial opt on wrapInNullable to avoid unnecessary null map allocation. #76489 (李扬).
  • Improve min/max performance for Decimal32/Decimal64/DateTime64. #76570 (李扬).
  • Actively evict data from the cache on parts removal. Do not let the cache grow to the maximum size if the amount of data is less. #76641 (Alexey Milovidov).
  • Query compilation (setting compile_expressions) now considers the machine type. This speeds up such queries significantly. #76753 (Robert Schulze).
  • Optimize arraySort. #76850 (李扬).
  • Speed-up building JOIN result by de-virtualizing calls to col->insertFrom(). #77350 (Alexander Gololobov).
  • Merge marks of the same part and write them to the query condition cache at one time to reduce the consumption of locks. #77377 (zhongyuankai).
  • Optimize order by single nullable or low-cardinality columns. #77789 (李扬).
  • Disable filesystem_cache_prefer_bigger_buffer_size when the cache is used passively, such as for merges. #77898 (Kseniia Sumarokova).
  • Implement trivial count optimization for Iceberg. Now queries with count() and without any filters should be faster. Closes #77639. #78090 (alesapin).
  • Support Iceberg data pruning based on lower_bound and uppert_bound values for columns. Fixes #77638. #78242 (alesapin).
  • Optimize memory usage for NativeReader. #78442 (Azat Khuzhin).
  • Trivial optimization: do not rewrite count(if()) to countIf if CAST is required. Close #78564. #78565 (李扬).

Improvements

  • Decrease the amount of Keeper requests by eliminating the use of single get requests, which could have caused a significant load on Keeper with the increased number of replicas, in places where multiRead is available. #56862 (Nikolay Degterinsky).
  • Add support for SSL authentication with named collections for MySQL. Closes #59111. #59452 (Nikolay Degterinsky).
  • Improve new analyzer infrastructure performance via storing ColumnPtr instead of Field in the ConstantNode. Related to #62245. #63198 (Dmitry Novik).
  • Reject queries when the server is overloaded. The decision is made based on the ratio of wait time (OSCPUWaitMicroseconds) to busy time (OSCPUVirtualTimeMicroseconds). The query is dropped with some probability, when this ratio is between min_os_cpu_wait_time_ratio_to_throw and max_os_cpu_wait_time_ratio_to_throw (those are query level settings). #63206 (Alexey Katsman).
  • Drop blocks as early as possible to reduce the memory requirements. #65647 (lgbo).
  • processors_profile_log table now has default configuration with TTL of 30 days. #66139 (Ilya Yatsishin).
  • Allow creating of a bloom_filter index on columns with datatype DateTime64. #66416 (Yutong Xiao).
  • Introduce latency buckets and use them to track first byte read/write and connect times for S3 requests. That way we can later use gathered data to calculate approximate percentiles and adapt timeouts. #69783 (Alexey Katsman).
  • Queries passed to Executable storage are no longer limited to single threaded execution. #70084 (yawnt).
  • Added HTTP headers to OpenTelemetry span logs table for enhanced traceability. #70516 (jonymohajanGmail).
  • Support writing of orc files by custom time zone, not always by GMT time zone. #70615 (kevinyhzou).
  • Replace table functions with their -Cluster alternatives if parallel replicas are enabled. Fixes #65024. #70659 (Konstantin Bogdanov).
  • Respect IO scheduling settings when writing backups across clouds. #71093 (János Benjamin Antal).
  • Reestablish connection to MySQL and Postgres dictionary replicas in the background so it wouldn't delay requests to corresponding dictionaries. #71101 (Yakov Olkhovskiy).
  • Add metric alias name to system.asynchronous_metrics. #71164 (megao).
  • Refreshes of refreshable materialized views now appear in system.query_log. #71333 (Michael Kolupaev).
  • Evaluate parquet bloom filters and min/max indexes together. Necessary to properly support: x = 3 or x > 5 where data = [1, 2, 4, 5]. #71383 (Arthur Passos).
  • Interactive metrics improvements. Fix metrics from parallel replicas not being fully displayed. Display the metrics in order of the most recent update, then lexicographically by name. Do not display stale metrics. #71631 (Julia Kartseva).
  • Historically for some reason, the query ALTER TABLE MOVE PARTITION TO TABLE checked SELECT and ALTER DELETE rights instead of dedicated ALTER_MOVE_PARTITION. This PR makes use of this access type. For compatibility, this permission is also will be granted implicitly if SELECT and ALTER DELETE are granted, but this behavior will be removed in future releases. Closes #16403. #71632 (pufit).
  • Enables setting use_hive_partitioning by default. #71636 (Yarik Briukhovetskyi).
  • Throw an exception when trying to materialize a column in the sort key instead of allowing it to break the sort order. Does not solve #71777, though. #71891 (Peter Nguyen).
  • Allow more general join planning algorithm when hash join algorithm is enabled. #71926 (János Benjamin Antal).
  • Hide secrets in EXPLAIN QUERY TREE. #72025 (Yakov Olkhovskiy).
  • Allow use of a configurable disk to store metadata files of databases and tables. The disk name can be set via database_disk.disk config parameter. #72027 (Tuan Pham Anh).
  • Support parquet integer logical types on native reader. #72105 (Arthur Passos).
  • Make JSON output format pretty by default. Add new setting output_format_json_pretty_print to control it and enable it by default. #72148 (Pavel Kruglov).
  • Interactively request credentials in the browser if the default user requires a password. In previous versions, the server returned HTTP 403; now, it returns HTTP 401. #72198 (Alexey Milovidov).
  • This PR converts access types CREATE_USER, ALTER_USER, DROP_USER, CREATE_ROLE, ALTER_ROLE, DROP_ROLE from global to parameterized. That means users can now grant access management grants more precise:. #72246 (pufit).
  • Allow to shard names in cluster configuration. #72276 (MikhailBurdukov).
  • Support CAST and ALTER between JSON types with different parameters. #72303 (Pavel Kruglov).
  • Add the latest_fail_error_code_name column to system.mutations. We need this column to introduce a new metric on stuck mutations and use it to build graphs of the errors encountered in the cloud as well as, optionally, adding a new less-noisy alert. #72398 (Miсhael Stetsyuk).
  • Reduce the amount of allocation in attaching of partitions. #72583 (Konstantin Morozov).
  • Make max_bytes_before_external_sort limit depends on total query memory consumption (previously it was number of bytes in the sorting block for one sorting thread, now it has the same meaning as max_bytes_before_external_group_by - it is total limit for the whole query memory for all threads). Also one more setting added to control on disk block size - min_external_sort_block_bytes. #72598 (Azat Khuzhin).
  • Ignore memory restrictions by trace collector. #72606 (Azat Khuzhin).
  • Support subcolumns in MergeTree sorting key and skip indexes. #72644 (Pavel Kruglov).
  • Add server settings dictionaries_lazy_load and wait_dictionaries_load_at_startup to system.server_settings. #72664 (Christoph Wurm).
  • Adds setting max_backup_bandwidth to the list of settings that can be specified as part of BACKUP/RESTORE queries. #72665 (Christoph Wurm).
  • Parallel replicas used historical information about replica availability to improve replica selection but did not update the replica's error count when the connection was unavailable. This PR updates the replica's error count when unavailable. #72666 (zoomxi).
  • Reducing the log level for appearing replicated parts in the ReplicatedMergeTree engine to help minimize the volume of logs generated in a replicated cluster. #72876 (mor-akamai).
  • A lot of new features will require better code incapsulation (what relates to Iceberg metadata) and better code abstractions. #72941 (Daniil Ivanik).
  • Support equal comparison for values of JSON column. #72991 (Pavel Kruglov).
  • Improve formatting of identifiers with JSON subcolumns to avoid unnecessary back quotes. #73085 (Pavel Kruglov).
  • Log PREWHERE conditions with Test level. #73116 (Vladimir Cherkasov).
  • Support SETTINGS with implicit ENGINE and mixing engine and query settings. #73120 (Raúl Marín).
  • Write parts with level 1 if optimize_on_insert is enabled. It allows to use several optimizations of queries with FINAL for freshly written parts. #73132 (Anton Popov).
  • For a query like, WHERE a<b AND b<c AND c<5, can generate new comparing conditions (a<5 and b<5) to have better filtering ability. #73164 (Shichao Jin).
  • Improve extraction of common expression in disjunctions. Allow simplifying the resulting filter expression even if there's no common subexpression for all the disjuncts. Continuation of #71537. #73271 (Dmitry Novik).
  • In Storage S3(Azure)Queue allow to add settings where table was created without settings. #73283 (Kseniia Sumarokova).
  • Clickhouse-client supports Ctrl+D to finish the query. Allow using Ctrl+D instead of typing a semicolon and pressing Enter. Additionally, make Ctrl+D work as Enter in the single-line mode. #73293 (Xiaozhe Yu).
  • Introduce a setting least_greatest_legacy_null_behavior (default: false) which controls if functions least and greatest handle NULL arguments by unconditionally returning NULL (if true) or by ignoring them (if false). #73344 (Robert Schulze).
  • Use Keeper multi requests in cleanup thread ObjectStorageQueueMetadata. #73357 (Antonio Andelic).
  • A new MongoDB driver is now the default. Users who like to continue using the legacy driver can set server setting use_legacy_mongodb_integration to true. #73359 (Robert Schulze).
  • When ClickHouse runs under cgroup, we will still collect system-wide asynchronous metrics related to system load, process scheduling, memory etc. They might provide useful signals when ClickHouse is the only process on the host with high resource consumption. #73369 (Nikita Taranov).
  • In storage S3Queue allow to transfer old ordered tables created before 24.6 to new structure with buckets. #73467 (Kseniia Sumarokova).
  • Add system.azure_queue similar to existing system.s3queue. #73477 (Kseniia Sumarokova).
  • Fixed a regression that using collation locales with modifiers throws an error. As an example, SELECT arrayJoin(['kk 50', 'KK 01', ' KK 2', ' KK 3', 'kk 1', 'x9y99', 'x9y100']) item ORDER BY item ASC COLLATE 'tr-u-kn-true-ka-shifted now works. #73544 (Robert Schulze).
  • Support type Nullable(JSON). #73556 (Pavel Kruglov).
  • Function parseDateTime64 (and its variants) now produces correct results for input dates before 1970 / after 2106. Example: SELECT parseDateTime64InJodaSyntax('2200-01-01 00:00:00.000', 'yyyy-MM-dd HH:mm:ss.SSS'). #73594 (zhanglistar).
  • Address some clickhouse-disks usability issues addressed by users. Closes #67136. #73616 (Daniil Ivanik).
  • Allow to alter commit settings in storage S3(Azure)Queue. (Commit settings are: max_processed_files_before_commit, max_processed_rows_before_commit, max_processed_bytes_before_commit, max_processing_time_sec_before_commit). #73635 (Kseniia Sumarokova).
  • In storage S3(Azure)Queue aggregate progress between sources to compare with commit limit settings. #73641 (Kseniia Sumarokova).
  • Support core settings in BACKUP/RESTORE query. #73650 (Vitaly Baranov).
  • Read output_format_compression_level on Parquet output. #73651 (Arthur Passos).
  • Adds reading arrow fixed_size_list as an Array instead of treating it as an unsupported type. #73654 (Julian Meyers).
  • This PR adds two backup engines: Memory (keeps backups inside the current user session), and Null (doesn't keep backups anywhere):. #73690 (Vitaly Baranov).
  • concurrent_threads_soft_limit_num and concurrent_threads_soft_limit_num_ratio_to_cores could be changed w/o restart of a server. #73713 (Sergei Trifonov).
  • Add support for extended numeric types (Decimal, big integers) in formatReadable functions. #73765 (Raúl Marín).
  • Respect column insensitive column matching for fields in tuple columns. Close https://github.com/apache/incubator-gluten/issues/8324. #73780 (李扬).
  • Support TLS for Postgres wire protocol. #73812 (scanhex12).
  • Allow LowCardinality(UUID) by default. This has proven practical among ClickHouse Cloud customers. #73826 (Alexey Milovidov).
  • Better message during installation. #73827 (Alexey Milovidov).
  • Better message about password reset for ClickHouse Cloud. #73831 (Alexey Milovidov).
  • Improve the error message with a File table that cannot perform appends into a file. #73832 (Alexey Milovidov).
  • Ask when a user accidentally requests to output binary format (such as Native, Parquet, Avro) in the terminal. This closes #59524. #73833 (Alexey Milovidov).
  • The BFloat16 data type is production-ready. #73840 (Alexey Milovidov).
  • Highlight trailing spaces in Pretty and Vertical formats in the terminal for better clarity. This is controlled with the output_format_pretty_highlight_trailing_spaces setting. Initial implementation by Braden Burns from #72996. Closes #71590. #73847 (Alexey Milovidov).
  • clickhouse-client and clickhouse-local will autodetect compression of stdin when it is redirected from a file. This closes #70865. #73848 (Alexey Milovidov).
  • Cut too long column names in pretty formats by default. This is controlled by the output_format_pretty_max_column_name_width_cut_to and output_format_pretty_max_column_name_width_min_chars_to_cut settings. This is the continuation of the work of tanmaydatta in #66502. This closes #65968. #73851 (Alexey Milovidov).
  • Make Pretty formats prettier: squash blocks if not much time passed since the output of the previous block. This is controlled by new settings output_format_pretty_squash_consecutive_ms (50 ms by default) and output_format_pretty_squash_max_wait_ms (1000 ms by default). Continuation of #49537. This closes #49153. #73852 (Alexey Milovidov).
  • Fix type mapping for SQLite (integer types into int64, floating points into float64). #73853 (Joanna Hulboj).
  • Add a metric on the number of currently merging source parts. This closes #70809. #73868 (Alexey Milovidov).
  • Highlight columns in the Vertical format if the output is to a terminal. This can be disabled with the output_format_pretty_color setting. #73898 (Alexey Milovidov).
  • Enhanced the MySQL compatibility to a level that now, mysqlsh (a rich MySQL CLI from Oracle) can connect to ClickHouse. This is needed to facilitate testing. #73912 (Alexey Milovidov).
  • If the number of output records is larger than N = output_format_pretty_max_rows, instead of displaying only the first N rows, we will cut the output table in the middle, displaying N/2 first rows and N/2 last rows. Continuation of #64200. This closes #59502. #73929 (Alexey Milovidov).
  • The function isIPv4String returned true if the correct IPv4 address was followed by a zero byte, while it should return false in this case. Continuation of #65387. #73946 (Alexey Milovidov).
  • Make the error code in the MySQL wire protocol compatible with MySQL. Continuation of #56831. Closes #50957. #73948 (Alexey Milovidov).
  • Add setting validate_enum_literals_in_opearators to validate enum literals in operators like IN, NOT IN against the enum type and throw an exception if the literal is not a valid enum value. #73985 (Vladimir Cherkasov).
  • In Storage S3(Azure)Queue commit all files (in a single butch defined by commit settings) in a single keeper transaction. #73991 (Kseniia Sumarokova).
  • Disable header detection for executable UDFs and dictionaries (could lead to Function 'X': wrong result, expected Y row(s), actual Y-1). #73992 (Azat Khuzhin).
  • Add the distributed option for EXPLAIN PLAN. Now, EXPLAIN distributed=1 ... appends remote plan to ReadFromParallelRemote* steps. #73994 (Nikolai Kochetov).
  • Use correct return type for not/xor with Dynamic arguments. #74013 (Pavel Kruglov).
  • Allow changing add_implicit_sign_column_constraint_for_collapsing_engine after table creation. #74014 (Christoph Wurm).
  • Support subcolumns in materialized view select query. #74030 (Pavel Kruglov).
  • Pretty formats can render multi-line fields inside a table cell, which improves readability. This is enabled by default and can be controlled by the setting output_format_pretty_multiline_fields. Continuation of the work by Volodyachan in #64094. This closes #56912. #74032 (Alexey Milovidov).
  • Support predicate-push-down optimization on the query plan level for the MergingAggregated step. It improves performance for some queries with the new analyzer. #74073 (Nikolai Kochetov).
  • There are now three simple ways to set a custom prompt in clickhouse-client: 1. via command-line parameter --prompt, 2. in the configuration file, via settings <prompt>[...]</prompt>, and 3. also in the configuration file, via per-connection settings <connections_credentials><prompt>[...]</prompt></connection_credentials>. #74168 (Christoph Wurm).
  • Change prometheus remote write response success status from 200/OK to 204/NoContent. #74170 (Michael Dempsey).
  • Expose X-ClickHouse HTTP headers to JavaScript in the browser. It makes writing applications more convenient. #74180 (Alexey Milovidov).
  • The JSONEachRowWithProgress format will include events with metadata, as well as totals and extremes. It also includes rows_before_limit_at_least and rows_before_aggregation. The format prints the exception properly if it arrives after partial results. The progress now includes elapsed nanoseconds. One final progress event is emitted at the end. The progress during query runtime will be printed no more frequently than the value of the interactive_delay setting. #74181 (Alexey Milovidov).
  • Hourglass will rotate smoothly in Play UI. #74182 (Alexey Milovidov).
  • Even if the HTTP response is compressed, send packets as soon as they arrive. This allows the browser to receive progress packets and compressed data. #74201 (Alexey Milovidov).
  • Add ability to reload max_remote_read_network_bandwidth_for_serve and max_remote_write_network_bandwidth_for_server on fly without restart server. #74206 (Kai Zhu).
  • Autodetect secure connection based on connecting to port 9440 in ClickHouse Client. #74212 (Christoph Wurm).
  • Authenticate users with username only for http_handlers (previously it requires user to put the password as well). #74221 (Azat Khuzhin).
  • Support for the alternative query languages PRQL and KQL was marked experimental. To use them, specify settings allow_experimental_prql_dialect = 1 and allow_experimental_kusto_dialect = 1. #74224 (Robert Schulze).
  • Support returning the default Enum type in more aggregate functions. #74272 (Raúl Marín).
  • In OPTIMIZE TABLE, it is now possible to specify keyword FORCE as an alternative to existing keyword FINAL. #74342 (Robert Schulze).
  • Added a merge tree setting materialize_skip_indexes_on_merge which suppresses the creation of skip indexes during merge. This allows users to control explicitly (via ALTER TABLE [..] MATERIALIZE INDEX [...]) when skip indexes are created. This can be useful if skip indexes are expensive to build (e.g. vector similarity indexes). #74401 (Robert Schulze).
  • Support subcolumns in default and materialized expressions. #74403 (Pavel Kruglov).
  • Optimize keeper requests in Storage(S3/Azure)Queue. #74410 (Kseniia Sumarokova).
  • Add the IsServerShuttingDown metric, which is needed to trigger an alert when the server shutdown takes too much time. #74429 (Miсhael Stetsyuk).
  • Added iceberg tables names to EXPLAIN. #74485 (alekseev-maksim).
  • Use up to 1000 parallel replicas by default. #74504 (Konstantin Bogdanov).
  • Provide a better error message when using RECURSIVE CTE with the old analyzer. #74523 (Raúl Marín).
  • Optimize keeper requests in Storage(S3/Azure)Queue. #74538 (Kseniia Sumarokova).
  • Improve HTTP session reuse when reading from s3 disk (#72401). #74548 (Julian Maicher).
  • Show extended error messages in system.errors. #74574 (Vitaly Baranov).
  • Enabled a backoff logic for all types of replicated tasks. It will provide the ability to reduce CPU usage, memory usage, and log file sizes. Added new settings max_postpone_time_for_failed_replicated_fetches_ms, max_postpone_time_for_failed_replicated_merges_ms and max_postpone_time_for_failed_replicated_tasks_ms which are similar to max_postpone_time_for_failed_mutations_ms. #74576 (MikhailBurdukov).
  • More accurate accounting for max_joined_block_size_rows setting for parallel_hash JOIN algorithm. Helps to avoid increased memory consumption compared to hash algorithm. #74630 (Nikita Taranov).
  • Added dfs.client.use.datanode.hostname libhdfs3 config option support. #74635 (Mikhail Tiukavkin).
  • Fixes Invalid: Codec 'snappy' doesn't support setting a compression level. #74659 (Arthur Passos).
  • Allow using password for client communication with clickhouse-keeper. This feature is not very useful if you specify proper SSL configuration for server and client, but still can be useful for some cases. Password cannot be longer than 16 characters. It's not connected with Keeper Auth model. #74673 (alesapin).
  • Allow using blob paths to calculate checksums while making a backup. #74729 (Vitaly Baranov).
  • Use dynamic sharding for JOIN if the JOIN key is a prefix of PK for both parts. This optimization is enabled with query_plan_join_shard_by_pk_ranges setting (disabled by default). #74733 (Nikolai Kochetov).
  • Add error code for config reloader. #74746 (Garrett Thomas).
  • Added support for IPv6 addresses in MySQL and PostgreSQL table functions and engines. #74796 (Mikhail Koviazin).
  • Parameters for the codec Gorilla will now always be saved in the table metadata in .sql file. This closes: #70072. #74814 (Nikita Mikhaylov).
  • Implement short circuit optimization for divideDecimal. Fixes #74280. #74843 (Kevin Mingtarja).
  • Improve performance of larger multi requests in Keeper. #74849 (Antonio Andelic).
  • Now users can be specified inside the startup scripts. #74894 (pufit).
  • Fetch parts in parallel in ALTER TABLE FETCH PARTITION (thread pool size is controlled with max_fetch_partition_thread_pool_size). #74978 (Azat Khuzhin).
  • Added a query ID column to system.query_cache (issue #68205). #74982 (NamNguyenHoai).
  • Enabled SSH protocol back. Fixed some critical vulnerabilities so that it is no longer possible to use custom pager or specify server-logs-file. Disabled the ability to pass client options through the environment variables by default (it is still possible via ssh-server.enable_client_options_passing in config.xml). Supported progress table, query cancellation, completion, profile events progress, stdin and send_logs_level option. This closes: #74340. #74989 (Nikita Mikhaylov).
  • Fix formatting of exceptions using a custom format if they appear during query interpretation. In previous versions, exceptions were formatted using the default format rather than the format specified in the query. This closes #55422. #74994 (Alexey Milovidov).
  • Implemented parsing enhancements (Sequence ID parsing: Added functionality to parse sequence identifiers in manifest files AND Avro metadata parsing: Redesigned the Avro metadata parser to be easily extendable for future enhancements). #75010 (Daniil Ivanik).
  • It is allowed to cancel ALTER TABLE ... FREEZE ... queries with KILL QUERY and timeout(max_execution_time). #75016 (Kirill).
  • Add support for groupUniqArrayArrayMap as SimpleAggregateFunction. #75034 (Miel Donkers).
  • Support prepared statements in postgres wire protocol. #75035 (scanhex12).
  • Hide catalog credential settings in database engine Iceberg. Closes #74559. #75080 (Kseniia Sumarokova).
  • Added a few missing features into BuzzHouse: ILIKE and REGEXP operators, <=> and IS NOT DISTINCT FROM. #75168 (Pedro Ferreira).
  • The setting min_chunk_bytes_for_parallel_parsing cannot be zero anymore. This fixes: #71110. #75239 (Nikita Mikhaylov).
  • intExp2 / intExp10: Define undefined behaviour: return 0 for too small argument, 18446744073709551615 for too big argument, throw exception if nan. #75312 (Vitaly Baranov).
  • Support s3.endpoint natively from catalog config in DatabaseIceberg. Closes #74558. #75375 (Kseniia Sumarokova).
  • Don't fail silently if user executing SYSTEM DROP REPLICA doesn't have enough permissions. #75377 (Bharat Nallan).
  • Add a ProfileEvent about the number of times any of system logs has failed to flush. #75466 (Alexey Milovidov).
  • Add check and logging for decrypting and decompressing. #75471 (Vitaly Baranov).
  • Added support for the micro sign (U+00B5) in the parseTimeDelta function. Now both the micro sign (U+00B5) and the Greek letter mu (U+03BC) are recognized as valid representations for microseconds, aligning ClickHouse's behavior with Go’s implementation (see time.go and time/format.go). #75472 (Vitaly Orlov).
  • Replace server setting (send_settings_to_client) with client setting (apply_settings_from_server) that controls whether client-side code (e.g. parsing INSERT data and formatting query output) should use settings from server's users.xml and user profile. Otherwise only settings from client command line, session, and the query are used. Note that this only applies to native client (not e.g. HTTP), and doesn't apply to most of query processing (which happens on the server). #75478 (Michael Kolupaev).
  • Keeper improvement: disable digest calculation when committing to in-memory storage for better performance. It can be enabled with keeper_server.digest_enabled_on_commit config. Digest is still calculated when preprocessing requests. #75490 (Antonio Andelic).
  • Push down filter expression from JOIN ON when possible. #75536 (Vladimir Cherkasov).
  • Better error messages at syntax errors. Previously, if the query was too large, and the token whose length exceeds the limit is a very large string literal, the message about the reason was lost in the middle of two examples of this very long token. Fix the issue when a query with UTF-8 was cut incorrectly in the error message. Fix excessive quoting of query fragments. This closes #75473. #75561 (Alexey Milovidov).
  • Add profile events in storage S3(Azure)Queue. #75618 (Kseniia Sumarokova).
  • Disable sending settings from server to client (send_settings_to_client=false) for compatibility (This feature will be re-implemented as client setting later for better usability). #75648 (Michael Kolupaev).
  • Add a config memory_worker_correct_memory_tracker to enable correction of internal memory tracker with information from different source read in the background thread periodically. #75714 (Antonio Andelic).
  • Use Analyzer in PrometheusRemoteReadProtocol. #75729 (Dmitry Novik).
  • We have support for gauge/counter metric types. However, they are insufficient for some metrics (e.g., the response times of requests to the keeper), so support for the histogram metric type is needed. The interface closely mirrors the Prometheus client, where you simply call observe(value) to increment the counter in the bucket corresponding to the value. The histogram metrics are exposed via system.histogram_metrics. #75736 (Miсhael Stetsyuk).
  • Add column normalized_query_hash into system.processes. Note: while it can be easily calculated on the fly with the normalizedQueryHash function, this is needed to prepare for subsequent changes. #75756 (Alexey Milovidov).
  • Querying system.tables will not throw even if there is a Merge table created over a database that no longer exists. Remove the getTotalRows method from Hive tables, because we don't allow it to do complex work. #75772 (Alexey Milovidov).
  • Web UI now has interactive database navigation. #75777 (Alexey Milovidov).
  • Allow to combine read-only and read-write disks in storage policy (as multiple volumes, or multiple disks). This allows to read data from the entire volume, while inserts will prefer the writable disk (i.e. Copy-on-Write storage policy). #75862 (Azat Khuzhin).
  • Remove trace_id from default ORDER BY for system.opentelemetry_span_log. #75907 (Azat Khuzhin).
  • Encryption (XML attribute encrypted_by) can now be applied to any configuration file (config.xml, users.xml, nested configuration files). Previously, it worked only for the top-level config.xml file. #75911 (Mikhail Gorshkov).
  • Store start_time/end_time for Backups with microseconds. #75929 (Aleksandr Musorin).
  • Add MemoryTrackingUncorrected metric showing value of internal global memory tracker which is not corrected by RSS. #75935 (Antonio Andelic).
  • Calculate columns and indices sizes lazily in MergeTree. #75938 (Pavel Kruglov).
  • Convert join to in subquery if output column is tied to the left table, need a uniqueness step at first, so disabled by default until the step is added later. #75942 (Shichao Jin).
  • Added a server setting throw_on_unknown_workload that allows to choose behavior on query with workload setting set to unknown value: either allow unlimited access (default) or throw a RESOURCE_ACCESS_DENIED error. It is useful to force all queries to use workload scheduling. #75999 (Sergei Trifonov).
  • Make the new, experimental Kafka table engine fully respect Keeper feature flags. #76004 (János Benjamin Antal).
  • Don't rewrite subcolumns to getSubcolumn in ARRAY JOIN if not necessary. #76018 (Pavel Kruglov).
  • Retry coordination errors when loading tables. #76020 (Alexander Tokmakov).
  • Improve the system.warnings table and add some dynamic warning messages that can be added, updated or removed. #76029 (Bharat Nallan).
  • Support flushing individual logs in SYSTEM FLUSH LOGS. #76132 (Raúl Marín).
  • Improved the /binary server's page. Using the Hilbert curve instead of the Morton curve. Display 512 MB worth of addresses in the square, which fills the square better (in previous versions, addresses fill only half of the square). Color addresses closer to the library name rather than the function name. Allow scrolling a bit more outside of the area. #76192 (Alexey Milovidov).
  • This PR makes it impossible to run a query ALTER USER user1 ADD PROFILES a, DROP ALL PROFILES because all DROP operations should come first in the order. #76242 (pufit).
  • Various enhancements for SYNC REPLICA (better error messages, better tests, sanity checks). #76307 (Azat Khuzhin).
  • Retry ON CLUSTER queries in case of TOO_MANY_SIMULTANEOUS_QUERIES. #76352 (Patrick Galbraith).
  • Changed the default value of output_format_pretty_max_rows from 10000 to 1000. I think it is better for usability. #76407 (Alexey Milovidov).
  • Support for a refresh in readonly MergeTree tables. #76467 (Alexey Milovidov).
  • Use correct fallback when multipart copy to S3 fails during backup with Access Denied. Multi part copy can generate Access Denied error when backup is done between buckets that have different credentials. #76515 (Antonio Andelic).
  • Faster ClickHouse Servers shutdown (get rid of 2.5sec delay). #76550 (Azat Khuzhin).
  • Add query_id to system.errors. Related ticket #75815. #76581 (Vladimir Baikov).
  • Upgraded librdkafka to version 2.8.0 and improved the shutdown sequence for Kafka tables, reducing delays during table drops and server restarts. The engine=Kafka no longer explicitly leaves the consumer group when a table is dropped. Instead, the consumer remains in the group until it is automatically removed after session_timeout_ms (default: 45 seconds) of inactivity. #76621 (filimonov).
  • Fix validation of s3 request settings. #76658 (Vitaly Baranov).
  • Avoid excess allocation in readbufferfroms3 and other remote reading buffers, reduce their memory consumption in half. #76692 (Sema Checherinda).
  • Support JSON type and subcolumns reading from View. #76903 (Pavel Kruglov).
  • Adding Support for Converting UInt128 to IPv6. This allows the bitAnd operation and arithmatics for IPv6 and conversion back to IPv6. Closes #76752. This allows the result from bitAnd operation on IPv6 to be converted back to IPv6, as well. See: https://github.com/ClickHouse/ClickHouse/pull/57707. #76928 (Muzammil Abdul Rehman).
  • System tables like server_settings or settings have a default value column which is convenient. only merge_tree_settings and replicated_merge_tree_settings do not have that column enabled. #76942 (Diego Nieto).
  • Don't parse special Bool values in text formats inside Variant type by default. It can be enabled using setting allow_special_bool_values_inside_variant. #76974 (Pavel Kruglov).
  • Support configurable per task waiting time of low priority query in session level and in server level. #77013 (VicoWu).
  • Added ProfileEvents::QueryPreempted, which has the same logic as CurrentMetrics::QueryPreempted. #77015 (VicoWu).
  • Previously database replicated might print credentials specified in a query to logs. This behaviour is fixed. This closes: #77123. #77133 (Nikita Mikhaylov).
  • Bump zstd from 1.5.5 to 1.5.7 which has pretty good performance improvements. #77137 (Pradeep Chhetri).
  • Allow ALTER TABLE DROP PARTITION for plain_rewritable disk. #77138 (Julia Kartseva).
  • Add the ability to randomly sleep up to 500ms independent of part sizes before merges/mutations execution in case of zero-copy replication. #77165 (Alexey Katsman).
  • Support atomic rename when TRUNCATE is used with INTO OUTFILE. Resolves #70323. #77181 (Onkar Deshpande).
  • Use FixedString for PostgreSQL's CHARACTER, CHAR and BPCHAR. #77304 (Pablo Marcos).
  • Allow to explicitly specify metadata file to read for Iceberg with storage/table function setting iceberg_metadata_file_path . Fixes #47412. #77318 (alesapin).
  • Support using a remote disk for databases to store metadata files. #77365 (Tuan Pham Anh).
  • Implement comparison for values of JSON data type. Now JSON objects can be compared similarly to Maps. #77397 (Pavel Kruglov).
  • Change reverted. #77399 (Yarik Briukhovetskyi).
  • Backup/restore setting allow_s3_native_copy now supports value three possible values: - False - s3 native copy will not be used; - True (old default) - ClickHouse will try s3 native copy first, if it fails then fallback to the reading+writing approach; - 'auto' (new default) - ClickHouse will compare the source and destination credentials first. If they are same, ClickHouse will try s3 native copy and then may fallback to the reading+writing approach. If they are different, ClickHouse will go directly to the reading+writing approach. #77401 (Vitaly Baranov).
  • Support ALTER TABLE ... ATTACH|DETACH|MOVE|REPLACE PARTITION for the plain_rewritable disk. #77406 (Julia Kartseva).
  • Skipping index cache is reverted. #77447 (Nikita Mikhaylov).
  • Reduce memory usage during prefetches of JSON column in Wide parts. #77640 (Pavel Kruglov).
  • Support aws session token and environment credentials usage in delta kernel for DeltaLake table engine. #77661 (Kseniia Sumarokova).
  • Support query parameters inside additional_table_filters setting. After the change, the following query would succeed:. #77680 (wxybear).
  • User-defined functions (UDFs) can now be marked as deterministic via a new tag in their XML definition. Also, the query cache now checks if UDFs called within a query are deterministic. If this is the case, it caches the query result. (Issue #59988). #77769 (Jimmy Aguilar Mena).
  • Added Buffer table engine parameters validation. #77840 (Pervakov Grigorii).
  • Add config enable_hdfs_pread to enable or disable hdfs pread. #77885 (kevinyhzou).
  • Add profile events for number of zookeeper 'multi' read and write requests. #77888 (JackyWoo).
  • Allow creating and inserting into temp table when disable_insertion_and_mutation is on. #77901 (Xu Jia).
  • Decrease max_insert_delayed_streams_for_parallel_write (to 100). #77919 (Azat Khuzhin).
  • Add ability to configure number of columns that merges can flush in parallel using max_merge_delayed_streams_for_parallel_write (this should reduce memory usage for vertical merges to S3 about 25x times). #77922 (Azat Khuzhin).
  • Fix year parsing in joda syntax like 'yyy'. #77973 (李扬).
  • Attaching parts of MergeTree tables will be performed in their block order, which is important for special merging algorithms, such as ReplacingMergeTree. This closes #71009. #77976 (Alexey Milovidov).
  • Query masking rules are now able to throw a LOGICAL_ERROR in case if the match happened. This will help to check if pre-defined password is leaking anywhere in logs. #78094 (Nikita Mikhaylov).
  • Added column index_length_column to information_schema.tables for better compatibility with MySQL. #78119 (Paweł Zakrzewski).
  • Introduce two new metrics: TotalMergeFailures and NonAbortedMergeFailures. These metrics are needed to detect the cases where too many merges fail within a short period. #78150 (Miсhael Stetsyuk).
  • Fix incorrect S3 uri parsing when key is not specified on path style. #78185 (Arthur Passos).
  • Fix incorrect values of BlockActiveTime, BlockDiscardTime, BlockWriteTime, BlockQueueTime, and BlockReadTime asynchronous metrics (before the change 1 second was incorrectly reported as 0.001). #78211 (filimonov).
  • Respect loading_retries limit for errors during push to materialized view for StorageS3(Azure)Queue. Before that such errors were retried indefinitely. #78313 (Kseniia Sumarokova).
  • In StorageDeltaLake with delta-kernel-rs implementation, fix performance and progress bar. #78368 (Kseniia Sumarokova).
  • Vector similarity index could over-allocate main memory by up to 2x. This fix reworks the memory allocation strategy, reducing the memory consumption and improving the effectiveness of the vector similarity index cache. (issue #78056). #78394 (Shankar Iyer).
  • Introduce a setting schema_type for system.metric_log table with schema type. There are three allowed schemas: wide -- current schema, each metric/event in a separate column (most effective for reads of separate columns), transposed -- similar to system.asynchronous_metric_log, metrics/events are stored as rows, and the most interesting transposed_with_wide_view -- create underlying table with transposed schema, but also introduce a view with wide schema which translates queries to underlying table. In transposed_with_wide_view subsecond resolution for view is not supported, event_time_microseconds is just an alias for backward compatibility. #78412 (alesapin).
  • Support include, from_env, from_zk for runtime disks. Closes #78177. #78470 (Kseniia Sumarokova).
  • Add several convenient ways to resolve root metadata.json file in an iceberg table function and engine. Closes #78455. #78475 (Daniil Ivanik).
  • Support partition pruning in delta lake. #78486 (Kseniia Sumarokova).
  • Support password based auth in SSH protocol in ClickHouse. #78586 (Nikita Mikhaylov).
  • Add a dynamic warning to the system.warnings table for long running mutations. #78658 (Bharat Nallan).
  • Drop connections if the CPU is massively overloaded. The decision is made based on the ratio of wait time (OSCPUWaitMicroseconds) to busy time (OSCPUVirtualTimeMicroseconds). The query is dropped with some probability, when this ratio is between min_os_cpu_wait_time_ratio_to_drop_connection and max_os_cpu_wait_time_ratio_to_drop_connection. #78778 (Alexey Katsman).
  • Allow empty value on hive partitioning. #78816 (Arthur Passos).
  • Fix IN clause type coercion for BFloat16 (i.e. SELECT toBFloat16(1) IN [1, 2, 3]; now returns 1). Closes #78754. #78839 (Raufs Dunamalijevs).
  • Do not check parts on other disks for MergeTree if disk= is set. #78855 (Azat Khuzhin).
  • Make data types in used_data_type_families in system.query_log canonical. #78972 (Kseniia Sumarokova).

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

  • Fix cannot create SEQUENTIAL node with keeper-client. #64177 (Duc Canh Le).
  • Fix identifier resolution from parent scopes. Allow the use of aliases to expressions in the WITH clause. Fixes #58994. Fixes #62946. Fixes #63239. Fixes #65233. Fixes #71659. Fixes #71828. Fixes #68749. #66143 (Dmitry Novik).
  • Fix incorrect character counting in PositionImpl::vectorVector. #71003 (思维).
  • Fix negate function monotonicity. In previous versions, the query select * from a where -x = -42; where x is the primary key, can return a wrong result. #71440 (Michael Kolupaev).
  • RESTORE operations for access entities required more permission than necessary because of unhandled partial revokes. This PR fixes the issue. Closes #71853. #71958 (pufit).
  • Avoid pause after ALTER TABLE REPLACE/MOVE PARTITION FROM/TO TABLE. Retrieve correct settings for background task scheduling. #72024 (Aleksei Filatov).
  • Fix empty tuple handling in arrayIntersect. This fixes #72578. #72581 (Amos Bird).
  • Fix handling of empty tuples in some input and output formats (e.g. Parquet, Arrow). #72616 (Michael Kolupaev).
  • Column-level GRANT SELECT/INSERT statements on wildcard databases/tables now throw an error. #72646 (Johann Gan).
  • Fix the situation when a user can't run REVOKE ALL ON *.* because of implicit grants in the target access entity. #72872 (pufit).
  • Fix stuck while processing pending batch for async distributed INSERT (due to i.e. No such file or directory). #72939 (Azat Khuzhin).
  • Add support for Azure SAS Tokens. #72959 (Azat Khuzhin).
  • Fix positive timezone formatting of formatDateTime scalar function. #73091 (ollidraese).
  • Fix to correctly reflect source port when connection made through PROXYv1 and auth_use_forwarded_address is set - previously proxy port was incorrectly used. Add currentQueryID() function. #73095 (Yakov Olkhovskiy).
  • Propagate format settings to NativeWriter in TCPHandler, so settings like output_format_native_write_json_as_string are applied correctly. #73179 (Pavel Kruglov).
  • Fix reading JSON sub-object subcolumns with incorrect prefix. #73182 (Pavel Kruglov).
  • Fix crash in StorageObjectStorageQueue. #73274 (Kseniia Sumarokova).
  • Fix rare crash in refreshable materialized view during server shutdown. #73323 (Michael Kolupaev).
  • The %f placeholder of function formatDateTime now unconditionally generates six (sub-second) digits. This makes the behavior compatible with MySQL DATE_FORMAT function. The previous behavior can be restored using setting formatdatetime_f_prints_scale_number_of_digits = 1. #73324 (ollidraese).
  • Improved datetime conversion during index analysis by enforcing saturating behavior for implicit Date to DateTime conversions. This resolves potential index analysis inaccuracies caused by datetime range limitations. This fixes #73307. It also fixes explicit toDateTime conversion when date_time_overflow_behavior = 'ignore' which is the default value. #73326 (Amos Bird).
  • Fixed filtering by _etag column while reading from s3 storage and table function. #73353 (Anton Popov).
  • Fix Not-ready Set is passed as the second argument for function 'in' error when IN (subquery) is used in JOIN ON expression, with the old analyzer. #73382 (Nikolai Kochetov).
  • Fix preparing for squashin for Dynamic and JSON columns. Previously in some cases new types could be inserted into shared variant/shared data even when the limit on types/paths is not reached. #73388 (Pavel Kruglov).
  • Check for corrupted sizes during types binary decoding to avoid too big allocations. #73390 (Pavel Kruglov).
  • Fixed a logical error when reading from single-replica cluster with parallel replicas enabled. #73403 (Michael Kolupaev).
  • Fix ObjectStorageQueue with ZooKeeper and older Keeper. #73420 (Antonio Andelic).
  • Implements fix, needed to enable hive partitioning by default. #73479 (Yarik Briukhovetskyi).
  • Fix data race when creating vector similarity index. #73517 (Antonio Andelic).
  • Fixes segfault when the source of the dictionary contains a function with wrong data. #73535 (Yarik Briukhovetskyi).
  • Fix retries on failed insert in storage S3(Azure)Queue. Closes #70951. #73546 (Kseniia Sumarokova).
  • Fixed error in function tupleElement which may appear in some cases for tuples with LowCardinality elelments and enabled setting optimize_functions_to_subcolumns. #73548 (Anton Popov).
  • Fix parsing enum glob followed by range one. Fixes #73473. #73569 (Konstantin Bogdanov).
  • Fixed parallel_replicas_for_non_replicated_merge_tree being ignored in subqueries for non-replicated tables. #73584 (Igor Nikonov).
  • Fix for std::logical_error thrown when a task cannot be scheduled. Found in stress tests. Example stacktrace: 2024.12.19 02:05:46.171833 [ 18190 ] {01f0daba-d3cc-4898-9e0e-c2c263306427} <Fatal> : Logical error: 'std::exception. Code: 1001, type: std::__1::future_error, e.what() = The associated promise has been destructed prior to the associated state becoming ready. (version 25.1.1.18724), Stack trace:. #73629 (Alexander Gololobov).
  • Do not interpret queries in EXPLAIN SYNTAX to avoid logical errors with incorrect processing stage for distributed queries. Fixes #65205. #73634 (Dmitry Novik).
  • Fix possible data inconsistency in Dynamic column. Fixes possible logical error Nested columns sizes are inconsistent with local_discriminators column size. #73644 (Pavel Kruglov).
  • Fixed NOT_FOUND_COLUMN_IN_BLOCK in queries with FINAL and SAMPLE. Fixed incorrect result in selects with FINAL from CollapsingMergeTree and enabled optimizations of FINAL . #73682 (Anton Popov).
  • Fix crash in LIMIT BY COLUMNS. #73686 (Raúl Marín).
  • Fix the bug when the normal projection is forced to use, and query is exactly the same as the projection defined, but the projection is not selected and thus error is prompted. #73700 (Shichao Jin).
  • Fix deserialization of Dynamic/Object structure. It could lead to CANNOT_READ_ALL_DATA exceptions. #73767 (Pavel Kruglov).
  • Skip metadata_version.txt in while restoring parts from a backup. #73768 (Vitaly Baranov).
  • Fix #73737. #73775 (zhanglistar).
  • Fixes #72078 ( S3 Express Support was broken ). #73777 (Sameer Tamsekar).
  • Allow merging of rows with invalid sign column values in CollapsingMergeTree tables. #73864 (Christoph Wurm).
  • Fix the following error Row 1: ────── hostname: c-test-wy-37-server-nlkyjyb-0.c-test-wy-37-server-headless.ns-test-wy-37.svc.cluster.local type: ExceptionWhileProcessing event_date: 2024-12-23 event_time: 2024-12-23 16:21:19 event_time_microseconds: 2024-12-23 16:21:19.824624 query_start_time: 2024-12-23 16:21:19 query_start_time_microseconds: 2024-12-23 16:21:19.747142 query_duration_ms: 77 read_rows: 1 read_bytes: 134 written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 memory_usage: 7824 current_database: default query: CREATE DATABASE db0 formatted_query: normalized_query_hash: 7820917191074023511 -- 7.82 quintillion query_kind: Create databases: ['db0'] tables: [] columns: [] partitions: [] projections: [] views: [] exception_code: 170 exception: Code: 170. DB::Exception: Bad get: has Null, requested Int64: While executing DDLOnClusterQueryStatus. (BAD_GET) (version 25.1.1.19134 (official build)) stack_trace: 0. ./build_docker/./src/Common/Exception.cpp:107: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000da5e53b 1. DB::Exception::Exception(PreformattedMessage&&, int) @ 0x00000000088aca4c 2. DB::Exception::Exception<std::basic_string_view<char, std::char_traits<char>>, std::basic_string_view<char, std::char_traits<char>>>(int, FormatStringHelperImpl<std::type_identity<std::basic_string_view<char, std::char_traits<char>>>::type, std::type_identity<std::basic_string_view<char, std::char_traits<char>>>::type>, std::basic_string_view<char, std::char_traits<char>>&&, std::basic_string_view<char, std::char_traits<char>>&&) @ 0x00000000088bae8b 3. auto& DB::Field::safeGet<long>() & @ 0x0000000008a3c748 4. ./src/Core/Field.h:484: DB::ColumnVector<long>::insert(DB::Field const&) @ 0x0000000012e44c0f 5. ./build_docker/./src/Interpreters/DDLOnClusterQueryStatusSource.cpp:53: DB::DDLOnClusterQueryStatusSource::generateChunkWithUnfinishedHosts() const @ 0x0000000012a40214 6. ./build_docker/./src/Interpreters/DDLOnClusterQueryStatusSource.cpp:104: DB::DDLOnClusterQueryStatusSource::handleTimeoutExceeded() @ 0x0000000012a41640 7. ./build_docker/./src/Interpreters/DDLOnClusterQueryStatusSource.cpp:109: DB::DDLOnClusterQueryStatusSource::stopWaitingOfflineHosts() @ 0x0000000012a41be9 8. ./build_docker/./src/Interpreters/DistributedQueryStatusSource.cpp:182: DB::DistributedQueryStatusSource::generate() @ 0x0000000011feb3bf 9. ./build_docker/./src/Processors/ISource.cpp:139: DB::ISource::tryGenerate() @ 0x0000000014148f5b 10. ./build_docker/./src/Processors/ISource.cpp:108: DB::ISource::work() @ 0x0000000014148c47 11. ./build_docker/./src/Processors/Executors/ExecutionThreadContext.cpp:49: DB::ExecutionThreadContext::executeTask() @ 0x0000000014164fc7 12. ./build_docker/./src/Processors/Executors/PipelineExecutor.cpp:290: DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic<bool>*) @ 0x00000000141577e5. #73876 (Tuan Pham Anh).
  • Fixes occasional failure to compare map() types due to possibility to create Map lacking explicit naming ('keys','values') of its nested tuple. #73878 (Yakov Olkhovskiy).
  • Ignore window functions during GROUP BY ALL clause resolution. Fix #73501. #73916 (Dmitry Novik).
  • Propogate Native format settings properly for client-server communication. #73924 (Pavel Kruglov).
  • Fix implicit privileges (worked as wildcard before). #73932 (Azat Khuzhin).
  • Fix high memory usage during nested Maps creation. #73982 (Pavel Kruglov).
  • Fix parsing nested JSON with empty keys. #73993 (Pavel Kruglov).
  • Fix: alias can be not added to the projection if it is referenced by another alias and selected in inverse order. #74033 (Yakov Olkhovskiy).
  • A disk using the plain_rewritable metadata can be shared among multiple server instances. It is expected for one instance to read a metadata object while another modifies it. Object not found errors are ignored during plain_rewritable initialization with Azure storage, similar to the behavior implemented for S3. #74059 (Julia Kartseva).
  • Fix behaviour of any and anyLast with enum types and empty table. #74061 (Joanna Hulboj).
  • Fixes case when the user specifies keyword arguments in the kafka table engine. #74064 (Yarik Briukhovetskyi).
  • Fix altering Storage S3Queue settings with "s3queue_" prefix to without and vice versa. #74075 (Kseniia Sumarokova).
  • Add a setting allow_push_predicate_ast_for_distributed_subqueries. This adds AST-based predicate push-down for distributed queries with the analyzer. This is a temporary solution that we use until distributed queries with query plan serialization are supported. Closes #66878 #69472 #65638 #68030 #73718. #74085 (Nikolai Kochetov).
  • Fixes issue when after #73095 port can be present in the forwarded_for field, which leads to inability to resolve host name with port included. #74116 (Yakov Olkhovskiy).
  • Fixed incorrect formatting of ALTER TABLE (DROP STATISTICS ...) (DROP STATISTICS ...). #74126 (Han Fei).
  • Fix for issue #66112. #74128 (Anton Ivashkin).
  • It is no longer possible to use Loop as a table engine in CREATE TABLE. This combination was previously causing segfaults. #74137 (Yarik Briukhovetskyi).
  • Fix security issue to prevent SQL injection in postgresql and sqlite table functions. #74144 (Pablo Marcos).
  • Fix crash when reading a subcolumn from the compressed Memory engine table. Fixes #74009. #74161 (Nikita Taranov).
  • Fixed an infinite loop occurring with queries to the system.detached_tables. #74190 (Konstantin Morozov).
  • Fix logical error in s3queue during setting file as failed. #74216 (Kseniia Sumarokova).
  • Check for not supported types for some storages. #74218 (Pavel Kruglov).
  • Fix crash with query INSERT INTO SELECT over PostgreSQL interface on macOS (issue #72938). #74231 (Artem Yurov).
  • Fix native copy settings (allow_s3_native_copy/allow_azure_native_copy) for RESTORE from base backup. #74286 (Azat Khuzhin).
  • Fixed the issue when the number of detached tables in the database is a multiple of max_block_size. #74289 (Konstantin Morozov).
  • Fix copying via ObjectStorage (i.e. S3) when source and destination credentials differs. #74331 (Azat Khuzhin).
  • Fixed uninitialized max_log_ptr in the replicated database. #74336 (Konstantin Morozov).
  • Fix detection of "use the Rewrite method in the JSON API" for native copy on GCS. #74338 (Azat Khuzhin).
  • Fix crash when inserting interval (issue #74299). #74478 (NamNguyenHoai).
  • Fix incorrect projection analysis when count(nullable) is used in aggregate projections. This fixes #74495 . This PR also adds some logs around projection analysis to clarify why a projection is used or why not. #74498 (Amos Bird).
  • Fix incorrect calculation of BackgroundMergesAndMutationsPoolSize (it was x2 from real value). #74509 (alesapin).
  • Fix the bug of leaking keeper watches when enable Cluster Discovery. #74521 (RinChanNOW).
  • Fix formatting constant JSON literals. Previously it could lead to syntax errors during sending the query to another server. #74533 (Pavel Kruglov).
  • Fix mem alignment issue reported by UBSan #74512. #74534 (Arthur Passos).
  • Fix KeeperMap concurrent cleanup during table creation. #74568 (Antonio Andelic).
  • Do not remove unused projection columns in subqueries in the presence of EXCEPT or INTERSECT to preserve the correct query result. Fixes #73930. Fixes #66465. #74577 (Dmitry Novik).
  • Fix broken create query when using constant partition expressions with implicit projections enabled. This fixes #74596 . #74634 (Amos Bird).
  • Fixed INSERT SELECT queries between tables with Tuple columns and enabled sparse serialization. #74698 (Anton Popov).
  • Function right works incorrectly for const negative offset. #74701 (Daniil Ivanik).
  • Fix insertion of gzip-ed data sometimes fails due to flawed decompression on client side. #74707 (siyuan).
  • Avoid leaving connection in broken state after INSERT finishes with exception. #74740 (Azat Khuzhin).
  • Avoid reusing connections that had been left in the intermediate state. #74749 (Azat Khuzhin).
  • Partial revokes with wildcard grants could remove more privileges than expected. Closes #74263. #74751 (pufit).
  • Fix crash during JSON type declaration parsing when type name is not uppercase. #74784 (Pavel Kruglov).
  • Keeper fix: fix reading log entries from disk. #74785 (Antonio Andelic).
  • Fixed checking grants for SYSTEM REFRESH/START/STOP VIEW, now it's not required to have this grant on *.* to execute a query for a specific view, only grant for this view are required. #74789 (Alexander Tokmakov).
  • The hasColumnInTable function doesn't account for alias columns. Fix it to also work for alias columns. #74841 (Bharat Nallan).
  • Keeper: fix logical_error when the connection had been terminated before establishing. #74844 (Michael Kolupaev).
  • Fix a behavior when the server couldn't startup when there's a table using AzureBlobStorage. Tables are loaded without any requests to Azure. #74880 (Alexey Katsman).
  • Fix missing used_privileges and missing_privileges fields in query_log for BACKUP and RESTORE operations. #74887 (Alexey Katsman).
  • Fix FILE_DOESNT_EXIST error occurring during data parts merge for a table with an empty column in Azure Blob Storage. #74892 (Julia Kartseva).
  • Fix projection column name when joining temporary tables, close #68872. #74897 (Vladimir Cherkasov).
  • HDFS refresh krb ticket if sasl error during hdfs select request. #74930 (inv2004).
  • Fix queries to Replicated database in startup_scripts. #74942 (Azat Khuzhin).
  • Fix issues with expressions type aliased in the JOIN ON clause when a null-safe comparison is used. #74970 (Vladimir Cherkasov).
  • Revert part's state from deleting back to outdated when remove operation has failed. #74985 (Sema Checherinda).
  • In previous versions, when there was a scalar subquery, we started writing the progress (accumulated from processing the subquery) during the initialization of the data format, which was before HTTP headers were written. This led to the loss of HTTP headers, such as X-ClickHouse-QueryId and X-ClickHouse-Format, as well as Content-Type. #74991 (Alexey Milovidov).
  • Fix CREATE TABLE AS... queries for database_replicated_allow_replicated_engine_arguments=0. #75000 (Bharat Nallan).
  • Fix leaving connection in a bad state in client after INSERT exceptions. #75030 (Azat Khuzhin).
  • Fix crash due to uncaught exception in PSQL replication. #75062 (Azat Khuzhin).
  • Sasl can fail any rpc call, the fix helps to repeat the call in case if krb5 ticker is expired. #75063 (inv2004).
  • Fixed usage of indexes (primary and secondary) for Array, Map and Nullable(..) columns with enabled setting optimize_function_to_subcolumns. Previously, indexes for these columns could have been ignored. #75081 (Anton Popov).
  • Disable flatten_nested when creating materialized views with inner tables since it will not be possible to use such flattened columns. #75085 (Christoph Wurm).
  • Fix for some of IPv6 addresses (such as ::ffff:1.1.1.1) in forwarded_for field is wrongly interpreted resulting in client disconnect with exception. #75133 (Yakov Olkhovskiy).
  • Fix nullsafe JOIN handling for LowCardinality nullable data type. Previously JOIN ON with nullsafe comparison, such as IS NOT DISTINCT FROM, <=> , a IS NULL AND b IS NULL OR a == b didn't work correctly with LowCardinality columns. #75143 (Vladimir Cherkasov).
  • Fix queries with unused interpolation with the new analyzer. #75173 (János Benjamin Antal).
  • Fix the crash bug of CTE with Insert. #75188 (Shichao Jin).
  • Keeper fix: avoid writing to broken changelogs when rolling back logs. #75197 (Antonio Andelic).
  • Use BFloat16 as a supertype where appropriate. This closes: #74404. #75236 (Nikita Mikhaylov).
  • Fix unexpected defaults in join result with any_join_distinct_right_table_keys and OR in JOIN ON. #75262 (Vladimir Cherkasov).
  • Mask azureblobstorage table engine credentials. #75319 (Garrett Thomas).
  • Fixed behavior when ClickHouse may erroneously do a filter pushdown to an external database like PostgreSQL, MySQL, or SQLite. This closes: #71423. #75320 (Nikita Mikhaylov).
  • Fix crash in protobuf schema cache that can happen during output in Protobuf format and parallel query SYSTEM DROP FORMAT SCHEMA CACHE. #75357 (Pavel Kruglov).
  • Fix a possible logical error or uninitialized memory issue when a filter from HAVING is pushed down with parallel replicas. #75363 (Vladimir Cherkasov).
  • Hide sensitive info for icebergS3, icebergAzure table functions and table engines. #75378 (Kseniia Sumarokova).
  • Function TRIM with computed empty trim characters are now correctly handled. Example: SELECT TRIM(LEADING concat('') FROM 'foo') (Issue #69922). #75399 (Manish Gill).
  • Fix data race in IOutputFormat. #75448 (Pavel Kruglov).
  • Fix possible error Elements ... and ... of Nested data structure ... (Array columns) have different array sizes when JSON subcolumns with Array type are used in JOIN over distributed tables. #75512 (Pavel Kruglov).
  • Fix invalid result buffer size calculation. Closes #70031. #75548 (Konstantin Bogdanov).
  • Fix interaction between allow_feature_tier and compatibility mergetree setting. #75635 (Raúl Marín).
  • Fix incorrect processed_rows value in system.s3queue_log in case file was retried. #75666 (Kseniia Sumarokova).
  • Respect materialized_views_ignore_errors when a materialized view writes to a URL engine and there is a connectivity issue. #75679 (Christoph Wurm).
  • Fixed rare crashes while reading from MergeTree table after multiple asynchronous RENAME queries (with alter_sync = 0) between columns with different types. #75693 (Anton Popov).
  • Fix Block structure mismatch in QueryPipeline stream error for some queries with UNION ALL. #75715 (Nikolai Kochetov).
  • Rebuild projection on alter modify of its PK column. Previously it could lead to CANNOT_READ_ALL_DATA errors during selects after alter modify of the column used in projection PK. #75720 (Pavel Kruglov).
  • Fix incorrect result of ARRAY JOIN for scalar subqueries (with analyzer). #75732 (Nikolai Kochetov).
  • Fixed null pointer dereference in DistinctSortedStreamTransform. #75734 (Nikita Taranov).
  • Fix allow_suspicious_ttl_expressions behaviour. #75771 (Aleksei Filatov).
  • Fix uninitialized memory read in function translate. This closes #75592. #75794 (Alexey Milovidov).
  • Propagate format settings to JSON as string formatting in Native format. #75832 (Pavel Kruglov).
  • Recorded the default enablement of parallel hash as join algorithm in v24.12 in the settings change history. This means that ClickHouse will continue to join using non-parallel hash if an older compatibility level than v24.12 is configured. #75870 (Robert Schulze).
  • Fixed a bug that tables with implicitly added min-max indices could not be copied into a new table (issue #75677). #75877 (Smita Kulkarni).
  • clickhouse-library-bridge allows opening arbitrary libraries from the filesystem, which makes it safe to run only inside an isolated environment. To prevent a vulnerability when it is run near the clickhouse-server, we will limit the paths of libraries to a location, provided in the configuration. This vulnerability was found with the ClickHouse Bug Bounty Program by Arseniy Dugin. #75954 (Alexey Milovidov).
  • We happened to use JSON serialization for some metadata, which was a mistake, because JSON does not support binary data inside string literals, including zero bytes. SQL queries can contain binary data and invalid UTF-8, so we have to support this in our metadata files as well. At the same time, ClickHouse's JSONEachRow and similar formats work around that by deviating from the JSON standard in favor of a perfect roundtrip for the binary data. See the motivation here: https://github.com/ClickHouse/ClickHouse/pull/73668#issuecomment-2560501790. The solution is to make Poco::JSON library consistent with the JSON format serialization in ClickHouse. This closes #73668. #75963 (Alexey Milovidov).
  • Fix Part <...> does not contain in snapshot of previous virtual parts. (PART_IS_TEMPORARILY_LOCKED) during DETACH PART. #76039 (Aleksei Filatov).
  • Fix check for commit limits in storage S3Queue. #76104 (Kseniia Sumarokova).
  • Fix attaching MergeTree tables with auto indexes (add_minmax_index_for_numeric_columns/add_minmax_index_for_string_columns). #76139 (Azat Khuzhin).
  • Fixed issue of stack traces from parent threads of a job (enable_job_stack_trace setting) are not printed out. Fixed issue enable_job_stack_trace setting is not properly propagated to the threads resulting stack trace content not always respects this setting. #76191 (Yakov Olkhovskiy).
  • Fix reinterpretAs with FixedString on big-endian architecture. #76253 (Azat Khuzhin).
  • Fix all sort of bugs due to race between UUID and table names (for instance it will fix the race between RENAME and RESTART REPLICA, in case of concurrent RENAME with SYSTEM RESTART REPLICA you may get end up restarting wrong replica, or/and leaving one of the tables in a Table X is being restarted state). #76308 (Azat Khuzhin).
  • Removed allocation from the signal handler. #76446 (Nikita Taranov).
  • Fix dynamic filesystem cache resize handling unexpected errors during eviction. #76466 (Kseniia Sumarokova).
  • Fixed used_flag initialization in parallel hash. It might cause a server crash. #76580 (Nikita Taranov).
  • Fix a logical error when calling defaultProfiles() function inside a projection. #76627 (pufit).
  • Do not request interactive basic auth in the browser in Web UI. Closes #76319. #76637 (Alexey Milovidov).
  • Fix THERE_IS_NO_COLUMN exception when selecting boolean literal from distributed tables. #76656 (Yakov Olkhovskiy).
  • The subpath inside the table directory is chosen in a more profound way. #76681 (Daniil Ivanik).
  • Fix an error Not found column in block after altering a table with a subcolumn in PK. After https://github.com/ClickHouse/ClickHouse/pull/72644, requires https://github.com/ClickHouse/ClickHouse/pull/74403. #76686 (Nikolai Kochetov).
  • Add performance tests for null short circuits and fix bugs. #76708 (李扬).
  • Flush output write buffers before finalizing them. Fix LOGICAL_ERROR generated during the finalization of some output format, e.g. JSONEachRowWithProgressRowOutputFormat. #76726 (Antonio Andelic).
  • Added support for MongoDB binary UUID (#74452) - Fixed WHERE pushdown to MongoDB when using the table function (#72210) - Changed the MongoDB - ClickHouse type mapping such that MongoDB's binary UUID can only be parsed to ClickHouse's UUID. This should avoid ambiguities and surprises in future. - Fixed OID mapping, preserving backward compatibility. #76762 (Kirill Nikiforov).
  • Fix exception handling in parallel prefixes deserialization of JSON subcolumns. #76809 (Pavel Kruglov).
  • Fix lgamma function behavior for negative integers. #76840 (Ilya Kataev).
  • Fix reverse key analysis for explicitly defined primary keys. Similar to #76654. #76846 (Amos Bird).
  • Fix pretty print of Bool values in JSON format. #76905 (Pavel Kruglov).
  • Fix possible crash because of bad JSON column rollback on error during async inserts. #76908 (Pavel Kruglov).
  • Previously, multi_if may return different types of columns during planning and main execution. This resulted in code producing undefined behavior from the C++ perspective. #76914 (Nikita Taranov).
  • Fixed incorrect serialization of constant nullable keys in MergeTree. This fixes #76939. #76985 (Amos Bird).
  • Fix sorting of BFloat16 values. This closes #75487. This closes #75669. #77000 (Alexey Milovidov).
  • Bug fix JSON with variant subcolumn by adding check to skip ephemeral subcolumns in part consistency check. #72187. #77034 (Smita Kulkarni).
  • Fix crash in template parsing in Values format in case of types mismatch. #77071 (Pavel Kruglov).
  • Don't allow creating EmbeddedRocksDB table with subcolumn in a primary key. Previously, such a table could be created but SELECT queries failed. #77074 (Pavel Kruglov).
  • Fix illegal comparison in distributed queries because pushing down predicates to remote doesn't respect literal types. #77093 (Duc Canh Le).
  • Fix crash during Kafka table creation with exception. #77121 (Pavel Kruglov).
  • Support new JSON and subcolumns in Kafka and RabbitMQ engines. #77122 (Pavel Kruglov).
  • Fix exceptions stack unwinding on MacOS. #77126 (Eduard Karacharov).
  • Fix reading 'null' subcolumn in getSubcolumn function. #77163 (Pavel Kruglov).
  • Fix not working skip indexes with expression with literals in analyzer and remove trivial casts during indexes analysis. #77229 (Pavel Kruglov).
  • Fix bloom filter index with Array and not supported functions. #77271 (Pavel Kruglov).
  • We should only check the restriction on the amount of tables during the initial CREATE query. #77274 (Nikolay Degterinsky).
  • SELECT toBFloat16(-0.0) == toBFloat16(0.0) now correctly returns true (from previously false). This makes the behavior consistent with Float32 and Float64. #77290 (Shankar Iyer).
  • Fix posbile incorrect reference to unintialized key_index variable, which may lead to crash in debug builds (this uninitialized reference won't cause issues in release builds because subsequent code are likely to throw errors.) ### documentation entry for user-facing changes. #77305 (wxybear).
  • Reverted. #77307 (Nikolai Kochetov).
  • Fix name for partition with a Bool value. It was broken in https://github.com/ClickHouse/ClickHouse/pull/74533. #77319 (Pavel Kruglov).
  • Fix comparison between tuples with nullable elements inside and strings. As an example, before the change comparison between a Tuple (1, null) and a String '(1,null)' would result in an error. Another example would be a comparison between a Tuple (1, a), where a is a Nullable column, and a String '(1, 2)'. This change addresses these issues. #77323 (Alexey Katsman).
  • Fix crash in ObjectStorageQueueSource. Was intoduced in https://github.com/ClickHouse/ClickHouse/pull/76358. #77325 (Pavel Kruglov).
  • Fix a bug when close_session query parameter didn't have any effect leading to named sessions being closed only after session_timeout. #77336 (Alexey Katsman).
  • Fix async_insert with input(). #77340 (Azat Khuzhin).
  • Fix: WITH FILL may fail with NOT_FOUND_COLUMN_IN_BLOCK when planer removes sorting column. Similar issue related to inconsistent DAG calculated for INTERPOLATE expression. #77343 (Yakov Olkhovskiy).
  • Reverted. #77390 (Vladimir Cherkasov).
  • Fixed receiving messages from nats server without attached mv. #77392 (Dmitry Novikov).
  • Fix logical error while reading from empty FileLog via merge table function, close #75575. #77441 (Vladimir Cherkasov).
  • Fix several LOGICAL_ERRORs around setting an alias of invalid AST nodes. #77445 (Raúl Marín).
  • In filesystem cache implementation fix error processing during file segment write. #77471 (Kseniia Sumarokova).
  • Make DatabaseIceberg use correct metadata file provided by catalog. Closes #75187. #77486 (Kseniia Sumarokova).
  • Use default format settings in Dynamic serialization from shared variant. #77572 (Pavel Kruglov).
  • Revert 'Avoid toAST() in execution of scalar subqueries'. #77584 (Raúl Marín).
  • Fix checking if the table data path exists on the local disk. #77608 (Tuan Pham Anh).
  • The query cache now assumes that UDFs are non-deterministic. Accordingly, results of queries with UDFs are no longer cached. Previously, users were able to define non-deterministic UDFs whose result would erronously be cached (issue #77553). #77633 (Jimmy Aguilar Mena).
  • Fix sending constant values to remote for some types. #77634 (Pavel Kruglov).
  • Fix system.filesystem_cache_log working only under setting enable_filesystem_cache_log. #77650 (Kseniia Sumarokova).
  • Fix a logical error when calling defaultRoles() function inside a projection. Follow-up for #76627. #77667 (pufit).
  • Fix crash because of expired context in StorageS3(Azure)Queue. #77720 (Kseniia Sumarokova).
  • Second arguments of type Nullable for function arrayResize are now disallowed. Previously, anything from errors to wrong results could happen with Nullable as second argument. (issue #48398). #77724 (Manish Gill).
  • Hide credentials in RabbitMQ, Nats, Redis, AzureQueue table engines. #77755 (Kseniia Sumarokova).
  • Fix undefined behaviour on NaN comparison in ArgMin/ArgMax. #77756 (Raúl Marín).
  • Regularly check if merges and mutations were cancelled even in case when the operation doesn't produce any blocks to write. #77766 (János Benjamin Antal).
  • Reverted. #77843 (Vladimir Cherkasov).
  • Fix possible crash when NOT_FOUND_COLUMN_IN_BLOCK error occurs. #77854 (Vladimir Cherkasov).
  • Fix crash that happens in the StorageSystemObjectStorageQueueSettings while filling data. #77878 (Bharat Nallan).
  • Disable fuzzy search for history in SSH server (since it requires skim). #78002 (Azat Khuzhin).
  • Fixes a bug that a vector search query on a non-indexed column was returning incorrect results if there was another vector column in the table with a defined vector similarity index. (Issue #77978). #78069 (Shankar Iyer).
  • Fix The requested output format {} is binary... Do you want to output it anyway? [y/N] prompt. #78095 (Azat Khuzhin).
  • Fix of a bug in case of toStartOfInterval with zero origin argument. #78096 (Yarik Briukhovetskyi).
  • Disallow specifying an empty session_id query parameter for HTTP interface. #78098 (Alexey Katsman).
  • Fix metadata override in Database Replicated which could have happened due to a RENAME query executed right after an ALTER query. #78107 (Nikolay Degterinsky).
  • Fix crash in NATS engine. #78108 (Dmitry Novikov).
  • Do not try to create a history_file in an embedded client for SSH. #78112 (Azat Khuzhin).
  • Fix system.detached_tables displaying incorrect information after RENAME DATABASE or DROP TABLE queries. #78126 (Nikolay Degterinsky).
  • Fix for checks for too many tables with Database Replicated after https://github.com/ClickHouse/ClickHouse/pull/77274. Also, perform the check before creating the storage to avoid creating unaccounted nodes in ZooKeeper in the case of RMT or KeeperMap. #78127 (Nikolay Degterinsky).
  • Fix possible crash due to concurrent S3Queue metadata initialization. #78131 (Azat Khuzhin).
  • groupArray* functions now produce BAD_ARGUMENTS error for Int-typed 0 value of max_size argument, like it's already done for UInt one, instead of trying to execute with it. #78140 (Eduard Karacharov).
  • Prevent crash on recoverLostReplica if the local table is removed before it's detached. #78173 (Raúl Marín).
  • Fix "alterable" column in system.s3_queue_settings returning always false. #78187 (Kseniia Sumarokova).
  • Mask azure access signature to be not visible to user or in logs. #78189 (Kseniia Sumarokova).
  • Fix prefetch of substreams with prefixes in Wide parts. #78205 (Pavel Kruglov).
  • Fixed crashes / incorrect result for mapFromArrays in case of LowCardinality(Nullable) type of key array. #78240 (Eduard Karacharov).
  • Fix delta-kernel auth options. #78255 (Kseniia Sumarokova).
  • Not schedule RefreshMV task if a replica's disable_insertion_and_mutation is true. A task is some insertion, it will failed if disable_insertion_and_mutation is true. #78277 (Xu Jia).
  • Validate access to underlying tables for the Merge engine. #78339 (Pervakov Grigorii).
  • FINAL modifier can be lost for Distributed engine table. #78428 (Yakov Olkhovskiy).
  • Bitmapmin returns uint32_max when the bitmap is empty(uint64_max when input type >= 8bits), which matches the behavior of empty roaring_bitmap's minimum(). #78444 (wxybear).
  • Revert "Apply preserve_most attribute at some places in code" since it may lead to crashes. #78449 (Azat Khuzhin).
  • Use insertion columns for INFILE schema inference. #78490 (Pervakov Grigorii).
  • Disable parallelize query processing right after reading FROM when distributed_aggregation_memory_efficient enabled, it may lead to logical error. Closes #76934. #78500 (flynn).
  • Set at least one stream for reading in case there are zero planned streams after applying max_streams_to_max_threads_ratio setting. #78505 (Eduard Karacharov).
  • In storage S3Queue fix logical error "Cannot unregister: table uuid is not registered". Closes #78285. #78541 (Kseniia Sumarokova).
  • ClickHouse is now able to figure out its cgroup v2 on systems with both cgroups v1 and v2 enabled. #78566 (Grigory Korolev).
  • ObjectStorage cluster table functions failed when used with table level-settings. #78587 (Daniil Ivanik).
  • Better checks for transactions are not supported by ReplicatedMergeTree on INSERTs. #78633 (Azat Khuzhin).
  • Apply query settings during attachment. #78637 (Raúl Marín).
  • Fixes a crash when an invalid path is specified in iceberg_metadata_file_path. #78688 (alesapin).
  • In DeltaLake table engine with delta-kernel implementation fix case when read schema is different from table schema and there are partition columns at the same time leading to not found column error. #78690 (Kseniia Sumarokova).
  • This update corrects a bug where a new named session would inadvertently close at the scheduled time of a previous session if both sessions shared the same name and the new one was created before the old one's timeout expired. #78698 (Alexey Katsman).
  • Don't block table shutdown while running CHECK TABLE. #78782 (Raúl Marín).
  • Keeper fix: fix ephemeral count in all cases. #78799 (Antonio Andelic).
  • Fix bad cast in StorageDistributed when using table functions other than view(). Closes #78464. #78828 (Konstantin Bogdanov).
  • Fix formatting for tupleElement(*, 1). Closes #78639. #78832 (Konstantin Bogdanov).
  • Dictionaries of type ssd_cache now reject zero or negative block_size and write_buffer_size parameters (issue #78314). #78854 (Elmi Ahmadov).
  • Fix crash in REFRESHABLE MV in case of ALTER after incorrect shutdown. #78858 (Azat Khuzhin).
  • Fix parsing of bad DateTime values in CSV format. #78919 (Pavel Kruglov).

Build/Testing/Packaging Improvement

  • The internal dependency LLVM is bumped from 16 to 18. #66053 (Nikita Mikhaylov).
  • Restore deleted nats integration tests and fix errors. - fixed some race conditions in nats engine - fixed data loss when streaming data to nats in case of connection loss - fixed freeze of receiving the last chunk of data when streaming from nats ended - nats_max_reconnect is deprecated and has no effect, reconnect is performed permanently with nats_reconnect_wait timeout. #69772 (Dmitry Novikov).
  • Fix the issue that asm files of contrib openssl cannot be generated. #72622 (RinChanNOW).
  • Fix stability for test 03210_variant_with_aggregate_function_type. #74012 (Anton Ivashkin).
  • Support build HDFS on both ARM and Intel Mac. #74244 (Yan Xin).
  • The universal installation script will propose installation even on macOS. #74339 (Alexey Milovidov).
  • Fix build when kerberos is not enabled. #74771 (flynn).
  • Update to embedded LLVM 19. #75148 (Konstantin Bogdanov).
  • Potentially breaking: Improvement to set even more restrictive defaults. The current defaults are already secure. The user has to specify an option to publish ports explicitly. But when the default user doesn’t have a password set by CLICKHOUSE_PASSWORD and/or a username changed by CLICKHOUSE_USER environment variables, it should be available only from the local system as an additional level of protection. #75259 (Mikhail f. Shiryaev).
  • Integration tests have a 1-hour timeout for single batch of parallel tests running. When this timeout is reached pytest is killed without some logs. Internal pytest timeout is set to 55 minutes to print results from a session and not trigger external timeout signal. Closes #75532. #75533 (Ilya Yatsishin).
  • Make all clickhouse-server related actions a function, and execute them only when launching the default binary in entrypoint.sh. A long-postponed improvement was suggested in #50724. Added switch --users to clickhouse-extract-from-config to get values from the users.xml. #75643 (Mikhail f. Shiryaev).
  • For stress tests if server did not exit while we collected stacktraces via gdb additional wait time is added to make Possible deadlock on shutdown (see gdb.log) detection less noisy. It will only add delay for cases when test did not finish successfully. #75668 (Ilya Yatsishin).
  • Restore deleted nats integration tests and fix errors. - fixed some race conditions in nats engine - fixed data loss when streaming data to nats in case of connection loss - fixed freeze of receiving the last chunk of data when streaming from nats ended - nats_max_reconnect is deprecated and has no effect, reconnect is performed permanently with nats_reconnect_wait timeout. #75850 (Dmitry Novikov).
  • Enable ICU and GRPC when cross-compiling for Darwin. #75922 (Raúl Marín).
  • Fixing splitting test's output because of sleep during the process group killing. #76090 (Mikhail f. Shiryaev).
  • Do not collect the docker-compose logs at the end of running since the script is often killed. Instead, collect them in the background. #76140 (Mikhail f. Shiryaev).
  • Split tests for kafka storage into a few files. Fixes #69452. #76208 (Mikhail f. Shiryaev).
  • clickhouse-odbc-bridge and clickhouse-library-bridge are moved to a separate repository, https://github.com/ClickHouse/odbc-bridge/. #76225 (Alexey Milovidov).
  • Remove about 20MB of dead code from the binary. #76226 (Alexey Milovidov).
  • Raise minimum required CMake version to 3.25 due to block() introduction. #76316 (Konstantin Bogdanov).
  • Update fmt to 11.1.3. #76547 (Raúl Marín).
  • Bump lz4 to 1.10.0. #76571 (Konstantin Bogdanov).
  • Bump curl to 8.12.1. #76572 (Konstantin Bogdanov).
  • Bump libcpuid to 0.7.1. #76573 (Konstantin Bogdanov).
  • Use a machine-readable format to parse pytest results. #76910 (Mikhail f. Shiryaev).
  • Fix rust cross-compilation and allow disabling Rust completely. #76921 (Raúl Marín).
  • Require clang 19 to build the project. #76945 (Raúl Marín).
  • The test is executed for 10+ seconds in the serial mode. It's too long for fast tests. #76948 (Mikhail f. Shiryaev).
  • Bump sccache to 0.10.0. #77580 (Konstantin Bogdanov).
  • Respect CPU target features in rust and enable LTO in all crates. #78590 (Raúl Marín).
  • Bump minizip-ng to 4.0.9. #78917 (Konstantin Bogdanov).