ClickHouse/docs/changelogs/v24.8.1.2684-lts.md

100 KiB
Raw Blame History

sidebar_position sidebar_label
1 2024

2024 Changelog

ClickHouse release v24.8.1.2684-lts (161c62fd29) FIXME as compared to v24.8.1.1-new (3f8b27d7ac)

Backward Incompatible Change

  • clickhouse-client and clickhouse-local now default to multi-query mode (instead single-query mode). As an example, clickhouse-client -q "SELECT 1; SELECT 2" now works, whereas users previously had to add --multiquery (or -n). The --multiquery/-n switch became obsolete. INSERT queries in multi-query statements are treated specially based on their FORMAT clause: If the FORMAT is VALUES (the most common case), the end of the INSERT statement is represented by a trailing semicolon ; at the end of the query. For all other FORMATs (e.g. CSV or JSONEachRow), the end of the INSERT statement is represented by two newlines \n\n at the end of the query. #63898 (FFish).
  • In previous versions, it was possible to use an alternative syntax for LowCardinality data types by appending WithDictionary to the name of the data type. It was an initial working implementation, and it was never documented or exposed to the public. Now, it is deprecated. If you have used this syntax, you have to ALTER your tables and rename the data types to LowCardinality. #66842 (Alexey Milovidov).
  • Fix logical errors with storage Buffer used with distributed destination table. It's a backward incompatible change: queries using Buffer with a distributed destination table may stop working if the table appears more than once in the query (e.g., in a self-join). #67015 (vdimir).
  • In previous versions, calling functions for random distributions based on the Gamma function (such as Chi-Squared, Student, Fisher) with negative arguments close to zero led to a long computation or an infinite loop. In the new version, calling these functions with zero or negative arguments will produce an exception. This closes #67297. #67326 (Alexey Milovidov).
  • The system table text_log is enabled by default. This is fully compatible with previous versions, but you may notice subtly increased disk usage on the local disk (this system table takes a tiny amount of disk space). #67428 (Alexey Milovidov).
  • In previous versions, arrayWithConstant can be slow if asked to generate very large arrays. In the new version, it is limited to 1 GB per array. This closes #32754. #67741 (Alexey Milovidov).
  • Fix REPLACE modifier formatting (forbid omitting brackets). #67774 (Azat Khuzhin).
  • Backported in #68349: Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. #68132 (Kruglov Pavel).

New Feature

  • Add new experimental Kafka storage engine to store offsets in Keeper instead of relying on committing them to Kafka. #57625 (János Benjamin Antal).
  • Add new TimeSeries table engine: - by default: CREATE TABLE tbl ENGINE=TimeSeries - or with specifying engines of its internal tables:. #64183 (Vitaly Baranov).
  • Support more join strictnesses (LEFT/RIGHT SEMI/ANTI/ANY JOIN) with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y (see setting allow_experimental_join_condition). #64281 (lgbo).
  • Add _etag virtual column for S3 table engine. Fixes #65312. #65386 (skyoct).
  • This pull request introduces Hive-style partitioning for different engines (File, URL, S3, AzureBlobStorage, HDFS). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). #65997 (Yarik Briukhovetskyi).
  • Add function printf for spark compatiability. #66257 (李扬).
  • Backported in #68450: Implement new JSON data type. #66444 (Kruglov Pavel).
  • Add a new server setting: disable_insertion_and_mutation Set it to true. This node will deny all insertions and mutations(Alter table delete/update/drop partition). Include async insertion. #66519 (Xu Jia).
  • Add options restore_replace_external_engines_to_null and restore_replace_external_table_functions_to_null to replace external engines and table_engines to Null engine that can be useful for testing. It should work for RESTORE and explicit table creation. #66536 (Ilya Yatsishin).
  • Added support for reading MULTILINESTRING geometry in WKT format using function readWKTLineString. #67647 (Jacob Reckhard).
  • Add a new table function fuzzQuery. This function allows the modification of a given query string with random variations. Example: SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;. #67655 (pufit).
  • Support query DROP DETACHED PARTITION ALL to drop all detached partitions. #67885 (Duc Canh Le).
  • Added a tagging (namespace) mechanism for the query cache. The same queries with different tags are considered different by the query cache. Example: SELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'abc' and SELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'def' now create different query cache entries. #68235 (sakulali).

Performance Improvement

  • Use adaptive read task size calculation method (adaptive meaning it depends on read column sizes) for parallel replicas. #60377 (Nikita Taranov).
  • Store the plain_rewritable disk directory metadata in __meta layout, separately from the merge tree data in the object storage. Move the plain_rewritable disk to a flat directory structure. #65751 (Julia Kartseva).
  • Enable compile_expressions (JIT compiler for fragments of ordinary expressions) by default. This closes #51264 and #56386. #66486 (Alexey Milovidov).
  • Improve columns squashing for String/Array/Map/Variant/Dynamic types by reserving required memory in advance for all subcolumns. #67043 (Kruglov Pavel).
  • Speed up system flush logs, flush logs on shutdown. #67472 (Sema Checherinda).
  • Backported in #68496: Improved overall performance of merges by reducing the overhead of scheduling steps of merges. #68016 (Anton Popov).
  • Setting optimize_functions_to_subcolumns is enabled by default. #68053 (Anton Popov).

Improvement

  • ClickHouse server now supports new setting max_keep_alive_requests. For keep-alive HTTP connections to the server it works in tandem with keep_alive_timeout - if idle timeout not expired but there already more than max_keep_alive_requests requests done through the given connection - it will be closed by the server. #61793 (Nikita Taranov).
  • As in the new version, SOURCES are checked based on Table Engine logic, even grant table engine is disabled by default, if a source is not granted, a prompt of table engine would popup instead, which is misleading. #65419 (jsc0218).
  • Added statistics type count_min (count-min sketches) which provide selectivity estimations for equality predicates like col = 'val'. Supported data types are string, date, datatime and numeric types. #65521 (JackyWoo).
  • Do not pass logs for keeper explicitly in the image to allow overriding. #65564 (Azat Khuzhin).
  • Use Atomic database by default in clickhouse-local. Address items 1 and 5 from #50647. Closes #44817. #65860 (Alexey Milovidov).
  • Add the rows_before_aggregation_at_least statistic to the query response when rows_before_aggregation is enabled. This statistic represents the number of rows read before aggregation. In the context of a distributed query, when using the group by or max aggregation function without a limit, rows_before_aggregation_at_least can reflect the number of rows hit by the query. #66084 (morning-color).
  • Introduced use_same_password_for_base_backup settings for BACKUP and RESTORE queries, allowing to create and restore incremental backups to/from password protected archives. #66214 (Samuele).
  • Ignore async_load_databases for ATTACH query (previously it was possible for ATTACH to return before the tables had been attached). #66240 (Azat Khuzhin).
  • [Replicated]MergeTreeSink has to properly cancel its delayed_chunk on onCancel() method. #66279 (Sema Checherinda).
  • Added logs and metrics for rejected connections (where there are not enough resources). #66410 (Alexander Tokmakov).
  • Setting allow_experimental_analyzer is renamed to enable_analyzer. The old name is preserved in a form of an alias. #66438 (Nikita Mikhaylov).
  • Support true UUID type for MongoDB engine. #66671 (Azat Khuzhin).
  • Added a new MergeTree setting deduplicate_merge_projection_mode to control the projections during merges (for specific engines) and OPTIMIZE DEDUPLICATE query. Supported options: throw (throw an exception in case the projection is not fully supported for *MergeTree engine), drop (remove projection during merge if it can't be merged itself consistently) and rebuild (rebuild projection from scratch, which is a heavy operation). #66672 (jsc0218).
  • Add replication lag and recovery time metrics. #66703 (Miсhael Stetsyuk).
  • Add S3DiskNoKeyErrors metric. #66704 (Miсhael Stetsyuk).
  • Ensure COMMENT clause works for all table engines. #66832 (Joe Lynch).
  • Update the usage of error code BAD_ARGUMENTS and ILLEGAL_TYPE_OF_ARGUMENT by more accurate error codes when appropriate. #66851 (Yohann Jardin).
  • Function mapFromArrays now accepts Map(K, V) as first argument, for example: SELECT mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb']) now works and returns {('a',4):'aa',('b',4):'bb'}. Also, if the 1st argument is an Array, it can now also be of type Array(Nullable(T)) or Array(LowCardinality(Nullable(T))) as long as the actual array values are not NULL. #67103 (李扬).
  • Read configuration for clickhouse-local from ~/.clickhouse-local. #67135 (Azat Khuzhin).
  • Rename setting input_format_orc_read_use_writer_time_zone to input_format_orc_reader_timezone and allow the user to set the reader timezone. #67175 (kevinyhzou).
  • Decrease level of 'Socket is not connected' error when HTTP connection immediately reset by peer after connecting, close #34218. #67177 (vdimir).
  • Speed up tables removal for DROP DATABASE query, increased the default value for database_catalog_drop_table_concurrency to 16. #67228 (Nikita Mikhaylov).
  • Add ability to load dashboards for system.dashboards from config (once set, they overrides the default dashboards preset). #67232 (Azat Khuzhin).
  • The window functions in SQL are traditionally in snake case. ClickHouse uses camelCase, so new aliases denseRank() and percentRank() have been created. These new functions can be called the exact same as the original dense_rank() and percent_rank() functions. Both snake case and camelCase syntaxes remain usable. A new test for each of the functions has been added as well. This closes #67042 . #67334 (Peter Nguyen).
  • Autodetect configuration file format if is not .xml, .yml or .yaml. If the file begins with < it might be XML, otherwise it might be YAML. Non regular file just parse as XML such as PIPE: /dev/fd/X. #67391 (sakulali).
  • Functions formatDateTime and formatDateTimeInJodaSyntax now treat their format parameter as optional. If it is not specified, format strings %Y-%m-%d %H:%i:%s and yyyy-MM-dd HH:mm:ss are assumed. Example: SELECT parseDateTime('2021-01-04 23:12:34') now returns DateTime value 2021-01-04 23:12:34 (previously, this threw an exception). #67399 (Robert Schulze).
  • Automatically retry Keeper requests in KeeperMap if they happen because of timeout or connection loss. #67448 (Antonio Andelic).
  • Rework usage of custom table's disks. #67684 (Sema Checherinda).
  • Various improvements in the advanced dashboard. This closes #67697. This closes #63407. This closes #51129. This closes #61204. #67701 (Alexey Milovidov).
  • Avoid allocate too much capacity for array column while writing orc & some minor refactors to make code cleaner. Performance speeds up 15% for array column. #67879 (李扬).
  • Support OPTIMIZE query on Join table engine to reduce Join tables memory footprint. #67883 (Duc Canh Le).
  • Add replication lag and recovery time metrics. #67913 (Miсhael Stetsyuk).
  • Add '-no-pie' to aarch64 Linux builds to allow proper introspection and symbolizing of stacktraces after a ClickHouse restart. #67916 (filimonov).
  • Backported in #68481: Added profile events for merges and mutations for better introspection. #68015 (Anton Popov).
  • Fix settings/current_database in system.processes for async BACKUP/RESTORE. #68163 (Azat Khuzhin).
  • Remove unnecessary logs for MergeTree that doesn't support replication. #68238 (Daniil Ivanik).
  • Backported in #68430: Improve schema inference of date times. Now DateTime64 used only when date time has fractional part, otherwise regular DateTime is used. Inference of Date/DateTime is more strict now, especially when date_time_input_format='best_effort' to avoid inferring date times from strings in corner cases. #68382 (Kruglov Pavel).

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

  • Fixed Not-ready Set in some system tables when filtering using subqueries. #66018 (Michael Kolupaev).
  • Fixed reading of subcolumns after ALTER ADD COLUMN query. #66243 (Anton Popov).
  • Fix boolean literals in query sent to external database (for engines like PostgreSQL). #66282 (vdimir).
  • Fix formatting of query with aliased JOIN ON expression, e.g. ... JOIN t2 ON (x = y) AS e ORDER BY x should be formatted as ... JOIN t2 ON ((x = y) AS e) ORDER BY x. #66312 (vdimir).
  • Fix cluster() for inter-server secret (preserve initial user as before). #66364 (Azat Khuzhin).
  • Fix possible runtime error while converting Array field with nulls to Array(Variant). #66727 (Kruglov Pavel).
  • Fix for occasional deadlock in Context::getDDLWorker. #66843 (Alexander Gololobov).
  • Fix creating KeeperMap table after an incomplete drop. #66865 (Antonio Andelic).
  • Fix broken part error while restoring to a s3_plain_rewritable disk. #66881 (Vitaly Baranov).
  • In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. #66898 (alesapin).
  • Fix invalid format detection in schema inference that could lead to logical error Format {} doesn't support schema inference. #66899 (Kruglov Pavel).
  • Fix possible deadlock on query cancel with parallel replicas. #66905 (Nikita Taranov).
  • Forbid create as select even when database_replicated_allow_heavy_create is set. It was unconditionally forbidden in 23.12 and accidentally allowed under the setting in unreleased 24.7. #66980 (vdimir).
  • Reading from the numbers could wrongly throw an exception when the max_rows_to_read limit was set. This closes #66992. #66996 (Alexey Milovidov).
  • Add proper type conversion to lagInFrame and leadInFrame window functions - fixes msan test. #67091 (Yakov Olkhovskiy).
  • Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. #67122 (Kruglov Pavel).
  • TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. #67129 (Alexander Tokmakov).
  • Use a separate client context in clickhouse-local. #67133 (Vitaly Baranov).
  • Fix error Cannot convert column because it is non constant in source stream but must be constant in result. for a query that reads from the Merge table over the Distriburted table with one shard. #67146 (Nikolai Kochetov).
  • Correct behavior of ORDER BY all with disabled enable_order_by_all and parallel replicas (distributed queries as well). #67153 (Igor Nikonov).
  • Fix wrong usage of input_format_max_bytes_to_read_for_schema_inference in schema cache. #67157 (Kruglov Pavel).
  • Fix the memory leak for count distinct, when exception issued during group by single nullable key. #67171 (Jet He).
  • This closes #67156. This closes #66447. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. #67178 (Maksim Kita).
  • Fix crash in DistributedAsyncInsert when connection is empty. #67219 (Pablo Marcos).
  • Fix error Conversion from AggregateFunction(name, Type) to AggregateFunction(name, Nullable(Type)) is not supported. The bug was caused by the optimize_rewrite_aggregate_function_with_if optimization. Fixes #67112. #67229 (Nikolai Kochetov).
  • Fix hung query when using empty tuple as lhs of function IN. #67295 (Duc Canh Le).
  • Fix crash of uniq and uniqTheta with tuple() argument. Closes #67303. #67306 (flynn).
  • It was possible to create a very deep nested JSON data that triggered stack overflow while skipping unknown fields. This closes #67292. #67324 (Alexey Milovidov).
  • Fix attaching ReplicatedMergeTree table after exception during startup. #67360 (Antonio Andelic).
  • Fix segfault caused by incorrectly detaching from thread group in Aggregator. #67385 (Antonio Andelic).
  • Fix one more case when a non-deterministic function is specified in PK. #67395 (Nikolai Kochetov).
  • Fixed bloom_filter index breaking queries with mildly weird conditions like (k=2)=(k=2) or has([1,2,3], k). #67423 (Michael Kolupaev).
  • Correctly parse file name/URI containing :: if it's not an archive. #67433 (Antonio Andelic).
  • Fix wait for tasks in ~WriteBufferFromS3 in case WriteBuffer was cancelled. #67459 (Kseniia Sumarokova).
  • Protect temporary part directories from removing during RESTORE. #67491 (Vitaly Baranov).
  • Fix execution of nested short-circuit functions. #67520 (Kruglov Pavel).
  • Fixes #66026. Avoid unresolved table function arguments traversal in ReplaceTableNodeToDummyVisitor. #67522 (Dmitry Novik).
  • Fix Logical error: Expected the argument №N of type T to have X rows, but it has 0. The error could happen in a remote query with constant expression in GROUP BY (with a new analyzer). #67536 (Nikolai Kochetov).
  • Fix join on tuple with NULLs: Some queries with the new analyzer and NULL inside the tuple in the JOIN ON section returned incorrect results. #67538 (vdimir).
  • Fix redundant reschedule of FileCache::freeSpaceRatioKeepingThreadFunc() in case of full non-evictable cache. #67540 (Kseniia Sumarokova).
  • Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. #67554 (János Benjamin Antal).
  • Fix for function toStartOfWeek which returned the wrong result with a small DateTime64 value. #67558 (Yarik Briukhovetskyi).
  • Fix creation of view with recursive CTE. #67587 (Yakov Olkhovskiy).
  • Fix Logical error: 'file_offset_of_buffer_end <= read_until_position' in filesystem cache. Closes #57508. #67623 (Kseniia Sumarokova).
  • Fixes #62282. Removed the call to convertFieldToString() and added datatype specific serialization code. Parameterized view substitution was broken for multiple datatypes when parameter value was a function or expression returning datatype instance. #67654 (Shankar).
  • Fix crash on percent_rank. percent_rank's default frame type is changed to range unbounded preceding and unbounded following. IWindowFunction's default window frame is considered and now window functions without window frame definition in sql can be put into different WindowTransfomers properly. #67661 (lgbo).
  • Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. #67665 (Antonio Andelic).
  • Fix possible logical error "Unexpected return type from if" with experimental Variant type and enabled setting use_variant_as_common_type in function if with Tuples and Maps. #67687 (Kruglov Pavel).
  • Due to a bug in Linux Kernel, a query can hung in TimerDescriptor::drain. This closes #37686. #67702 (Alexey Milovidov).
  • Fix completion of RESTORE ON CLUSTER command. #67720 (Vitaly Baranov).
  • Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading. #67751 (Azat Khuzhin).
  • Fix potential stack overflow in JSONMergePatch function. Renamed this function from jsonMergePatch to JSONMergePatch because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes #67304. #67756 (Alexey Milovidov).
  • Queries like SELECT count() FROM t WHERE cast(c = 1 or c = 9999 AS Bool) SETTINGS use_skip_indexes=1 with bloom filter indexes on c now work correctly. #67781 (jsc0218).
  • Fix wrong aggregation result in some queries with aggregation without keys and filter, close #67419. #67804 (vdimir).
  • Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. #67911 (Kruglov Pavel).
  • Fix DateTime64 parsing after constant folding in distributed queries, close #66773. #67920 (vdimir).
  • Fix wrong count() result when there is non-deterministic function in predicate. #67922 (János Benjamin Antal).
  • Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. #67963 (Robert Schulze).
  • Now ClickHouse doesn't consider part as broken if projection doesn't exist on disk but exists in checksums.txt. #68003 (alesapin).
  • Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. #68052 (Anton Popov).
  • Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. #68098 (Salvatore Mesoraca).
  • Removes an incorrect optimization to remove sorting in subqueries that use OFFSET. Fixes #67906. #68099 (Graham Campbell).
  • Attempt to fix Block structure mismatch in AggregatingStep stream: different types for aggregate projection optimization. #68107 (Nikolai Kochetov).
  • Backported in #68343: Try fix postgres crash when query is cancelled. #68288 (Kseniia Sumarokova).
  • Backported in #68400: Fix missing sync replica mode in query SYSTEM SYNC REPLICA. #68326 (Duc Canh Le).

Build/Testing/Packaging Improvement

  • Improved test_storage_s3 tests: increased s3_max_single_read_retries for read from "unstable" s3 source and allowed all tests to run multiple times in a row. #66896 (Ilya Yatsishin).
  • Integration tests flaky check will not run each test case multiple times to find more issues in tests and make them more reliable. It is using pytest-repeat library to run test case multiple times for the same environment. It is important to cleanup tables and other entities in the end of a test case to pass. Repeat works much faster than several pytest runs as it starts necessary containers only once. #66986 (Ilya Yatsishin).
  • Allow to use CLion with ClickHouse. In previous versions, CLion freezed for a minute on every keypress. This closes #66994. #66995 (Alexey Milovidov).
  • Getauxval: avoid crash under sanitizer re-exec due to high aslr entropy. #67081 (Raúl Marín).
  • Some parts of client code are extracted to a single file and highest possible level optimization is applied to them even for debug builds. This closes: #65745. #67215 (Nikita Mikhaylov).

NO CL CATEGORY

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT