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

85 KiB
Raw Permalink Blame History

sidebar_position sidebar_label
1 2024

2024 Changelog

ClickHouse release v24.2.1.2248-stable (891689a415) FIXME as compared to v24.1.1.2048-stable (5a024dfc09)

Backward Incompatible Change

  • Validate suspicious/experimental types in nested types. Previously we didn't validate such types (except JSON) in nested types like Array/Tuple/Map. #59385 (Kruglov Pavel).
  • The sort clause ORDER BY ALL (introduced with v23.12) is replaced by ORDER BY *. The previous syntax was too error-prone for tables with a column all. #59450 (Robert Schulze).
  • Rename the setting extract_kvp_max_pairs_per_row to extract_key_value_pairs_max_pairs_per_row. The bug (unnecessary abbreviation in the setting name) was introduced in https://github.com/ClickHouse/ClickHouse/pull/43606. Fix the documentation of this setting. #59683 (Alexey Milovidov).
  • Rename the setting extract_kvp_max_pairs_per_row to extract_key_value_pairs_max_pairs_per_row. The bug (unnecessary abbreviation in the setting name) was introduced in https://github.com/ClickHouse/ClickHouse/pull/43606. Fix the documentation of this setting. #59960 (jsc0218).
  • Add sanity check for number of threads and block sizes. #60138 (Raúl Marín).

New Feature

  • Added maximum sequential login failures to the quota. #54737 (Alexey Gerasimchuck).
  • Added new syntax which allows to specify definer user in View/Materialized View. This allows to execute selects/inserts from views without explicit grants for underlying tables. #54901 (pufit).
  • Backup & Restore support for AzureBlobStorage resolves #50747. #56988 (SmitaRKulkarni).
  • Implemented automatic conversion of merge tree tables of different kinds to replicated engine. Create empty convert_to_replicated file in table's data directory (/clickhouse/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/) and that table will be converted automatically on next server start. #57798 (Kirill).
  • Added table function mergeTreeIndex. It represents the contents of index and marks files of MergeTree tables. It can be used for introspection. Syntax: mergeTreeIndex(database, table, [with_marks = true]) where database.table is an existing table with MergeTree engine. #58140 (Anton Popov).
  • Added function seriesOutliersTukey to detect outliers in series data using Tukey's fences algorithm. #58632 (Bhavna Jindal).
  • The user can now specify the template string directly in the query using format_schema_rows_template as an alternative to format_template_row. Closes #31363. #59088 (Shaun Struwig).
  • Try to detect file format automatically during schema inference if it's unknown in file/s3/hdfs/url/azureBlobStorage engines. Closes #50576. #59092 (Kruglov Pavel).
  • Add function variantType that returns Enum with variant type name for each row. #59398 (Kruglov Pavel).
  • Added query ALTER TABLE table FORGET PARTITION partition that removes ZooKeeper nodes, related to an empty partition. #59507 (Sergei Trifonov).
  • Support JWT credentials file for the NATS table engine. #59543 (Nickolaj Jepsen).
  • Provides new aggregate function groupArrayIntersect. Follows up: #49862. #59598 (Yarik Briukhovetskyi).
  • Implemented system.dns_cache table, which can be useful for debugging DNS issues. #59856 (Kirill Nikiforov).
  • The codec LZ4HC will accept a new level 2, which is faster than the previous minimum level 3, at the expense of less compression. In previous versions, LZ4HC(2) and less was the same as LZ4HC(3). Author: Cyan4973. #60090 (Alexey Milovidov).
  • Implemented system.dns_cache table, which can be useful for debugging DNS issues. New server setting dns_cache_max_size. #60257 (Kirill Nikiforov).
  • Support single-argument version for the merge table function, as merge(['db_name', ] 'tables_regexp'). #60372 (豪肥肥).
  • Added new syntax which allows to specify definer user in View/Materialized View. This allows to execute selects/inserts from views without explicit grants for underlying tables. #60439 (pufit).

Performance Improvement

  • Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section. #52230 (JackyWoo).
  • Vectorized distance functions used in vector search. #58866 (Robert Schulze).
  • Continue optimizing branch miss of if function when result type is float*/decimal*/int* , follow up of https://github.com/ClickHouse/ClickHouse/pull/57885. #59148 (李扬).
  • Optimize if function when input type is map, speed up by ~10x. #59413 (李扬).
  • Improve performance of Int8 type by implementing strict aliasing. #59485 (Raúl Marín).
  • Optimize performance of sum/avg conditionally for bigint and big decimal types by reducing branch miss. #59504 (李扬).
  • Improve performance of SELECTs with active mutations. #59531 (Azat Khuzhin).
  • Optimized function isNotNull with AVX2. #59621 (李扬).
  • Reuse the result of FunctionFactory::instance().get("isNotNull", context) and FunctionFactory::instance().get("assumeNotNull", context). Make sure it is called once during the lifetime of FunctionCoalesce. #59627 (李扬).
  • Improve ASOF JOIN performance for sorted or almost sorted data. #59731 (Maksim Kita).
  • Primary key will use less amount of memory. #60049 (Alexey Milovidov).
  • Improve memory usage for primary key and some other operations. #60050 (Alexey Milovidov).
  • The tables' primary keys will be loaded in memory lazily on first access. This is controlled by the new MergeTree setting primary_key_lazy_load, which is on by default. This provides several advantages: - it will not be loaded for tables that are not used; - if there is not enough memory, an exception will be thrown on first use instead of at server startup. This provides several disadvantages: - the latency of loading the primary key will be paid on the first query rather than before accepting connections; this theoretically may introduce a thundering-herd problem. This closes #11188. #60093 (Alexey Milovidov).
  • Vectorized function dotProduct which is useful for vector search. #60202 (Robert Schulze).
  • As is shown in Fig 1, the replacement of "&&" with "&" could generate the SIMD code. image Fig 1. Code compiled from '&&' (left) and '&' (right). #60498 (Zhiguo Zhou).

Improvement

  • Added support for parameterized view with analyzer to not analyze create parameterized view. Refactor existing parameterized view logic to not analyze create parameterized view. #54211 (SmitaRKulkarni).
  • Do not consider data part as broken if projection is broken. Closes #56593. #56864 (Kseniia Sumarokova).
  • Add short-circuit ability for dictGetOrDefault function. Closes #52098. #57767 (jsc0218).
  • Running ALTER COLUMN MATERIALIZE on a column with DEFAULT or MATERIALIZED expression now writes the correct values: The default value for existing parts with default value or the non-default value for existing parts with non-default value. Previously, the default value was written for all existing parts. #58023 (Duc Canh Le).
  • Enabled a backoff logic (e.g. exponential). Will provide an ability for reduced CPU usage, memory usage and log file sizes. #58036 (MikhailBurdukov).
  • Add improvement to count InitialQuery. #58195 (Unalian).
  • Support negative positional arguments. Closes #57736. #58292 (flynn).
  • Implement auto-adjustment for asynchronous insert timeouts. The following settings are introduced: async_insert_poll_timeout_ms, async_insert_use_adaptive_busy_timeout, async_insert_busy_timeout_min_ms, async_insert_busy_timeout_max_ms, async_insert_busy_timeout_increase_rate, async_insert_busy_timeout_decrease_rate. #58486 (Julia Kartseva).
  • Allow to define volume_priority in storage_configuration. #58533 (Andrey Zvonov).
  • Add support for Date32 type in T64 codec. #58738 (Hongbin Ma (Mahone)).
  • Support LEFT JOIN, ALL INNER JOIN, and simple subqueries for parallel replicas (only with analyzer). New setting parallel_replicas_prefer_local_join chooses local JOIN execution (by default) vs GLOBAL JOIN. All tables should exist on every replica from cluster_for_parallel_replicas. New settings min_external_table_block_size_rows and min_external_table_block_size_bytes are used to squash small blocks that are sent for temporary tables (only with analyzer). #58916 (Nikolai Kochetov).
  • Allow trailing commas in types with several items. #59119 (Aleksandr Musorin).
  • Allow parallel and distributed processing for S3Queue table engine. For distributed processing use setting s3queue_total_shards_num (by default 1). Setting s3queue_processing_threads_num previously was not allowed for Ordered processing mode, now it is allowed. Warning: settings s3queue_processing_threads_num(processing threads per each shard) and s3queue_total_shards_num for ordered mode change how metadata is stored (make the number of max_processed_file nodes equal to s3queue_processing_threads_num * s3queue_total_shards_num), so they must be the same for all shards and cannot be changed once at least one shard is created. #59167 (Kseniia Sumarokova).
  • Allow concurrent table creation in DatabaseReplicated during recoverLostReplica. #59277 (Konstantin Bogdanov).
  • Settings for the Distributed table engine can now be specified in the server configuration file (similar to MergeTree settings), e.g. <distributed> <flush_on_detach>false</flush_on_detach> </distributed>. #59291 (Azat Khuzhin).
  • Use MergeTree as a default table engine. It makes the usability much better, and closer to ClickHouse Cloud. #59316 (Alexey Milovidov).
  • Retry disconnects and expired sessions when reading system.zookeeper. This is helpful when reading many rows from system.zookeeper table especially in the presence of fault-injected disconnects. #59388 (Alexander Gololobov).
  • Do not interpret numbers with leading zeroes as octals when input_format_values_interpret_expressions=0. #59403 (Joanna Hulboj).
  • At startup and whenever config files are changed, ClickHouse updates the hard memory limits of its total memory tracker. These limits are computed based on various server settings and cgroups limits (on Linux). Previously, setting /sys/fs/cgroup/memory.max (for cgroups v2) was hard-coded. As a result, cgroup v2 memory limits configured for nested groups (hierarchies), e.g. /sys/fs/cgroup/my/nested/group/memory.max were ignored. This is now fixed. The behavior of v1 memory limits remains unchanged. #59435 (Robert Schulze).
  • New profile events added to observe the time spent on calculating PK/projections/secondary indices during INSERT-s. #59436 (Nikita Taranov).
  • Allow to define a starting point for S3Queue with Ordered mode at creation using setting s3queue_last_processed_path. #59446 (Kseniia Sumarokova).
  • Keeper improvement: cache only a certain amount of logs in-memory controlled by latest_logs_cache_size_threshold and commit_logs_cache_size_threshold. #59460 (Antonio Andelic).
  • Made comments for system tables also available in system.tables in clickhouse-local. #59493 (Nikita Mikhaylov).
  • Don't infer floats in exponential notation by default. Add a setting input_format_try_infer_exponent_floats that will restore previous behaviour (disabled by default). Closes #59476. #59500 (Kruglov Pavel).
  • Allow alter operations to be surrounded by parenthesis. The emission of parentheses can be controlled by the format_alter_operations_with_parentheses config. By default in formatted queries the parentheses are emitted as we store the formatted alter operations in some places as metadata (e.g.: mutations). The new syntax clarifies some of the queries where alter operations end in a list. E.g.: ALTER TABLE x MODIFY TTL date GROUP BY a, b, DROP COLUMN c cannot be parsed properly with the old syntax. In the new syntax the query ALTER TABLE x (MODIFY TTL date GROUP BY a, b), (DROP COLUMN c) is obvious. Older versions are not able to read the new syntax, therefore using the new syntax might cause issues if newer and older version of ClickHouse are mixed in a single cluster. #59532 (János Benjamin Antal).
  • The previous default value equals to 1 MB for async_insert_max_data_size appeared to be too small. The new one would be 10 MiB. #59536 (Nikita Mikhaylov).
  • Previously the whole result was accumulated in memory and returned as one big chunk. This change should help to reduce memory consumption when reading many rows from system.zookeeper, allow showing intermediate progress (how many rows have been read so far) and avoid hitting connection timeout when result set is big. #59545 (Alexander Gololobov).
  • Now dashboard understands both compressed and uncompressed state of URL's #hash (backward compatibility). Continuation of #59124 . #59548 (Amos Bird).
  • Bumped Intel QPL (used by codec DEFLATE_QPL) from v1.3.1 to v1.4.0 . Also fixed a bug for polling timeout mechanism, as we observed in same cases timeout won't work properly, if timeout happen, IAA and CPU may process buffer concurrently. So far, we'd better make sure IAA codec status is not QPL_STS_BEING_PROCESSED, then fallback to SW codec. #59551 (jasperzhu).
  • Keeper improvement: reduce size of data node even more. #59592 (Antonio Andelic).
  • Do not show a warning about the server version in ClickHouse Cloud because ClickHouse Cloud handles seamless upgrades automatically. #59657 (Alexey Milovidov).
  • After self-extraction temporary binary is moved instead copying. #59661 (Yakov Olkhovskiy).
  • Fix stack unwinding on MacOS. This closes #53653. #59690 (Nikita Mikhaylov).
  • Check for stack overflow in parsers even if the user misconfigured the max_parser_depth setting to a very high value. This closes #59622. #59697 (Alexey Milovidov).
  • Perform synchronous inserts if dependent MV deduplication is enabled through deduplicate_blocks_in_dependent_materialized_views=1. #59699 (Julia Kartseva).
  • Added settings split_parts_ranges_into_intersecting_and_non_intersecting_final and split_intersecting_parts_ranges_into_layers_final. This settings are needed to disable optimizations for queries with FINAL and needed for debug only. #59705 (Maksim Kita).
  • Unify xml and sql created named collection behaviour in kafka storage. #59710 (Pervakov Grigorii).
  • In case when merge_max_block_size_bytes is small enough and tables contain wide rows (strings or tuples) background merges may stuck in an endless loop. This behaviour is fixed. Follow-up for https://github.com/ClickHouse/ClickHouse/pull/59340. #59812 (Nikita Mikhaylov).
  • Allow uuid in replica_path if CREATE TABLE explicitly has it. #59908 (Azat Khuzhin).
  • Add column metadata_version of ReplicatedMergeTree table in system.tables system table. #59942 (Maksim Kita).
  • Keeper improvement: send only Keeper related metrics/events for Prometheus. #59945 (Antonio Andelic).
  • The dashboard will display metrics across different ClickHouse versions even if the structure of system tables has changed after the upgrade. #59967 (Alexey Milovidov).
  • Allow loading AZ info from a file. #59976 (Konstantin Bogdanov).
  • Keeper improvement: add retries on failures for Disk related operations. #59980 (Antonio Andelic).
  • Add new config setting backups.remove_backup_files_after_failure: <clickhouse> <backups> <remove_backup_files_after_failure>true</remove_backup_files_after_failure> </backups> </clickhouse>. #60002 (Vitaly Baranov).
  • Use multiple threads while reading the metadata of tables from a backup while executing the RESTORE command. #60040 (Vitaly Baranov).
  • Now if StorageBuffer has more than 1 shard (num_layers > 1) background flush will happen simultaneously for all shards in multiple threads. #60111 (alesapin).
  • Support specifying users for specific S3 settings in config using user key. #60144 (Antonio Andelic).
  • Copy S3 file GCP fallback to buffer copy in case GCP returned Internal Error with GATEWAY_TIMEOUT HTTP error code. #60164 (Maksim Kita).
  • Implement comparison operator for Variant values and proper Field inserting into Variant column. Don't allow creating Variant type with similar variant types by default (allow uder a setting allow_suspicious_variant_types) Closes #59996. Closes #59850. #60198 (Kruglov Pavel).
  • Short circuit execution for ULIDStringToDateTime. #60211 (Juan Madurga).
  • Added query_id column for tables system.backups and system.backup_log. Added error stacktrace to error column. #60220 (Maksim Kita).
  • Connections through the MySQL port now automatically run with setting prefer_column_name_to_alias = 1 to support QuickSight out-of-the-box. Also, settings mysql_map_string_to_text_in_show_columns and mysql_map_fixed_string_to_text_in_show_columns are now enabled by default, affecting also only MySQL connections. This increases compatibility with more BI tools. #60365 (Robert Schulze).
  • When output format is Pretty format and a block consists of a single numeric value which exceeds one million, A readable number will be printed on table right. e.g. ┌──────count()─┐ │ 233765663884 │ -- 233.77 billion └──────────────┘. #60379 (rogeryk).
  • Fix a race condition in JavaScript code leading to duplicate charts on top of each other. #60392 (Alexey Milovidov).
  • Check for stack overflow in parsers even if the user misconfigured the max_parser_depth setting to a very high value. This closes #59622. #60434 (Alexey Milovidov).

Build/Testing/Packaging Improvement

  • Added builds and tests with coverage collection with introspection. Continuation of #56102. #58792 (Alexey Milovidov).
  • Include pytest-reportlog in integration test CI runner Dockerfile to enable JSON test reports. #58926 (MyroTk).
  • Update the rust toolchain in corrosion-cmake when the CMake cross-compilation toolchain variable is set. #59309 (Aris Tritas).
  • Add some fuzzing to ASTLiterals. #59383 (Raúl Marín).
  • If you want to run initdb scripts every time when ClickHouse container is starting you shoud initialize environment varible CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS. #59808 (Alexander Nikolaev).
  • Remove ability to disable generic clickhouse components (like server/client/...), but keep some that requires extra libraries (like ODBC or keeper). #59857 (Azat Khuzhin).
  • Query fuzzer will fuzz SETTINGS inside queries. #60087 (Alexey Milovidov).
  • Add test that validates projections still work after attach partition. #60415 (Arthur Passos).
  • Add test that validates attach partition fails if structure differs because of materialized column. #60418 (Arthur Passos).
  • Add support for building ClickHouse with clang-19 (master). #60448 (Alexey Milovidov).
  • Speedup check-whitespaces check. #60496 (Raúl Marín).

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

  • Support IN (subquery) in table TTL expression. Initially, it was allowed to create such a TTL expression, but any TTL merge would fail with Not-ready Set error in the background. Now, TTL is correctly applied. Subquery is executed for every TTL merge, and its result is not cached or reused by other merges. Use such configuration with special care, because subqueries in TTL may lead to high memory consumption and, possibly, a non-deterministic result of TTL merge on different replicas (which is correctly handled by replication, however). #57430 (Nikolai Kochetov).
  • Fix quantilesGK bug, close #57683. #58216 (李扬).
  • Disable parallel replicas JOIN with CTE (not analyzer). #59239 (Raúl Marín).
  • Fixes bug with for function intDiv with decimal arguments. Fixes #56414. #59243 (Yarik Briukhovetskyi).
  • Fix translate() with FixedString input. Could lead to crashes as it'd return a String column (vs the expected FixedString). This issue was found through ClickHouse Bug Bounty Program YohannJardin. #59356 (Raúl Marín).
  • Keeper fix: fix digest calculation for nodes. #59439 (Antonio Andelic).
  • Fix stacktraces for binaries without debug symbols. #59444 (Azat Khuzhin).
  • Fix formatting of alter commands in case of column specific settings. #59445 (János Benjamin Antal).
  • SELECT * FROM [...] ORDER BY ALL SETTINGS allow_experimental_analyzer = 1 now works. #59462 (zhongyuankai).
  • Fix possible uncaught exception during distributed query cancellation. Closes #59169. #59487 (Azat Khuzhin).
  • Make MAX use the same rules as permutation for complex types. #59498 (Raúl Marín).
  • Fix a corner case when passing update_insert_deduplication_token_in_dependent_materialized_views setting. There is one corner case not covered due to the absence of tables in the path:. #59544 (Jordi Villar).
  • Fix incorrect result of arrayElement / map[] on empty value. #59594 (Raúl Marín).
  • Fix crash in topK when merging empty states. #59603 (Raúl Marín).
  • Fix distributed table with a constant sharding key. #59606 (Vitaly Baranov).
  • Fix segmentation fault in KQL parser when the input query exceeds the max_query_size. Also re-enable the KQL dialect. Fixes #59036 and #59037. #59626 (Yong Wang).
  • Fix error Read beyond last offset for AsynchronousBoundedReadBuffer. #59630 (Vitaly Baranov).
  • Maintain function alias in RewriteSumFunctionWithSumAndCountVisitor. #59658 (Raúl Marín).
  • Fix query start time on non initial queries. #59662 (Raúl Marín).
  • Validate types of arguments for minmax skipping index. #59733 (Anton Popov).
  • Fix leftPad / rightPad function with FixedString input. #59739 (Raúl Marín).
  • Fixed an exception in function countMatches with non-const FixedString haystack arguments, e.g. SELECT countMatches(materialize(toFixedString('foobarfoo', 9)), 'foo');. #59752 (Robert Schulze).
  • Fix having neigher acked nor nacked messages. If exception happens during read-write phase, messages will be nacked. #59775 (Kseniia Sumarokova).
  • Fixed queries that read a Parquet file over HTTP (url()/URL()) executing in one thread instead of max_threads. #59833 (Michael Kolupaev).
  • Fixed uninitialized value in s3 queue, which happened during upgrade to a new version if table had Ordered mode and resulted in an error "Existing table metadata in ZooKeeper differs in s3queue_processing_threads_num setting". #59897 (Kseniia Sumarokova).
  • Fix parsing of partition expressions that are surrounded by parentheses, e.g.: ALTER TABLE test DROP PARTITION ('2023-10-19'). #59901 (János Benjamin Antal).
  • Fix crash in JSONColumnsWithMetadata format over http. Closes #59853. #59925 (Kruglov Pavel).
  • Do not rewrite sum() to count() if return value differs in analyzer. #59926 (Azat Khuzhin).
  • Fix crash during deserialization of aggregation function states that internally use UniqExactSet. Introduced https://github.com/ClickHouse/ClickHouse/pull/59009. #59928 (Maksim Kita).
  • ReplicatedMergeTree fix invalid metadata_version node initialization in Zookeeper during creation of non first replica. Closes #54902. #59946 (Maksim Kita).
  • Fixed data race on cluster object between StorageDistributed and Context::reloadClusterConfig(). Former held const reference to its member while the latter destroyed the object (in process of replacing it with a new one). #59987 (Nikita Taranov).
  • Fixes #59989: runs init scripts when force-enabled or when no database exists, rather than the inverse. #59991 (jktng).
  • This PR fixes scale conversion for DateTime64 values (for example, DateTime64(6)->DateTime64(3)). ```SQL create table test (result DateTime64(3)) engine=Memory;. #60004 (Yarik Briukhovetskyi).
  • Fix INSERT into SQLite with single quote (by properly escaping single quotes with a quote instead of backslash). #60015 (Azat Khuzhin).
  • Fix several logical errors in arrayFold. Fixes support for Nullable and LowCardinality. #60022 (Raúl Marín).
  • Fix optimize_uniq_to_count removing the column alias. #60026 (Raúl Marín).
  • Fix possible error while dropping s3queue table, like "no node shard0". #60036 (Kseniia Sumarokova).
  • Fix formatting of NOT with single literals. #60042 (Raúl Marín).
  • Use max_query_size from context in parsing changed settings in DDLWorker. Previously with large number of changed settings DDLWorker could fail with Max query size exceeded error and don't process log entries. #60083 (Kruglov Pavel).
  • Fix inconsistent formatting of queries containing tables named table. Fix wrong formatting of queries with UNION ALL, INTERSECT, and EXCEPT when their structure wasn't linear. This closes #52349. Fix wrong formatting of SYSTEM queries, including SYSTEM ... DROP FILESYSTEM CACHE, SYSTEM ... REFRESH/START/STOP/CANCEL/TEST VIEW, SYSTEM ENABLE/DISABLE FAILPOINT. Fix formatting of parameterized DDL queries. Fix the formatting of the DESCRIBE FILESYSTEM CACHE query. Fix incorrect formatting of the SET param_... (a query setting a parameter). Fix incorrect formatting of CREATE INDEX queries. Fix inconsistent formatting of CREATE USER and similar queries. Fix inconsistent formatting of CREATE SETTINGS PROFILE. Fix incorrect formatting of ALTER ... MODIFY REFRESH. Fix inconsistent formatting of window functions if frame offsets were expressions. Fix inconsistent formatting of RESPECT NULLS and IGNORE NULLS if they were used after a function that implements an operator (such as plus). Fix idiotic formatting of SYSTEM SYNC REPLICA ... LIGHTWEIGHT FROM .... Fix inconsistent formatting of invalid queries with GROUP BY GROUPING SETS ... WITH ROLLUP/CUBE/TOTALS. Fix inconsistent formatting of GRANT CURRENT GRANTS. Fix inconsistent formatting of CREATE TABLE (... COLLATE). Additionally, I fixed the incorrect formatting of EXPLAIN in subqueries (#60102). Fixed incorrect formatting of lambda functions (#60012). Added a check so there is no way to miss these abominations in the future. #60095 (Alexey Milovidov).
  • Queries like SELECT * FROM (EXPLAIN ...) were formatted incorrectly. #60102 (Alexey Milovidov).
  • Fix cosineDistance crash with Nullable. #60150 (Raúl Marín).
  • Boolean values in string representation now cast to true bools. E.g. this query previously threw an exception but now works: SELECT true = 'true'. #60160 (Robert Schulze).
  • Fix non-filled column table_uuid in system.s3queue_log. Added columns database and table. Renamed table_uuid to uuid. #60166 (Kseniia Sumarokova).
  • Fix arrayReduce with nullable aggregate function name. #60188 (Raúl Marín).
  • Fix actions execution during preliminary filtering (PK, partition pruning). #60196 (Azat Khuzhin).
  • Hide sensitive info for S3Queue table engine. #60233 (Kseniia Sumarokova).
  • Restore the previous syntax ORDER BY ALL which has temporarily (for a few days) been replaced by ORDER BY *. #60248 (Robert Schulze).
  • Fixed a minor bug that caused all http return codes to be 200 (success) instead of a relevant code on exception. #60252 (Austin Kothig).
  • Fix bug in S3Queue table engine with ordered parallel mode. #60282 (Kseniia Sumarokova).
  • Fix use-of-uninitialized-value and invalid result in hashing functions with IPv6. #60359 (Kruglov Pavel).
  • Fix OptimizeDateOrDateTimeConverterWithPreimageVisitor with null arguments. #60453 (Raúl Marín).
  • Fixed a minor bug that prevented distributed table queries sent from either KQL or PRQL dialect clients to be executed on replicas. #60470 (Alexey Milovidov).
  • Fix incomplete results with s3Cluster when multiple threads are used. #60477 (Antonio Andelic).

CI Fix or Improvement (changelog entry is not required)

  • ... #60457 (Max K.).
  • ... #60512 (Max K.).
  • Arm and amd docker build jobs use similar job names and thus overwrite job reports - aarch64 and amd64 suffixes added to fix this. #60554 (Max K.).
  • ... #60557 (Max K.).
  • BUG: build job can report success cache record on failed build Add a check relying on job report fail. #60587 (Max K.).

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT