ClickHouse/docs/changelogs/v23.8.1.2992-lts.md
2024-05-30 18:10:12 +02:00

108 KiB

sidebar_position sidebar_label
1 2023

2023 Changelog

ClickHouse release v23.8.1.2992-lts (ebc7d9a9f3) FIXME as compared to v23.7.1.2470-stable (a70127baec)

Backward Incompatible Change

  • Deprecate the metadata cache feature. It is experimental and we have never used it. The feature is dangerous: #51182. Remove the system.merge_tree_metadata_cache system table. The metadata cache is still available in this version but will be removed soon. This closes #39197. #51303 (Alexey Milovidov).
  • If a dynamic disk contains a name, it should be specified as disk = disk(name = 'disk_name', ...) in disk function arguments. In previous version it could be specified as disk = disk_<disk_name>(...), which is no longer supported. #52820 (Kseniia Sumarokova).
  • clickhouse-benchmark will establish connections in parallel when invoked with --concurrency more than one. Previously it was unusable if you ran it with 1000 concurrent connections from Europe to the US. Correct calculation of QPS for connections with high latency. Backward incompatible change: the option for JSON output of clickhouse-benchmark is removed. If you've used this option, you can also extract data from the system.query_log in JSON format as a workaround. #53293 (Alexey Milovidov).
  • The microseconds column is removed from the system.text_log, and the milliseconds column is removed from the system.metric_log, because they are redundant in the presence of the event_time_microseconds column. #53601 (Alexey Milovidov).
  • Changed zookeeper paths for storage S3Queue metadata. #54137 (Kseniia Sumarokova).

New Feature

  • Add column ptr to system.trace_log for trace_type = 'MemorySample'. This column contains an address of allocation. Added function flameGraph which can build flamegraph containing allocated and not released memory. Reworking of #38391. #45322 (Nikolai Kochetov).
  • Add setting rewrite_count_distinct_if_with_count_distinct_implementation to rewrite countDistinctIf with count_distinct_implementation. Closes #30642. #46051 (flynn).
  • Add new table engine S3Queue for streaming data import from s3. Closes #37012. #49086 (s-kat).
  • SevenZipArchiveReader - TarArchiveReader - Table Function file('path_to_archive :: filename') - Functional tests for "Table Function file('path_to_archive :: filename')" - Unit tests for TarArchiveReader/SevenZipArchiveReader. #50321 (nikitakeba).
  • Added table function azureBlobStorageCluster table function. The supported set of features is very similar to table function S3Cluster. #50795 (SmitaRKulkarni).
  • Allow using cluster, clusterAllReplicas, remote, remoteRaw and remoteSecure without table name in issue #50808. #50848 (Yangkuan Liu).
  • System table to monitor kafka consumers. #50999 (Ilya Golshtein).
  • Added max_sessions_for_user setting. #51724 (Alexey Gerasimchuck).
  • Now that clickhouse do not have a function to convert UTC timezone timestamp to other timezone timestamp, which is not same as spark, and so we and the functions toUTCTimestamp/fromUTCTimestamp to act same as spark's to_utc_timestamp/from_utc_timestamp. #52117 (KevinyhZou).
  • Add new functions structureToCapnProtoSchema/structureToProtobufSchema that convert ClickHouse table structure to CapnProto/Protobuf format schema. Allow to intput/output data in CapnProto/Protobuf format without external format schema using autogenerated schema from table structure (controled by settings format_capn_proto_use_autogenerated_schema/format_protobuf_use_autogenerated_schema). Allow to export autogenerated schema while input/outoput using setting output_format_schema. #52278 (Kruglov Pavel).
  • A new field "query_cache_usage" in SYSTEM.QUERY_LOG now shows if and how the query cache was used. #52384 (Robert Schulze).
  • Add new function startsWithUTF8 and endsWithUTF8. #52555 (李扬).
  • Allow variable number of columns in TSV/CuatomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns. Add settings input_format_tsv_allow_variable_number_of_columns, input_format_custom_allow_variable_number_of_columns, input_format_json_compact_allow_variable_number_of_columns. #52692 (Kruglov Pavel).
  • Added SYSTEM STOP/START PULLING REPLICATION LOG queries (for testing ReplicatedMergeTree). #52881 (Alexander Tokmakov).
  • Allow to execute constant non-deterministic functions in mutations on initiator. #53129 (Anton Popov).
  • Add input format One that doesn't read any data and always returns single row with column dummy with type UInt8 and value 0 like system.one. It can be used together with _file/_path virtual columns to list files in file/s3/url/hdfs/etc table functions without reading any data. #53209 (Kruglov Pavel).
  • Add tupleConcat function. Closes #52759. #53239 (Nikolay Degterinsky).
  • Support TRUNCATE DATABASE operation. #53261 (Bharat Nallan).
  • Add max_threads_for_indexes setting to limit number of threads used for primary key processing. #53313 (Joris Giovannangeli).
  • Add experimental support for HNSW as approximate neighbor search method. #53447 (Davit Vardanyan).
  • Re-add SipHash keyed functions. #53525 (Salvatore Mesoraca).
  • (#52755 , #52895) Added functions arrayRotateLeft, arrayRotateRight, arrayShiftLeft, arrayShiftRight. #53557 (Mikhail Koviazin).
  • Add column name to system.clusters as an alias to cluster. #53605 (irenjj).
  • The advanced dashboard now allows mass editing (save/load). #53608 (Alexey Milovidov).
  • Add support for plural units. #53641 (irenjj).
  • Support function isNotDistinctFrom in join on section for null-safe comparison, ref #53061. #53755 (vdimir).
  • Added the "hide_in_preprocessed" attribute to ClickHouse's server configuration XML dialect. This is a mechanism to hide certain settings from appearing in preprocessed server configuration files. Useful e.g. for passwords or private keys that should not appear verbatim in files. #53818 (Roman Vasin).
  • Added server setting validate_tcp_client_information determines whether validation of client information enabled when query packet is received. #53907 (Alexey Gerasimchuck).

Performance Improvement

  • Enable JIT compilation for AArch64, PowerPC, SystemZ, RISCV. #38217 (Maksim Kita).
  • This patch will provide a method to deal with all the hashsets in parallel before merge. #50748 (Jiebin Sun).
  • Optimize aggregation performance of nullable string key when using aggregationmethodserialized. #51399 (LiuNeng).
  • The performance experiments of SSB on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of 8.5% to the geomean QPS when the experimental analyzer is enabled. The details are shown below: image. #52091 (Zhiguo Zhou).
  • Parquet filter pushdown. I.e. when reading Parquet files, row groups (chunks of the file) are skipped based on the WHERE condition and the min/max values in each column. In particular, if the file is roughly sorted by some column, queries that filter by a short range of that column will be much faster. #52951 (Michael Kolupaev).
  • Optimize the merge if all hashSets are singleLevel in UniqExactSet. #52973 (Jiebin Sun).
  • StorageJoin: do not create clone hash join with all columns. #53046 (Duc Canh Le).
  • Optimize reading small row groups by batching them together in Parquet. Closes #53069. #53281 (Kruglov Pavel).
  • Implement native orc input format without arrow to improve performance. #53324 (李扬).
  • The dashboard will tell the server to compress the data, which is useful for large time frames over slow internet connections. For example, one chart with 86400 points can be 1.5 MB uncompressed and 60 KB compressed with br. #53569 (Alexey Milovidov).
  • Optimize count from files in most input formats. Closes #44334. #53637 (Kruglov Pavel).
  • Better utilization of thread pool for BACKUPs&RESTOREs. #53649 (Nikita Mikhaylov).
  • Remove useless slow on client performance check. #53695 (Raúl Marín).

Improvement

  • Bloom filter indices are pruned so that they correlate with cardinality of the data set they are tracking. #35102 (Anton Kozlov).
  • Add stderr_reaction configuration/setting to control the reaction (none, log or throw) when external command stderr has data. This helps make debugging external command easier. #43210 (Amos Bird).
  • Https://github.com/clickhouse/clickhouse/issues/48720. @kgoralski helped with some thought about system.merges part. :d. #48990 (Jianfei Hu).
  • If a dictionary is created with a complex key, automatically choose the "complex key" layout variant. #49587 (xiebin).
  • Add setting use_concurrency_control for better testing of the new concurrency control feature. #49618 (Alexey Milovidov).
  • Added suggestions for mistyped names for db and tables with different scenarios commented. #49801 (Yarik Briukhovetskyi).
  • While read small files from hdfs by gluten, we found that it will cost more times when compare to directly query by spark. #50063 (KevinyhZou).
  • Too many worthless error logs after session expiration. #50171 (helifu).
  • Introduce fallback ZooKeeper sessions which are time-bound. Fixed index column in system.zookeeper_connection for DNS addresses. #50424 (Anton Kozlov).
  • Add ability to log when max_partitions_per_insert_block is reached ... #50948 (Sean Haynes).
  • Added a bunch of custom commands (mostly to make ClickHouse debugging easier). #51117 (pufit).
  • Updated check for connection_string as connection string with sas does not always begin with DefaultEndPoint and updated connection url to include sas token after adding container to url. #51141 (SmitaRKulkarni).
  • Fix description for filtering sets in full_sorting_merge join. #51329 (ttanay).
  • The sizes of the (index) uncompressed/mark, mmap and query caches can now be configured dynamically at runtime. #51446 (Robert Schulze).
  • Fixed memory consumption in Aggregator when max_block_size is huge. #51566 (Nikita Taranov).
  • Add SYSTEM SYNC FILESYSTEM CACHE command. It will compare in-memory state of filesystem cache with what it has on disk and fix in-memory state if needed. #51622 (Kseniia Sumarokova).
  • Attempt to create a generic proxy resolver for CH while keeping backwards compatibility with existing S3 storage conf proxy resolver. #51749 (Arthur Passos).
  • Support reading tuple subcolumns from file/s3/hdfs/url/azureBlobStorage table functions. #51806 (Kruglov Pavel).
  • Function arrayIntersect now returns the values sorted like the first argument. Closes #27622. #51850 (Yarik Briukhovetskyi).
  • Add new queries, which allow to create/drop of access entities in specified access storage or move access entities from one access storage to another. #51912 (pufit).
  • ALTER TABLE FREEZE are not replicated in Replicated engine. #52064 (Mikhail Kot).
  • Added possibility to flush logs to the disk on crash - Added logs buffer configuration. #52174 (Alexey Gerasimchuck).
  • Fix S3 table function does not work for pre-signed URL. close #50846. #52310 (Jensen).
  • System.events and system.metrics tables add column name as an alias to event and metric. close #51257. #52315 (Jensen).
  • Added support of syntax CREATE UNIQUE INDEX in parser for better SQL compatibility. UNIQUE index is not supported. Set create_index_ignore_unique=1 to ignore UNIQUE keyword in queries. #52320 (Ilya Yatsishin).
  • Add support of predefined macro ({database} and {table}) in some kafka engine settings: topic, consumer, client_id, etc. #52386 (Yury Bogomolov).
  • Disable updating fs cache during backup/restore. Filesystem cache must not be updated during backup/restore, it seems it just slows down the process without any profit (because the BACKUP command can read a lot of data and it's no use to put all the data to the filesystem cache and immediately evict it). #52402 (Vitaly Baranov).
  • Updated parameterized view implementation to create new StorageView with substituted parameters for every SELECT query of a parameterized view. #52569 (SmitaRKulkarni).
  • The configuration of S3 endpoint allow using it from the root, and append '/' automatically if needed. #47809. #52600 (xiaolei565).
  • Added support for adding and subtracting arrays: [5,2] + [1,7]. Division and multiplication were not implemented due to confusion between pointwise multiplication and the scalar product of arguments. Closes #49939. #52625 (Yarik Briukhovetskyi).
  • Add support for string literals as table name. Closes #52178. #52635 (hendrik-m).
  • For clickhouse-local allow positional options and populate global UDF settings (user_scripts_path and user_defined_executable_functions_config). #52643 (Yakov Olkhovskiy).
  • System.asynchronous_metrics now includes metrics "querycacheentries" and "querycachebytes" to inspect the query cache. #52650 (Robert Schulze).
  • Added possibility use s3_storage_class parameter in SETTINGS of BACKUP statement for backups to S3. #52658 (Roman Vasin).
  • Improve insert retries on keeper session expiration. #52688 (Raúl Marín).
  • Add utility print-backup-info.py which parses a backup metadata file and prints information about the backup. #52690 (Vitaly Baranov).
  • Closes #49510. Currently we have database and table names case-sensitive, but the tools query information_schema sometimes in lowercase, sometimes in uppercase. For this reason we have information_schema database, containing lowercase tables, such as information_schema.tables and INFORMATION_SCHEMA database, containing uppercase tables, such as INFORMATION_SCHEMA.TABLES. But some tools are querying INFORMATION_SCHEMA.tables and information_schema.TABLES. The proposed solution is to duplicate both lowercase and uppercase tables in lowercase and uppercase information_schema database. #52695 (Yarik Briukhovetskyi).
  • GET_PART and ATTACH_PART are almost identical, so they should use same executor pool. #52716 (Duc Canh Le).
  • QueryCHECK TABLE has better performance and usability (sends progress updates, cancellable). #52745 (vdimir).
  • Add modulo, intDiv, intDivOrZero for tuple. #52758 (Yakov Olkhovskiy).
  • Search for default yaml and yml configs in clickhouse-client after xml. #52767 (Alexey Milovidov).
  • When merging into non-'clickhouse' rooted configuration, configs with different root node name just bypassed without exception. #52770 (Yakov Olkhovskiy).
  • Now it's possible to specify min (memory_profiler_sample_min_allocation_size) and max (memory_profiler_sample_max_allocation_size) size for allocations to be tracked with sampling memory profiler. #52779 (alesapin).
  • Add precise_float_parsing setting to switch float parsing methods (fast/precise). #52791 (Andrey Zvonov).
  • Use the same default paths for clickhouse_keeper (symlink) as for clickhouse_keeper (executable). #52861 (Vitaly Baranov).
  • CVE-2016-2183: disable 3DES. #52893 (Kenji Noguchi).
  • Load filesystem cache metadata on startup in parallel. Configured by load_metadata_threads (default: 1) cache config setting. Related to #52037. #52943 (Kseniia Sumarokova).
  • Improve error message for table function remote. Closes #40220. #52959 (Jiyoung Yoo).
  • Added the possibility to specify custom storage policy in the SETTINGS clause of RESTORE queries. #52970 (Victor Krasnov).
  • Add the ability to throttle the S3 requests on backup operations (BACKUP and RESTORE commands now honor s3_max_[get/put]_[rps/burst]). #52974 (Daniel Pozo Escalona).
  • Add settings to ignore ON CLUSTER clause in queries for management of replicated user-defined functions or access control entities with replicated storage. #52975 (Aleksei Filatov).
  • Enable parallel reading from replicas over distributed table. Related to #49708. #53005 (Igor Nikonov).
  • EXPLAIN actions for JOIN step. #53006 (Maksim Kita).
  • Make hasTokenOrNull and hasTokenCaseInsensitiveOrNull return null for empty needles. #53059 (ltrk2).
  • Allow to restrict allowed paths for filesystem caches. Mainly useful for dynamic disks. If in server config filesystem_caches_path is specified, all filesystem caches' paths will be restricted to this directory. E.g. if the path in cache config is relative - it will be put in filesystem_caches_path; if path in cache config is absolute, it will be required to lie inside filesystem_caches_path. If filesystem_caches_path is not specified in config, then behaviour will be the same as in earlier versions. #53124 (Kseniia Sumarokova).
  • Added a bunch of custom commands (mostly to make ClickHouse debugging easier). #53127 (pufit).
  • Add diagnostic info about file name during schema inference - it helps when you process multiple files with globs. #53135 (Alexey Milovidov).
  • Client will load suggestions using the main connection if the second connection is not allowed to create a session. #53177 (Alexey Gerasimchuck).
  • Add EXCEPT clause to SYSTEM STOP/START LISTEN QUERIES [ALL/DEFAULT/CUSTOM] query, for example SYSTEM STOP LISTEN QUERIES ALL EXCEPT TCP, HTTP. #53280 (Nikolay Degterinsky).
  • Change the default of max_concurrent_queries from 100 to 1000. It's ok to have many concurrent queries if they are not heavy, and mostly waiting for the network. Note: don't confuse concurrent queries and QPS: for example, ClickHouse server can do tens of thousands of QPS with less than 100 concurrent queries. #53285 (Alexey Milovidov).
  • Add ability to override credentials for accessing base backup in S3 (since tokens may be expired). #53326 (Azat Khuzhin).
  • Improve move_primary_key_columns_to_end_of_prewhere. #53337 (Han Fei).
  • Limit number of concurrent background partition optimize merges. #53405 (Duc Canh Le).
  • Added a setting allow_moving_table_directory_to_trash that allows to ignore Directory for table data already exists error when replicating/recovering a Replicated database. #53425 (Alexander Tokmakov).
  • Server settings asynchronous_metrics_update_period_s and asynchronous_heavy_metrics_update_period_s configured to 0 now fail gracefully instead of crash the server. #53428 (Robert Schulze).
  • Previously the caller could register the same watch callback multiple times. In that case each entry was consuming memory and the same callback was called multiple times which didn't make much sense. In order to avoid this the caller could have some logic to not add the same watch multiple times. With this change this deduplication is done internally if the watch callback is passed via shared_ptr. #53452 (Alexander Gololobov).
  • The ClickHouse server now respects memory limits changed via cgroups when reloading its configuration. #53455 (Robert Schulze).
  • Add ability to turn off flush of Distributed tables on DETACH/DROP/server shutdown (flush_on_detach setting for Distributed). #53501 (Azat Khuzhin).
  • Domainrfc support ipv6(ip literal within square brackets). #53506 (Chen768959).
  • Use filter by file/path before reading in url/file/hdfs table functins. #53529 (Kruglov Pavel).
  • Use longer timeout for S3 CopyObject requests. #53533 (Michael Kolupaev).
  • Added server setting aggregate_function_group_array_max_element_size. This setting is used to limit array size for groupArray function at serialization. The default value is 16777215. #53550 (Nikolai Kochetov).
  • SCHEMA() was added as alias for DATABASE() to improve MySQL compatibility. #53587 (Daniël van Eeden).
  • Add asynchronous metrics about tables in the system database. For example, TotalBytesOfMergeTreeTablesSystem. This closes #53603. #53604 (Alexey Milovidov).
  • SQL editor in the Play UI and Dashboard will not use Grammarly. #53614 (Alexey Milovidov).
  • The advanced dashboard now has an option to maximize charts and move them around. #53622 (Alexey Milovidov).
  • As expert-level settings, it is now possible to 1. configure the size_ratio (i.e. the relative size of the protected queue) of the [index] mark/uncompressed caches, 2. configure the cache policy of the index mark and index uncompressed caches. #53657 (Robert Schulze).
  • More careful thread management will improve the speed of the S3 table function over a large number of files by more than ~25%. #53668 (pufit).
  • Upgrade snappy to 1.1.10, clickhouse may benefit from it. #53672 (李扬).
  • Added client info validation to the query packet in TCPHandler. #53673 (Alexey Gerasimchuck).
  • Cache number of rows in files for count in file/s3/url/hdfs/azure functions. The cache can be enabled/disabled by setting use_cache_for_count_from_files (enabled by default). Continuation of https://github.com/ClickHouse/ClickHouse/pull/53637. #53692 (Kruglov Pavel).
  • Updated to retry loading part in case of Azure::Core::Http::TransportException (https://github.com/ClickHouse/ClickHouse/issues/39700#issuecomment-1686442785). #53750 (SmitaRKulkarni).
  • Stacktrace for exceptions, Materailized view exceptions are propagated. #53766 (Ilya Golshtein).
  • If no hostname or port were specified, keeper client will try to search for a connection string in the ClickHouse's config.xml. #53769 (pufit).
  • Add profile event PartsLockMicroseconds which shows the amount of microseconds we hold the data parts lock in MergeTree table engine family. #53797 (alesapin).
  • Make reconnect limit in raft limits configurable for keeper. This configuration can help to make keeper to rebuild connection with peers quicker if the current connection is broken. #53817 (Pengyuan Bian).
  • Supported globs in select from file in clickhouse-local. #53863 (Alexey Gerasimchuck).
  • ...Ignore foreign keys in tables definition to improve compatibility with MySQL, so a user wouldn't need to rewrite his SQL of the foreign key part, ref #53380. #53864 (jsc0218).
  • 'from' is supported as a Expression. #53914 (Chen768959).
  • Changes of the server configuration are now detected with high precision (milliseconds and less). #54065 (Mikhail Koviazin).

Build/Testing/Packaging Improvement

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

  • Fix results of queries utilizing the Annoy index when the part has more than one mark. #51325 (Tian Xinhui).
  • Fix usage of temporary directories during RESTORE. #51493 (Azat Khuzhin).
  • Fixed binary arithmetic for Nullable(IPv4). #51642 (Yakov Olkhovskiy).
  • Support IPv4 and IPv6 as dictionary attributes. #51756 (Yakov Olkhovskiy).
  • Updated checkDataPart to read compress marks as compressed file by checking its extension resolves #51337. #51777 (SmitaRKulkarni).
  • Fix mistakenly comma parsing as part of datetime in CSV datetime best effort parsing. Closes #51059. #51950 (Kruglov Pavel).
  • Fixed exception when executable udf was provided with a parameter. #51961 (Nikita Taranov).
  • Fixed recalculation of skip indexes and projections in ALTER DELETE queries. #52530 (Anton Popov).
  • Fixed the infinite loop in ReadBuffer when the pos overflows the end of the buffer in MaterializedMySQL. #52621 (Val Doroshchuk).
  • Do not try to load suggestions in clickhouse-local when a the dialect is not clickhouse. #52628 (János Benjamin Antal).
  • Remove mutex from CaresPTRResolver and create ares_channel on demand. Trying to fix: https://github.com/ClickHouse/ClickHouse/pull/52327#issuecomment-1643021543. #52634 (Arthur Passos).
  • Fix filtering by virtual columns with OR expression (i.e. by _table for Merge engine). #52653 (Azat Khuzhin).
  • Fix crash in function tuple with one sparse column argument. #52659 (Anton Popov).
  • Fix named collections related statements: if [not] exists, on cluster. Closes #51609. #52687 (Al Korgun).
  • Fix reading of unnecessary column in case of multistage PREWHERE. #52689 (Anton Popov).
  • Fix unexpected sort result on multi columns with nulls first direction. #52761 (ZhiHong Zhang).
  • Keeper fix: fix data race during reconfiguration. #52804 (Antonio Andelic).
  • Fixed sorting of sparse columns in case of ORDER BY ... LIMIT n clause and large values of n. #52827 (Anton Popov).
  • Keeper fix: platforms that used poll() would delay responding to requests until the client sent a heartbeat. #52833 (Andy Fiddaman).
  • Make regexp analyzer recognize named capturing groups. #52840 (Han Fei).
  • Fix possible assert in ~PushingAsyncPipelineExecutor in clickhouse-local. #52862 (Kruglov Pavel).
  • Fix reading of empty Nested(Array(LowCardinality(...))) columns (added by ALTER TABLE ... ADD COLUMN ... query and not materialized in parts) from compact parts of MergeTree tables. #52949 (Anton Popov).
  • Fixed the record inconsistency in session_log between login and logout. #52958 (Alexey Gerasimchuck).
  • Fix password leak in show create mysql table. #52962 (Duc Canh Le).
  • Fix possible crash in full sorting merge join on sparse columns, close #52978. #53000 (vdimir).
  • Fix very rare race condition with empty key prefix directory deletion in fs cache. #53055 (Kseniia Sumarokova).
  • Fixed output_format_parquet_compression_method='zstd' producing invalid Parquet files sometimes. In older versions, use setting output_format_parquet_use_custom_encoder = 0 as a workaround. #53064 (Michael Kolupaev).
  • Fix query_id in part_log with async flush queries. #53103 (Raúl Marín).
  • Fix possible error from filesystem cache "Read unexpected size". #53121 (Kseniia Sumarokova).
  • Disable the new parquet encoder: it has a bug. #53130 (Alexey Milovidov).
  • Not-ready Set is passed as the second argument for function 'in' could happen with limited max_result_rows and result_overflow_mode = 'break'. #53162 (Nikolai Kochetov).
  • Fix character escaping in the PostgreSQL engine (\' -> '', \\ -> \). Closes #49821. #53250 (Nikolay Degterinsky).
  • Fixed the record inconsistency in session_log between login and logout. #53255 (Alexey Gerasimchuck).
  • Fixed the record inconsistency in session_log between login and logout. #53302 (Alexey Gerasimchuck).
  • Fixed adding intervals of a fraction of a second to DateTime producing incorrect result. #53309 (Michael Kolupaev).
  • Fix the "Context has expired" error in dictionaries when using subqueries. #53342 (Alexey Milovidov).
  • Fix incorrect normal projection AST format when single function is used in ORDER BY. This fixes #52607. #53347 (Amos Bird).
  • Forbid use_structure_from_insertion_table_in_table_functions when execute Scalar. Closes #52494. #53348 (flynn).
  • Avoid loading tables from lazy database when not needed Follow up to #43840. #53372 (SmitaRKulkarni).
  • Fixed system.data_skipping_indices columns data_compressed_bytes and data_uncompressed_bytes for MaterializedMySQL. #53381 (Filipp Ozinov).
  • Fix processing single carriage return in TSV file segmentation engine that could lead to parsing errors. Closes #53320. #53407 (Kruglov Pavel).
  • Fix the "Context has expired" error when using subqueries with functions file() (regular function, not table function), joinGet(), joinGetOrNull(), connectionId(). #53433 (Michael Kolupaev).
  • Fix timeout_overflow_mode when having subquery in the rhs of IN. #53439 (Duc Canh Le).
  • This PR fixes #53152. #53440 (Zhiguo Zhou).
  • Fix the JSON_QUERY function can not parse the json string while path is numberic. like in the query SELECT JSON_QUERY('{"123":"abcd"}', '.123'), we would encounter the exceptions ``` DB::Exception: Unable to parse JSONPath: While processing JSON_QUERY('{"123":"acd"}', '.123'). (BAD_ARGUMENTS) ```. #53470 (KevinyhZou).
  • Fix possible crash for queries with parallel FINAL where ORDER BY and PRIMARY KEY are different in table definition. #53489 (Nikolai Kochetov).
  • Fixed ReplacingMergeTree to properly process single-partition cases when do_not_merge_across_partitions_select_final=1. Previously SELECT could return rows that were marked as deleted. #53511 (Vasily Nemkov).
  • Fix bug in flushing of async insert queue on graceful shutdown. #53547 (joelynch).
  • Fix crash in join on sparse column. #53548 (vdimir).
  • Fix possible UB in Set skipping index for functions with incorrect args. #53559 (Azat Khuzhin).
  • Fix possible UB in inverted indexes (experimental feature). #53560 (Azat Khuzhin).
  • Fixed bug for interpolate when interpolated column is aliased with the same name as a source column. #53572 (Yakov Olkhovskiy).
  • Fixed a bug in EXPLAIN PLAN index=1 where the number of dropped granules was incorrect. #53616 (wangxiaobo).
  • Correctly handle totals and extremes when DelayedSource is used. #53644 (Antonio Andelic).
  • Fix Pipeline stuck error in mutation with IN (subquery WITH TOTALS) where ready set was taken from cache. #53645 (Nikolai Kochetov).
  • Allow to use JSON subcolumns in predicates of UPDATE and DELETE queries. #53677 (zps).
  • Fix possible logical error exception during filter pushdown for full_sorting_merge join. #53699 (vdimir).
  • Fix NULL::LowCardinality(Nullable(...)) with IN. #53706 (Andrey Zvonov).
  • Fixes possible crashes in DISTINCT queries with enabled optimize_distinct_in_order and sparse columns. #53711 (Igor Nikonov).
  • Correctly handle default column with multiple rows in transform. #53742 (Salvatore Mesoraca).
  • Fix crash in SQL function parseDateTime() with non-const timezone argument. #53764 (Robert Schulze).
  • Fix uncaught exception in getCreateTableQueryImpl. #53832 (Kseniia Sumarokova).
  • Fix possible segfault while using PostgreSQL engine. Closes #36919. #53847 (Kseniia Sumarokova).
  • Fix named_collection_admin alias to named_collection_control not working from config. #54066 (Kseniia Sumarokova).
  • A distributed query could miss rows_before_limit_at_least in the query result in case it was executed on a replica with a delay more than max_replica_delay_for_distributed_queries. #54122 (Nikolai Kochetov).

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT