ClickHouse/docs/changelogs/v24.1.1.2048-stable.md
2024-05-30 16:58:36 +02:00

83 KiB

sidebar_position sidebar_label
1 2024

2024 Changelog

ClickHouse release v24.1.1.2048-stable (5a024dfc09) FIXME as compared to v23.12.1.1368-stable (a2faa65b08)

Backward Incompatible Change

  • The setting print_pretty_type_names is turned on by default. You can turn it off to keep the old behavior or SET compatibility = '23.12'. #57726 (Alexey Milovidov).
  • The MergeTree setting clean_deleted_rows is deprecated, it has no effect anymore. The CLEANUP keyword for OPTIMIZE is not allowed by default (unless allow_experimental_replacing_merge_with_cleanup is enabled). #58316 (Alexander Tokmakov).
  • The function reverseDNSQuery is no longer available. This closes #58368. #58369 (Alexey Milovidov).
  • Enable various changes to improve the access control in the configuration file. These changes affect the behavior, and you check the config.xml in the access_control_improvements section. In case you are not confident, keep the values in the configuration file as they were in the previous version. #58584 (Alexey Milovidov).
  • Allow queries without aliases for subqueries for PASTE JOIN. #58654 (Yarik Briukhovetskyi).
  • Fix sumMapFiltered with NaN values. NaN values are now placed at the end (instead of randomly) and considered different from any values. -0 is now also treated as equal to 0; since 0 values are discarded, -0 values are discarded too. #58959 (Raúl Marín).
  • The function visibleWidth will behave according to the docs. In previous versions, it simply counted code points after string serialization, like the lengthUTF8 function, but didn't consider zero-width and combining characters, full-width characters, tabs, and deletes. Now the behavior is changed accordingly. If you want to keep the old behavior, set function_visible_width_behavior to 0, or set compatibility to 23.12 or lower. #59022 (Alexey Milovidov).
  • Kusto dialect is disabled until these two bugs will be fixed: #59037 and #59036. #59305 (Alexey Milovidov).

New Feature

  • Allow partitions from tables with different partition expressions to be attached when the destination table partition expression doesn't re-partition/ split the part. #39507 (Arthur Passos).
  • Added statement SYSTEM RELOAD ASYNCHRONOUS METRICS which updates the asynchronous metrics. Mostly useful for testing and development. #53710 (Robert Schulze).
  • Certain settings (currently min_compress_block_size and max_compress_block_size) can now be specified at column-level where they take precedence over the corresponding table-level setting. Example: CREATE TABLE tab (col String SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840)) ENGINE = MergeTree ORDER BY tuple();. #55201 (Duc Canh Le).
  • Add quantileDDSketch aggregate function as well as the corresponding quantilesDDSketch and medianDDSketch. It is based on the DDSketch https://www.vldb.org/pvldb/vol12/p2195-masson.pdf. ### Documentation entry for user-facing changes. #56342 (Srikanth Chekuri).
  • Added function seriesDecomposeSTL() which decomposes a time series into a season, a trend and a residual component. #57078 (Bhavna Jindal).
  • Introduced MySQL Binlog Client for MaterializedMySQL: One binlog connection for many databases. #57323 (Val Doroshchuk).
  • Intel QuickAssist Technology (QAT) provides hardware-accelerated compression and cryptograpy. ClickHouse got a new compression codec ZSTD_QAT which utilizes QAT for zstd compression. The codec uses Intel's QATlib and Inte's QAT ZSTD Plugin. Right now, only compression can be accelerated in hardware (a software fallback kicks in in case QAT could not be initialized), decompression always runs in software. #57509 (jasperzhu).
  • Implementing the new way how object storage keys are generated for s3 disks. Now the format could be defined in terms of re2 regex syntax with key_template option in disc description. #57663 (Sema Checherinda).
  • Table system.dropped_tables_parts contains parts of system.dropped_tables tables (dropped but not yet removed tables). #58038 (Yakov Olkhovskiy).
  • Implement Variant data type that represents a union of other data types. Type Variant(T1, T2, ..., TN) means that each row of this type has a value of either type T1 or T2 or ... or TN or none of them (NULL value). Variant type is available under a setting allow_experimental_variant_type. Reference: #54864. #58047 (Kruglov Pavel).
  • Add settings max_materialized_views_size_for_table to limit the number of materialized views attached to a table. #58068 (zhongyuankai).
  • clickhouse-format improvements: * support INSERT queries with VALUES * support comments (use --comments to output them) * support --max_line_length option to format only long queries in multiline. #58246 (vdimir).
  • Added null_status_on_timeout_only_active and throw_only_active modes for distributed_ddl_output_mode that allow to avoid waiting for inactive replicas. #58350 (Alexander Tokmakov).
  • Add table system.database_engines. #58390 (Bharat Nallan).
  • Added FROM modifier for SYSTEM SYNC REPLICA LIGHTWEIGHT query. The FROM modifier ensures we wait for for fetches and drop-ranges only for the specified source replicas, as well as any replica not in zookeeper or with an empty source_replica. #58393 (Jayme Bird).
  • Add function arrayShingles() to compute subarrays, e.g. arrayShingles([1, 2, 3, 4, 5], 3) returns [[1,2,3],[2,3,4],[3,4,5]]. #58396 (Zheng Miao).
  • Added functions punycodeEncode(), punycodeDecode(), idnaEncode() and idnaDecode() which are useful for translating international domain names to an ASCII representation according to the IDNA standard. #58454 (Robert Schulze).
  • Added string similarity functions dramerauLevenshteinDistance(), jaroSimilarity() and jaroWinklerSimilarity(). #58531 (Robert Schulze).
  • Add two settings output_format_compression_level to change output compression level and output_format_compression_zstd_window_log to explicitly set compression window size and enable long-range mode for zstd compression if output compression method is zstd. Applied for INTO OUTFILE and when writing to table functions file, url, hdfs, s3, and azureBlobStorage. #58539 (Duc Canh Le).
  • Automatically disable ANSI escape sequences in Pretty formats if the output is not a terminal. Add new auto mode to setting output_format_pretty_color. #58614 (Shaun Struwig).
  • Added setting update_insert_deduplication_token_in_dependent_materialized_views. This setting allows to update insert deduplication token with table identifier during insert in dependent materialized views. Closes #59165. #59238 (Maksim Kita).

Performance Improvement

  • More cache-friendly final implementation. Note on the behaviour change: previously queries with FINAL modifier that read with a single stream (e.g. max_threads=1) produced sorted output without explicitly provided ORDER BY clause. This behaviour no longer exists when enable_vertical_final = true (and it is so by default). #54366 (Duc Canh Le).
  • Optimize array element function when input is array(map)/array(array(num)/array(array(string))/array(bigint)/array(decimal). Current implementation causes too many reallocs. The optimization speed up by ~6x especially when input type is array(map). #56403 (李扬).
  • Bypass Poco::BasicBufferedStreamBuf abstraction when reading from S3 (namely ReadBufferFromIStream) to avoid extra copying of data. #56961 (Nikita Taranov).
  • Read column once while reading more that one subcolumn from it in Compact parts. #57631 (Kruglov Pavel).
  • Rewrite the AST of sum(column + literal) function. #57853 (Jiebin Sun).
  • The evaluation of function match() now utilizes skipping indices ngrambf_v1 and tokenbf_v1. #57882 (凌涛).
  • Default coordinator for parallel replicas is rewritten for better cache locality (same mark ranges are almost always assigned to the same replicas). Consistent hashing is used also during work stealing, so better tail latency is expected. It has been tested for linear scalability on a hundred of replicas. #57968 (Nikita Taranov).
  • MergeTree FINAL to not compare rows from same non-L0 part. #58142 (Duc Canh Le).
  • Speed up iota calls (filling array with consecutive numbers). #58271 (Raúl Marín).
  • The evaluation of function match() now utilizes inverted indices. #58284 (凌涛).
  • Speedup MIN/MAX for non numeric types. #58334 (Raúl Marín).
  • Enable JIT compilation for aggregation without a key. Closes #41461. Originally #53757. #58440 (Alexey Milovidov).
  • The performance experiments of OnTime on the Intel server with up to AVX2 (and BMI2) support show that this change could effectively improve the QPS of Q2 and Q3 by 5.0% and 3.7% through reducing the cycle ratio of the hotspot, DB::MergeTreeRangeReader::ReadResult::optimize, from 11.48% to 1.09% and from 8.09% to 0.67% respectively while having no impact on others. #58800 (Zhiguo Zhou).
  • Use one thread less in clickhouse-local. #58968 (Alexey Milovidov).
  • Large aggregation states of uniqExact will be merged in parallel in distrubuted queries. #59009 (Nikita Taranov).
  • Lower memory usage after reading from MergeTree tables. #59290 (Anton Popov).
  • Lower memory usage in vertical merges. #59340 (Anton Popov).

Improvement

  • Enable MySQL/MariaDB on macOS. This closes #21191. #46316 (Robert Schulze).
  • Do not interpret numbers with leading zeroes as octals. #55575 (Joanna Hulboj).
  • Replace HTTP outgoing buffering based on std ostream with CH Buffer. Add bytes counting metrics for interfaces. #56064 (Yakov Olkhovskiy).
  • Disable max_rows_in_set_to_optimize_join by default. #56396 (vdimir).
  • Add <host_name> config parameter that allows avoiding resolving hostnames in DDLWorker. This mitigates the possibility of the queue being stuck in case of a change in cluster definition. Closes #57573. #57603 (Nikolay Degterinsky).
  • Increase load_metadata_threads to 16 for the filesystem cache. It will make the server start up faster. #57732 (Alexey Milovidov).
  • Improve the multiIf function performance when the type is Nullable. #57745 (KevinyhZou).
  • Add ability to throttle merges/mutations (max_mutations_bandwidth_for_server/max_merges_bandwidth_for_server). #57877 (Azat Khuzhin).
  • Replaced undocumented (boolean) column is_hot_reloadable in system table system.server_settings by (Enum8) column changeable_without_restart with possible values No, Yes, IncreaseOnly and DecreaseOnly. Also documented the column. #58029 (skyoct).
  • ClusterDiscovery supports setting username and password, close #58063. #58123 (vdimir).
  • Support query parameters in ALTER TABLE ... PART. #58297 (Azat Khuzhin).
  • Create consumers for Kafka tables on fly (but keep them for some period - kafka_consumers_pool_ttl_ms, since last used), this should fix problem with statistics for system.kafka_consumers (that does not consumed when nobody reads from Kafka table, which leads to live memory leak and slow table detach) and also this PR enables stats for system.kafka_consumers by default again. #58310 (Azat Khuzhin).
  • Sparkbar as an alias to sparkbar. #58335 (凌涛).
  • Avoid sending ComposeObject requests after upload to GCS. #58343 (Azat Khuzhin).
  • Correctly handle keys with dot in the name in configurations XMLs. #58354 (Azat Khuzhin).
  • Added comments (brief descriptions) to all columns of system tables. The are several reasons fro this: - We use system tables a lot and sometimes is could be very difficult for developer to understand the purpose and the meaning of a particular column. - We change (add new ones or modify existing) system tables a lot and the documentation for them is always outdated. For example take a look at the documentation page for system.parts. It misses a lot of columns - We would like to eventually generate documentation directly from ClickHouse. #58356 (Nikita Mikhaylov).
  • Allow to configure any kind of object storage with any kind of metadata type. #58357 (Kseniia Sumarokova).
  • Make function format return constant on constant arguments. This closes #58355. #58358 (Alexey Milovidov).
  • Attach all system tables in clickhouse-local, including system.parts. This closes #58312. #58359 (Alexey Milovidov).
  • Support for Enum data types in function transform. This closes #58241. #58360 (Alexey Milovidov).
  • Allow registering database engines independently. #58365 (Bharat Nallan).
  • Adding a setting max_estimated_execution_time to separate max_execution_time and max_estimated_execution_time. #58402 (Zhang Yifan).
  • Allow registering interpreters independently. #58443 (Bharat Nallan).
  • Provide hint when an invalid database engine name is used. #58444 (Bharat Nallan).
  • Avoid huge memory consumption during Keeper startup for more cases. #58455 (Antonio Andelic).
  • Add settings for better control of indexes type in Arrow dictionary. Use signed integer type for indexes by default as Arrow recommends. Closes #57401. #58519 (Kruglov Pavel).
  • Added function sqidDecode() which decodes Sqids. #58544 (Robert Schulze).
  • Allow to read Bool values into String in JSON input formats. It's done under a setting input_format_json_read_bools_as_strings that is enabled by default. #58561 (Kruglov Pavel).
  • Implement #58575 Support CLICKHOUSE_PASSWORD_FILE environment variable when running the docker image. #58583 (Eyal Halpern Shalev).
  • When executing some queries, which require a lot of streams for reading data, the error "Paste JOIN requires sorted tables only" was previously thrown. Now the numbers of streams resize to 1 in that case. #58608 (Yarik Briukhovetskyi).
  • Add SYSTEM JEMALLOC PURGE for purging unused jemalloc pages, SYSTEM JEMALLOC [ ENABLE | DISABLE | FLUSH ] PROFILE for controlling jemalloc profile if the profiler is enabled. Add jemalloc-related 4LW command in Keeper: jmst for dumping jemalloc stats, jmfp, jmep, jmdp for controlling jemalloc profile if the profiler is enabled. #58665 (Antonio Andelic).
  • Better message for INVALID_IDENTIFIER error. #58703 (Yakov Olkhovskiy).
  • Improved handling of signed numeric literals in normalizeQuery. #58710 (Salvatore Mesoraca).
  • Support Point data type for MySQL. #58721 (Kseniia Sumarokova).
  • When comparing a Float32 column and a const string, read the string as Float32 (instead of Float64). #58724 (Raúl Marín).
  • Improve S3 compatible, add Ecloud EOS storage support. #58786 (xleoken).
  • Allow KILL QUERY to cancel backups / restores. This PR also makes running backups and restores visible in system.processes. Also there is a new setting in the server configuration now - shutdown_wait_backups_and_restores (default=true) which makes the server either wait on shutdown for all running backups and restores to finish or just cancel them. #58804 (Vitaly Baranov).
  • Avro format support Zstd codec. Closes #58735. #58805 (flynn).
  • MySQL interface gained support for net_write_timeout and net_read_timeout settings. net_write_timeout is translated into the native send_timeout ClickHouse setting and, similarly, net_read_timeout into receive_timeout. Fixed an issue where it was possible to set MySQL sql_select_limit setting only if the entire statement was in upper case. #58835 (Serge Klochkov).
  • Fixing a problem described in #58719. #58841 (Yarik Briukhovetskyi).
  • Make sure that for custom (created from SQL) disks ether filesystem_caches_path (a common directory prefix for all filesystem caches) or custom_cached_disks_base_directory (a common directory prefix for only filesystem caches created from custom disks) is specified in server config. custom_cached_disks_base_directory has higher priority for custom disks over filesystem_caches_path, which is used if the former one is absent. Filesystem cache setting path must lie inside that directory, otherwise exception will be thrown preventing disk to be created. This will not affect disks created on an older version and server was upgraded - then the exception will not be thrown to allow the server to successfully start). custom_cached_disks_base_directory is added to default server config as /var/lib/clickhouse/caches/. Closes #57825. #58869 (Kseniia Sumarokova).
  • MySQL interface gained compatibility with SHOW WARNINGS/SHOW COUNT(*) WARNINGS queries, though the returned result is always an empty set. #58929 (Serge Klochkov).
  • Skip unavailable replicas when executing parallel distributed INSERT SELECT. #58931 (Alexander Tokmakov).
  • Display word-descriptive log level while enabling structured log formatting in json. #58936 (Tim Liou).
  • MySQL interface gained support for CAST(x AS SIGNED) and CAST(x AS UNSIGNED) statements via data type aliases: SIGNED for Int64, and UNSIGNED for UInt64. This improves compatibility with BI tools such as Looker Studio. #58954 (Serge Klochkov).
  • Function seriesDecomposeSTL() now returns a baseline component as season + trend components. #58961 (Bhavna Jindal).
  • Fix memory management in copyDataToS3File. #58962 (Vitaly Baranov).
  • Change working directory to data path in docker container. #58975 (cangyin).
  • Added setting for Azure Blob Storage azure_max_unexpected_write_error_retries , can also be set from config under azure section. #59001 (SmitaRKulkarni).
  • Keeper improvement: reduce Keeper's memory usage for stored nodes. #59002 (Antonio Andelic).
  • Allow server to start with broken data lake table. Closes #58625. #59080 (Kseniia Sumarokova).
  • Fixes https://github.com/ClickHouse/ClickHouse/pull/59120#issuecomment-1906177350. #59122 (Arthur Passos).
  • The state of URL's #hash in the dashboard is now compressed using lz-string. The default size of the state string is compressed from 6856B to 2823B. #59124 (Amos Bird).
  • Allow to ignore schema evolution in Iceberg table engine and read all data using schema specified by the user on table creation or latest schema parsed from metadata on table creation. This is done under a setting iceberg_engine_ignore_schema_evolution that is disabled by default. Note that enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema. #59133 (Kruglov Pavel).
  • Prohibit mutable operations (INSERT/ALTER/OPTIMIZE/...) on read-only/write-once storages with a proper TABLE_IS_READ_ONLY error (to avoid leftovers). Avoid leaving left-overs on write-once disks (format_version.txt) on CREATE/ATTACH. Ignore DROP for ReplicatedMergeTree (so as for MergeTree). Fix iterating over s3_plain (MetadataStorageFromPlainObjectStorage::iterateDirectory). Note read-only is web disk, and write-once is s3_plain. #59170 (Azat Khuzhin).
  • MySQL interface gained support for net_write_timeout and net_read_timeout settings. net_write_timeout is translated into the native send_timeout ClickHouse setting and, similarly, net_read_timeout into receive_timeout. Fixed an issue where it was possible to set MySQL sql_select_limit setting only if the entire statement was in upper case. #59293 (Serge Klochkov).
  • Fix bug in experimental _block_number column which could lead to logical error during complex combination of ALTERs and merges. Fixes #56202. Replaces #58601. CC @SmitaRKulkarni. #59295 (alesapin).
  • Play UI understands when an exception is returned inside JSON. Adjustment for #52853. #59303 (Alexey Milovidov).
  • /binary HTTP handler allows to specify user, host, and optionally, password in the query string. #59311 (Alexey Milovidov).
  • Support backups for compressed in-memory tables. This closes #57893. #59315 (Alexey Milovidov).
  • Improve exception message of function regexp_extract, close #56393. #59319 (李扬).
  • Support the FORMAT clause in BACKUP and RESTORE queries. #59338 (Vitaly Baranov).
  • Function concatWithSeparator() now supports arbitrary argument types (instead of only String and FixedString arguments). For example, SELECT concatWithSeparator('.', 'number', 1) now returns number.1. #59341 (Robert Schulze).

Build/Testing/Packaging Improvement

  • Improve aliases for clickhouse binary (now ch/clickhouse is clickhouse-local or clickhouse depends on the arguments) and add bash completion for new aliases. #58344 (Azat Khuzhin).
  • Add settings changes check to CI to check that all settings changes are reflected in settings changes history. #58555 (Kruglov Pavel).
  • Use tables directly attached from S3 in stateful tests. #58791 (Alexey Milovidov).
  • Save the whole fuzzer.log as an archive instead of the last 100k lines. tail -n 100000 often removes lines with table definitions. Example:. #58821 (Dmitry Novik).
  • Enable Rust on OSX ARM64 (this will add fuzzy search in client with skim and prql language, though I don't think that are people who hosts ClickHouse on darwin, so it is mostly for fuzzy search in client I would say). #59272 (Azat Khuzhin).

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

  • Fix possible errors when joining sub-types with low cardinality (e.g., Array(LowCardinality(T)) with Array(T)). #51550 (vdimir).
  • Flatten only true Nested type if flatten_nested=1, not all Array(Tuple). #56132 (Kruglov Pavel).
  • Fix a bug with projections and the aggregate_functions_null_for_empty setting during insertion. This is an addition to #42198 and #49873. The bug was found by fuzzer in #56666. This PR also fix potential issues with projections and the transform_null_in setting. #56944 (Amos Bird).
  • Fixed (a rare) exception in case when user's assigned profiles are updated right after user logging in, which could cause a missing entry in session_log or problems with logging in. #57263 (Vasily Nemkov).
  • Fix working with read buffers in StreamingFormatExecutor, previously it could lead to segfaults in Kafka and other streaming engines. #57438 (Kruglov Pavel).
  • Ignore MVs with dropped target table during pushing to views in insert to a source table. #57520 (Kruglov Pavel).
  • Eliminate possible race between ALTER_METADATA and MERGE_PARTS (that leads to checksum mismatch - CHECKSUM_DOESNT_MATCH). #57755 (Azat Khuzhin).
  • Fix the exprs order bug in group by with rollup. #57786 (Chen768959).
  • Fix a bug in zero-copy-replication (an experimental feature) that could lead to The specified key does not exist error and data loss. It could happen when dropping a replica with broken or unexpected/ignored detached parts. Fixes #57985. #58333 (Alexander Tokmakov).
  • Fix a bug that users cannot work with symlinks in user_files_path. #58447 (Duc Canh Le).
  • Fix segfault when graphite table does not have agg function. #58453 (Duc Canh Le).
  • Fix reading multiple times from KafkaEngine in materialized views. #58477 (János Benjamin Antal).
  • Fix Part ... intersects part ... error that might occur in ReplicatedMergeTree when the server was restarted just after [automatically] dropping [an empty] part and adjacent parts were merged. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/56282. #58482 (Alexander Tokmakov).
  • MergeTreePrefetchedReadPool disable for LIMIT only queries, because time spend during filling per thread tasks can be greater than whole query execution for big tables with small limit. #58505 (Maksim Kita).
  • While restore is underway in Clickhouse, restore should allow the database with an ordinary engine. #58520 (Jihyuk Bok).
  • Fix read buffer creation in Hive engine when thread_pool read method is used. Closes #57978. #58537 (sunny).
  • Hide credentials in base_backup_name column of system.backup_log. #58550 (Daniel Pozo Escalona).
  • While executing queries like SELECT toStartOfInterval(toDateTime64('2023-10-09 10:11:12.000999', 6), toIntervalMillisecond(1));, the result was not rounded to 1 millisecond previously. Current PR solves this issue. Also, current PR will solve some problems appearing in https://github.com/ClickHouse/ClickHouse/pull/56738. #58557 (Yarik Briukhovetskyi).
  • Fix logical error in parallel_hash working with max_joined_block_size_rows. #58595 (vdimir).
  • Fix error in join with USING when one of the table has Nullable key. #58596 (vdimir).
  • The (optional) fraction argument in function makeDateTime64() can now be non-const. This was possible already with ClickHouse <= 23.8. #58597 (Robert Schulze).
  • Fix possible server crash during symbolizing inline frames. #58607 (Azat Khuzhin).
  • The query cache now denies access to entries when the user is re-created or assumes another role. This improves prevents attacks where 1. an user with the same name as a dropped user may access the old user's cache entries or 2. a user with a different role may access cache entries of a role with a different row policy. #58611 (Robert Schulze).
  • Fix broken partition key analysis when doing projection optimization with force_index_by_date = 1. This fixes #58620. We don't need partition key analysis for projections after https://github.com/ClickHouse/ClickHouse/pull/56502 . #58638 (Amos Bird).
  • The query cache now behaves properly when per-user quotas are defined and SYSTEM DROP QUERY CACHE ran. #58731 (Robert Schulze).
  • Fix data stream partitioning for window functions when there are different window descriptions with similar prefixes but different partitioning. Fixes #58714. #58739 (Dmitry Novik).
  • Fix double destroy call on exception throw in addBatchLookupTable8. #58745 (Raúl Marín).
  • Keeper fix: don't process requests during shutdown because it will lead to invalid state. #58765 (Antonio Andelic).
  • Fix a crash in the polygon dictionary. Fixes #58612. #58771 (Yarik Briukhovetskyi).
  • Fix possible crash in JSONExtract function extracting LowCardinality(Nullable(T)) type. #58808 (vdimir).
  • Table CREATE DROP Poco::Logger memory leak fix. Closes #57931. Closes #58496. #58831 (Maksim Kita).
  • Fix HTTP compressors. Follow-up #58475. #58846 (Yakov Olkhovskiy).
  • Fix reading multiple times from FileLog engine in materialized views. #58877 (János Benjamin Antal).
  • Prevent specifying an access_key_id that does not match the correct correct pattern. #58900 (MikhailBurdukov).
  • Fix possible crash in clickhouse-local during loading suggestions. Closes #58825. #58907 (Kruglov Pavel).
  • Fix crash when indexHint function is used without arguments in the filters. #58911 (Dmitry Novik).
  • Fixed URL and S3 engines losing the headers argument on server restart. #58933 (Michael Kolupaev).
  • Fix analyzer - insertion from select with subquery referencing insertion table should process only insertion block for all table expressions. Fixes #58080. follow-up #50857. #58958 (Yakov Olkhovskiy).
  • Fixed reading parquet files from archives. #58966 (Michael Kolupaev).
  • Experimental feature of inverted indices: ALTER TABLE DROP INDEX for an inverted index now removes all inverted index files from the new part (issue #59039). #59040 (mochi).
  • Fix data race on collecting factories info for system.query_log. #59049 (Kseniia Sumarokova).
  • Fixs: #58967. #59099 (skyoct).
  • Fixed wrong aggregation results in mixed x86_64 and ARM clusters. #59132 (Harry Lee).
  • Fix a deadlock that can happen during the shutdown of the server due to metadata loading failure. #59137 (Sergei Trifonov).
  • The combination of LIMIT BY and LIMIT could produce an incorrect result in distributed queries (parallel replicas included). #59153 (Igor Nikonov).
  • Fixes crash with for toString() with timezone in nullable format. Fixes #59126. #59190 (Yarik Briukhovetskyi).
  • Fix abort in iceberg metadata on bad file paths. #59275 (Kruglov Pavel).
  • Fix architecture name in select of Rust target. #59307 (p1rattttt).
  • Fix Not-ready Set for queries from system.tables with table IN (subquery) filter expression. Fixes #59342. #59351 (Nikolai Kochetov).
  • Fix lazy initialization in RabbitMQ that could lead to logical error and not initialized state. #59352 (Kruglov Pavel).

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT