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

99 KiB

sidebar_position sidebar_label
1 2024

2024 Changelog

ClickHouse release v24.3.1.2672-lts (2c5c589a88) FIXME as compared to v24.2.1.2248-stable (891689a415)

Backward Incompatible Change

  • Don't allow to set max_parallel_replicas to 0 as it doesn't make sense. Setting it to 0 could lead to unexpected logical errors. Closes #60140. #60430 (Kruglov Pavel).
  • Change the column name from duration_ms to duration_microseconds in the system.zookeeper table to reflect the reality that the duration is in the microsecond resolution. #60774 (Duc Canh Le).
  • Reject incoming INSERT queries in case when query-level settings async_insert and deduplicate_blocks_in_dependent_materialized_views are enabled together. This behaviour is controlled by a setting throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert and enabled by default. This is a continuation of https://github.com/ClickHouse/ClickHouse/pull/59699 needed to unblock https://github.com/ClickHouse/ClickHouse/pull/59915. #60888 (Nikita Mikhaylov).
  • Utility clickhouse-copier is moved to a separate repository on GitHub: https://github.com/ClickHouse/copier. It is no longer included in the bundle but is still available as a separate download. This closes: #60734 This closes: #60540 This closes: #60250 This closes: #52917 This closes: #51140 This closes: #47517 This closes: #47189 This closes: #46598 This closes: #40257 This closes: #36504 This closes: #35485 This closes: #33702 This closes: #26702 ### Documentation entry for user-facing changes. #61058 (Nikita Mikhaylov).
  • To increase compatibility with MySQL, function locate now accepts arguments (needle, haystack[, start_pos]) by default. The previous behavior (haystack, needle, [, start_pos]) can be restored by setting function_locate_has_mysql_compatible_argument_order = 0. #61092 (Robert Schulze).
  • The obsolete in-memory data parts have been deprecated since version 23.5 and have not been supported since version 23.10. Now the remaining code is removed. Continuation of #55186 and #45409. It is unlikely that you have used in-memory data parts because they were available only before version 23.5 and only when you enabled them manually by specifying the corresponding SETTINGS for a MergeTree table. To check if you have in-memory data parts, run the following query: SELECT part_type, count() FROM system.parts GROUP BY part_type ORDER BY part_type. To disable the usage of in-memory data parts, do ALTER TABLE ... MODIFY SETTING min_bytes_for_compact_part = DEFAULT, min_rows_for_compact_part = DEFAULT. Before upgrading from old ClickHouse releases, first check that you don't have in-memory data parts. If there are in-memory data parts, disable them first, then wait while there are no in-memory data parts and continue the upgrade. #61127 (Alexey Milovidov).
  • Forbid SimpleAggregateFunction in ORDER BY of MergeTree tables (like AggregateFunction is forbidden, but they are forbidden because they are not comparable) by default (use allow_suspicious_primary_key to allow them). #61399 (Azat Khuzhin).
  • ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. This is controlled by the settings, output_format_parquet_string_as_string, output_format_orc_string_as_string, output_format_arrow_string_as_string. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases. Parquet/ORC/Arrow supports many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools lack support for the faster lz4 compression method, that's why we set zstd by default. This is controlled by the settings output_format_parquet_compression_method, output_format_orc_compression_method, and output_format_arrow_compression_method. We changed the default to zstd for Parquet and ORC, but not Arrow (it is emphasized for low-level usages). #61817 (Alexey Milovidov).
  • In the new ClickHouse version, the functions geoDistance, greatCircleDistance, and greatCircleAngle will use 64-bit double precision floating point data type for internal calculations and return type if all the arguments are Float64. This closes #58476. In previous versions, the function always used Float32. You can switch to the old behavior by setting geo_distance_returns_float64_on_float64_arguments to false or setting compatibility to 24.2 or earlier. #61848 (Alexey Milovidov).

New Feature

  • Topk/topkweighed support mode, which return count of values and it's error. #54508 (UnamedRus).
  • Add generate_series as a table function. This function generates table with an arithmetic progression with natural numbers. #59390 (Daniil Ivanik).
  • Support reading and writing backups as tar archives. #59535 (josh-hildred).
  • Implemented support for S3Express buckets. #59965 (Nikita Taranov).
  • Allow to attach parts from a different disk * attach partition from the table on other disks using copy instead of hard link (such as instant table) * attach partition using copy when the hard link fails even on the same disk. #60112 (Unalian).
  • Added function toMillisecond which returns the millisecond component for values of typeDateTime or DateTime64. #60281 (Shaun Struwig).
  • Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. #60420 (豪肥肥).
  • Add four properties to the StorageMemory (memory-engine) min_bytes_to_keep, max_bytes_to_keep, min_rows_to_keep and max_rows_to_keep - Add tests to reflect new changes - Update memory.md documentation - Add table context property to MemorySink to enable access to table parameter bounds. #60612 (Jake Bamrah).
  • Added function toMillisecond which returns the millisecond component for values of typeDateTime or DateTime64. #60649 (Robert Schulze).
  • Separate limits on number of waiting and executing queries. Added new server setting max_waiting_queries that limits the number of queries waiting due to async_load_databases. Existing limits on number of executing queries no longer count waiting queries. #61053 (Sergei Trifonov).
  • Add support for ATTACH PARTITION ALL. #61107 (Kirill Nikiforov).
  • Add a new function, getClientHTTPHeader. This closes #54665. Co-authored with @lingtaolf. #61820 (Alexey Milovidov).

Performance Improvement

  • Improve the performance of serialized aggregation method when involving multiple [nullable] columns. This is a general version of #51399 that doesn't compromise on abstraction integrity. #55809 (Amos Bird).
  • Lazy build join output to improve performance of ALL join. #58278 (LiuNeng).
  • Improvements to aggregate functions ArgMin / ArgMax / any / anyLast / anyHeavy, as well as ORDER BY {u8/u16/u32/u64/i8/i16/u32/i64) LIMIT 1 queries. #58640 (Raúl Marín).
  • Trivial optimize on column filter. Avoid those filter columns whoes underlying data type is not number being filtered with result_size_hint = -1. Peak memory can be reduced to 44% of the original in some cases. #59698 (李扬).
  • If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns with the value 0.9 by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. #60255 (Alexey Milovidov).
  • Execute multiIf function columnarly when result_type's underlying type is number. #60384 (李扬).
  • Faster (almost 2x) mutexes (was slower due to ThreadFuzzer). #60823 (Azat Khuzhin).
  • Move connection drain from prepare to work, and drain multiple connections in parallel. #60845 (lizhuoyu5).
  • Optimize insertManyFrom of nullable number or nullable string. #60846 (李扬).
  • Optimized function dotProduct to omit unnecessary and expensive memory copies. #60928 (Robert Schulze).
  • Operations with the filesystem cache will suffer less from the lock contention. #61066 (Alexey Milovidov).
  • Optimize ColumnString::replicate and prevent memcpySmallAllowReadWriteOverflow15Impl from being optimized to built-in memcpy. Close #61074. ColumnString::replicate speeds up by 2.46x on x86-64. #61075 (李扬).
  • 30x faster printing for 256-bit integers. #61100 (Raúl Marín).
  • If a query with a syntax error contained COLUMNS matcher with a regular expression, the regular expression was compiled each time during the parser's backtracking, instead of being compiled once. This was a fundamental error. The compiled regexp was put to AST. But the letter A in AST means "abstract" which means it should not contain heavyweight objects. Parts of AST can be created and discarded during parsing, including a large number of backtracking. This leads to slowness on the parsing side and consequently allows DoS by a readonly user. But the main problem is that it prevents progress in fuzzers. #61543 (Alexey Milovidov).
  • Add a new analyzer pass to optimize in single value. #61564 (LiuNeng).

Improvement

  • While running the MODIFY COLUMN query for materialized views, check the inner table's structure to ensure every column exists. #47427 (sunny).
  • Added table system.keywords which contains all the keywords from parser. Mostly needed and will be used for better fuzzing and syntax highlighting. #51808 (Nikita Mikhaylov).
  • Ordinary database engine is deprecated. You will receive a warning in clickhouse-client if your server is using it. This closes #52229. #56942 (shabroo).
  • All zero copy locks related to a table have to be dropped when the table is dropped. The directory which contains these locks has to be removed also. #57575 (Sema Checherinda).
  • Allow declaring enum in external table structure. #57857 (Duc Canh Le).
  • Consider lightweight deleted rows when selecting parts to merge. #58223 (Zhuo Qiu).
  • This PR makes http/https connections reusable for all uses cases. Even when response is 3xx or 4xx. #58845 (Sema Checherinda).
  • Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. #59016 (Nikita Mikhaylov).
  • Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like _part_offset. #59033 (Amos Bird).
  • Add ability to skip read-only replicas for INSERT into Distributed engine (Controlled with distributed_insert_skip_read_only_replicas setting, by default OFF - backward compatible). #59176 (Azat Khuzhin).
  • Instead using a constant key, now object storage generates key for determining remove objects capability. #59495 (Sema Checherinda).
  • Add positional pread in libhdfs3. If you want to call positional read in libhdfs3, use the hdfsPread function in hdfs.h as follows. tSize hdfsPread(hdfsFS fs, hdfsFile file, void * buffer, tSize length, tOffset position);. #59624 (M1eyu).
  • Add asynchronous WriteBuffer for AzureBlobStorage similar to S3. #59929 (SmitaRKulkarni).
  • Allow "local" as object storage type instead of "local_blob_storage". #60165 (Kseniia Sumarokova).
  • Improved overall usability of virtual columns. Now it is allowed to use virtual columns in PREWHERE (it's worthwhile for non-const virtual columns like _part_offset). Now a builtin documentation is available for virtual columns as a comment of column in DESCRIBE query with enabled setting describe_include_virtual_columns. #60205 (Anton Popov).
  • Parallel flush of pending INSERT blocks of Distributed engine on DETACH/server shutdown and SYSTEM FLUSH DISTRIBUTED (Parallelism will work only if you have multi disk policy for table (like everything in Distributed engine right now)). #60225 (Azat Khuzhin).
  • Filter setting is improper in joinRightColumnsSwitchNullability, resolve #59625. #60259 (lgbo).
  • Add a setting to force read-through cache for merges. #60308 (Kseniia Sumarokova).
  • Issue #57598 mentions a variant behaviour regarding transaction handling. An issued COMMIT/ROLLBACK when no transaction is active is reported as an error contrary to MySQL behaviour. #60338 (PapaToemmsn).
  • Added none_only_active mode for distributed_ddl_output_mode setting. #60340 (Alexander Tokmakov).
  • Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make / redirect to the Play UI. #60390 (Alexey Milovidov).
  • The advanced dashboard has slightly better colors for multi-line graphs. #60391 (Alexey Milovidov).
  • Function substring now has a new alias byteSlice. #60494 (Robert Schulze).
  • Renamed server setting dns_cache_max_size to dns_cache_max_entries to reduce ambiguity. #60500 (Kirill Nikiforov).
  • SHOW INDEX | INDEXES | INDICES | KEYS no longer sorts by the primary key columns (which was unintuitive). #60514 (Robert Schulze).
  • Keeper improvement: abort during startup if an invalid snapshot is detected to avoid data loss. #60537 (Antonio Andelic).
  • Added MergeTree read split ranges into intersecting and non intersecting fault injection using merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_fault_probability setting. #60548 (Maksim Kita).
  • The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. #60692 (Alexey Milovidov).
  • String types and Enums can be used in the same context, such as: arrays, UNION queries, conditional expressions. This closes #60726. #60727 (Alexey Milovidov).
  • Update tzdata to 2024a. #60768 (Raúl Marín).
  • Support files without format extension in Filesystem database. #60795 (Kruglov Pavel).
  • Keeper improvement: support leadership_expiry_ms in Keeper's settings. #60806 (Brokenice0415).
  • Always infer exponential numbers in JSON formats regardless of the setting input_format_try_infer_exponent_floats. Add setting input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects that allows to use String type for ambiguous paths instead of an exception during named Tuples inference from JSON objects. #60808 (Kruglov Pavel).
  • Add support for START TRANSACTION syntax typically used in MySQL syntax, resolving https://github.com/ClickHouse/ClickHouse/discussions/60865. #60886 (Zach Naimon).
  • Add a flag for SMJ to treat null as biggest/smallest. So the behavior can be compitable with other SQL systems, like Apache Spark. #60896 (loudongfeng).
  • Clickhouse version has been added to docker labels. Closes #54224. #60949 (Nikolay Monkov).
  • Add a setting parallel_replicas_allow_in_with_subquery = 1 which allows subqueries for IN work with parallel replicas. #60950 (Nikolai Kochetov).
  • DNSResolver shuffles set of resolved IPs. #60965 (Sema Checherinda).
  • Support detect output format by file exctension in clickhouse-client and clickhouse-local. #61036 (豪肥肥).
  • Check memory limit update periodically. #61049 (Han Fei).
  • Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. #61096 (Azat Khuzhin).
  • Add the function toUInt128OrZero, which was missed by mistake (the mistake is related to https://github.com/ClickHouse/ClickHouse/pull/945). The compatibility aliases FROM_UNIXTIME and DATE_FORMAT (they are not ClickHouse-native and only exist for MySQL compatibility) have been made case insensitive, as expected for SQL-compatibility aliases. #61114 (Alexey Milovidov).
  • Improvements for the access checks, allowing to revoke of unpossessed rights in case the target user doesn't have the revoking grants either. Example: ```sql GRANT SELECT ON . TO user1; REVOKE SELECT ON system.* FROM user1;. #61115 (pufit).
  • Fix an error in previeous opt: https://github.com/ClickHouse/ClickHouse/pull/59698: remove break to make sure the first filtered column has minimum size cc @jsc0218. #61145 (李扬).
  • Fix has() function with Nullable column (fixes #60214). #61249 (Mikhail Koviazin).
  • Now it's possible to specify attribute merge="true" in config substitutions for subtrees <include from_zk="/path" merge="true">. In case this attribute specified, clickhouse will merge subtree with existing configuration, otherwise default behavior is append new content to configuration. #61299 (alesapin).
  • Add async metrics for virtual memory mappings: VMMaxMapCount & VMNumMaps. Closes #60662. #61354 (Tuan Pham Anh).
  • Use temporary_files_codec setting in all places where we create temporary data, for example external memory sorting and external memory GROUP BY. Before it worked only in partial_merge JOIN algorithm. #61456 (Maksim Kita).
  • Remove duplicated check containing_part.empty(), It's already being checked here: 1296dac3c7/src/Storages/MergeTree/MergeTreeData.cpp (L6141). #61467 (William Schoeffel).
  • Add a new setting max_parser_backtracks which allows to limit the complexity of query parsing. #61502 (Alexey Milovidov).
  • Support parallel reading for azure blob storage. #61503 (SmitaRKulkarni).
  • Less contention during dynamic resize of filesystem cache. #61524 (Kseniia Sumarokova).
  • Disallow sharded mode of StorageS3 queue, because it will be rewritten. #61537 (Kseniia Sumarokova).
  • Fixed typo: from use_leagcy_max_level to use_legacy_max_level. #61545 (William Schoeffel).
  • Remove some duplicate entries in blob_storage_log. #61622 (YenchangChan).
  • Enable allow_experimental_analyzer setting by default. #61652 (Dmitry Novik).
  • Added current_user function as a compatibility alias for MySQL. #61770 (Yarik Briukhovetskyi).
  • Use managed identity for backups IO when using Azure Blob Storage. Add a setting to prevent ClickHouse from attempting to create a non-existent container, which requires permissions at the storage account level. #61785 (Daniel Pozo Escalona).
  • Enable output_format_pretty_row_numbers by default. It is better for usability. #61791 (Alexey Milovidov).
  • In the previous version, some numbers in Pretty formats were not pretty enough. #61794 (Alexey Milovidov).
  • A long value in Pretty formats won't be cut if it is the single value in the resultset, such as in the result of the SHOW CREATE TABLE query. #61795 (Alexey Milovidov).
  • Similarly to clickhouse-local, clickhouse-client will accept the --output-format option as a synonym to the --format option. This closes #59848. #61797 (Alexey Milovidov).
  • If stdout is a terminal and the output format is not specified, clickhouse-client and similar tools will use PrettyCompact by default, similarly to the interactive mode. clickhouse-client and clickhouse-local will handle command line arguments for input and output formats in a unified fashion. This closes #61272. #61800 (Alexey Milovidov).
  • Underscore digit groups in Pretty formats for better readability. This is controlled by a new setting, output_format_pretty_highlight_digit_groups. #61802 (Alexey Milovidov).
  • Add ability to override initial INSERT SETTINGS via SYSTEM FLUSH DISTRIBUTED. #61832 (Azat Khuzhin).
  • Fixed grammar from "a" to "the" in the warning message. There is only one Atomic engine, so it should be "to the new Atomic engine" instead of "to a new Atomic engine". #61952 (shabroo).

Build/Testing/Packaging Improvement

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

  • Fix function execution over const and LowCardinality with GROUP BY const for analyzer. #59986 (Azat Khuzhin).
  • Fix finished_mutations_to_keep=0 for MergeTree (as docs says 0 is to keep everything). #60031 (Azat Khuzhin).
  • Fixed a bug in parallel optimization for queries with FINAL, which could give an incorrect result in rare cases. #60041 (Maksim Kita).
  • Updated to not include account_name in endpoint if flag endpoint_contains_account_name is set and fixed issue with empty container name. #60251 (SmitaRKulkarni).
  • Fix LRUResource Cache implementation that can be triggered by incorrect component usage. Error can't be triggered with current ClickHouse usage. close #60122. #60262 (shanfengp).
  • Force reanalysis of the query if parallel replicas isn't supported in a subquery. #60362 (Raúl Marín).
  • Fix usage of plain metadata type for new disks configuration option. #60396 (Kseniia Sumarokova).
  • Fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike. #60451 (Kruglov Pavel).
  • Avoid calculation of scalar subqueries for CREATE TABLE. Fixes #59795 and #59930. Attempt to re-implement https://github.com/ClickHouse/ClickHouse/pull/57855. #60464 (Nikolai Kochetov).
  • Fix deadlock in parallel parsing when lots of rows are skipped due to errors. #60516 (Kruglov Pavel).
  • Fix the issue of max_query_size for KQL compound operator like mv-expand. Related to #59626. #60534 (Yong Wang).
  • Keeper fix: add timeouts when waiting for commit logs. Keeper could get stuck if the log successfully gets replicated but never committed. #60544 (Antonio Andelic).
  • Reduce the number of read rows from system.numbers. Fixes #59418. #60546 (JackyWoo).
  • Don't output number tips for date types. #60577 (Raúl Marín).
  • Fix unexpected result during reading from tables with virtual columns when filter contains non-deterministic functions. Closes #61106. #60586 (Kruglov Pavel).
  • Fix logical error on bad compatibility setting value type. Closes #60590. #60596 (Kruglov Pavel).
  • Fixed potentially inconsistent aggregate function states in mixed x86-64 / ARM clusters. #60610 (Harry Lee).
  • Isolates the ClickHouse binary from any panics in prqlc. #60615 (Maximilian Roos).
  • Fixing bug where intDiv with decimal and date/datetime as arguments leads to crash. Closes #60653. #60672 (Yarik Briukhovetskyi).
  • Fix bug when attempt to 'ALTER TABLE ... MODIFY QUERY' with CTE ends up with "Table [CTE] does not exist" exception (Code: 60). #60682 (Yakov Olkhovskiy).
  • Fix system.parts for non-Atomic/Ordinary database engine (i.e. Memory - major user is clickhouse-local). #60689 (Azat Khuzhin).
  • Fix "Invalid storage definition in metadata file" for parameterized views. #60708 (Azat Khuzhin).
  • Fix buffer overflow that can happen if the attacker asks the HTTP server to decompress data with a composition of codecs and size triggering numeric overflow. Fix buffer overflow that can happen inside codec NONE on wrong input data. This was submitted by TIANGONG research team through our Bug Bounty program. #60731 (Alexey Milovidov).
  • Functions for SQL/JSON were able to read uninitialized memory. This closes #60017. Found by Fuzzer. #60738 (Alexey Milovidov).
  • Remove wrong sanitize checking in aggregate function quantileGK: sampled_len in ApproxSampler is not guaranteed to be less than default_compress_threshold. default_compress_threshold is a just soft limitation while executing ApproxSampler::insert. cc @Algunenano. This issue was reproduced in https://github.com/oap-project/gluten/pull/4829. #60740 (李扬).
  • Fix the issue causing undesired deduplication on insert-select queries passing a custom insert_deduplication_token. The change sets streams to 1 in those cases to prevent the issue from happening at the expense of ignoring max_insert_threads > 1. #60745 (Jordi Villar).
  • Do not set aws custom metadata x-amz-meta-* headers on UploadPart & CompleteMultipartUpload calls. #60748 (Francisco J. Jurado Moreno).
  • One more fix for toStartOfInterval returning wrong result for interval smaller than second. #60763 (Andrey Zvonov).
  • Fix crash in arrayEnumerateRanked. #60764 (Raúl Marín).
  • Fix crash when using input() in INSERT SELECT JOIN. Closes #60035. #60765 (Kruglov Pavel).
  • Fix crash when allow_experimental_analyzer setting value is changed in the subqueries. #60770 (Dmitry Novik).
  • Avoid segfault if too many keys are skipped when reading from S3. #60849 (Antonio Andelic).
  • Fix possible stuck on error while reloading dictionary with SHARDS. #60926 (vdimir).
  • Fix async RESTORE with Replicated database. #60934 (Antonio Andelic).
  • Fix csv write tuple in a wrong format and can not read it. #60994 (shuai.xu).
  • Fixed deadlock in async inserts to Log tables via native protocol. #61055 (Anton Popov).
  • Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary that could lead to nullptr dereference on bad column types in FunctionsConversion. Closes #56661. #61196 (Kruglov Pavel).
  • Fix multiple bugs in groupArraySorted. #61203 (Raúl Marín).
  • Keeper: fix runtime reconfig for standalone binary. #61233 (Antonio Andelic).
  • Fix usage of session_token in S3 engine. Fixes https://github.com/ClickHouse/ClickHouse/pull/57850#issuecomment-1966404710. #61234 (Kruglov Pavel).
  • Fix possible incorrect result of aggregate function uniqExact. #61257 (Anton Popov).
  • Fix bugs in show database. #61269 (Raúl Marín).
  • Fix possible LOGICAL_ERROR in case storage with RabbitMQ engine has unsupported MATERIALIZED|ALIAS|DEFAULT columns. #61320 (vdimir).
  • This PR fixes CREATE OR REPLACE DICTIONARY with lazy_load turned off. #61356 (Vitaly Baranov).
  • Fix possible crash in Object('json') data type parsing array with nulls. #61364 (vdimir).
  • Fix the ATTACH query with the ON CLUSTER clause when the database does not exist on the initiator node. Closes #55009. #61365 (Nikolay Degterinsky).
  • Fixed possible wrong result of aggregation with nullable keys. #61393 (Anton Popov).
  • ActionsDAG::split can't make sure that "Execution of first then second parts on block is equivalent to execution of initial DAG.". #61458 (Raúl Marín).
  • Fix finishing a failed RESTORE. #61466 (Vitaly Baranov).
  • Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings. #61468 (Raúl Marín).
  • Fix deadlock during restore database execution if restore_threads was set to 1. #61475 (Nikita Taranov).
  • Fix incorrect results when filtering system.parts or system.parts_columns using UUID. #61479 (Dan Wu).
  • Fix the ALTER QUERY MODIFY SQL SECURITY queries to override the table's DDL correctly. #61480 (pufit).
  • The experimental "window view" feature (it is disabled by default), which should not be used in production, could lead to a crash. Issue was identified by YohannJardin via Bugcrowd program. #61526 (Alexey Milovidov).
  • Fix repeat with non-native integers (e.g. UInt256). #61527 (Antonio Andelic).
  • Fix clickhouse-client -s argument, it was broken by defining it two times. #61530 (Mikhail f. Shiryaev).
  • Fix too high part level reported in #58558 by resetting MergeTree part levels upon attach from disk just like ReplicatedMergeTree does. #61536 (Arthur Passos).
  • Fix crash in arrayPartialReverseSort. #61539 (Raúl Marín).
  • Fix string search with constant start position which previously could lead to memory corruption. #61547 (Antonio Andelic).
  • Fix the issue where the function addDays (and similar functions) reports an error when the first parameter is DateTime64. #61561 (Shuai li).
  • Disallow LowCardinality type for the column containing JSON input in the JSONExtract function. #61617 (Julia Kartseva).
  • Add parts to system.part_log when created using async insert with deduplication. #61620 (Antonio Andelic).
  • Fix Not-ready Set error while reading from system.parts (with IN subquery). Was introduced in #60510. #61666 (Nikolai Kochetov).
  • Don't allow the same expression in ORDER BY with and without WITH FILL. Such invalid expression could lead to logical error Invalid number of rows in Chunk. #61667 (Kruglov Pavel).
  • Fixed Entry actual part isn't empty yet. This is a bug. (LOGICAL_ERROR) that might happen in rare cases after executing REPLACE PARTITION, MOVE PARTITION TO TABLE or ATTACH PARTITION FROM. #61675 (Alexander Tokmakov).
  • Fix columns after executing ALTER TABLE MODIFY QUERY for a materialized view with internal table. A materialized view must have the same columns as its internal table if any, however MODIFY QUERY could break that rule before this PR causing the materialized view to be inconsistent. #61734 (Vitaly Baranov).
  • Fix crash in multiSearchAllPositionsCaseInsensitiveUTF8 when specifying incorrect UTF-8 sequence. Example: #61714. #61749 (pufit).
  • Fix RANGE frame is not supported for Nullable columns. ``` SELECT number, sum(number) OVER (ORDER BY number ASC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) AS sum FROM values('number Nullable(Int8)', 1, 1, 2, 3, NULL). #61766 (YuanLiu).
  • Fix incorrect results when filtering system.parts or system.parts_columns using UUID. #61779 (János Benjamin Antal).

CI Fix or Improvement (changelog entry is not required)

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT

Packaging Improvement

  • Adding the --now option to enable and start service automatically when installing the database server completely. #60656 (Chun-Sheng, Li).