ClickHouse/docs/changelogs/v22.7.1.2484-stable.md
2024-05-23 13:54:45 +02:00

82 KiB

sidebar_position sidebar_label
1 2022

2022 Changelog

ClickHouse release v22.7.1.2484-stable (f4f05ec786) FIXME as compared to v22.6.1.1985-stable (7000c4e003)

Backward Incompatible Change

  • Enable setting enable_positional_arguments by default. It allows queries like SELECT ... ORDER BY 1, 2 where 1, 2 are the references to the select clause. If you need to return the old behavior, disable this setting. #38204 (Alexey Milovidov).
  • Ordinary database engine and old storage definition syntax for *MergeTree tables are deprecated. By default it's not possible to create new ones. If system database has Ordinary engine it will be automatically converted to Atomic on server startup. There are settings to keep old behavior (allow_deprecated_database_ordinary and allow_deprecated_syntax_for_merge_tree), but these settings may be removed in future releases. #38335 (Alexander Tokmakov).
    • Force rewriting comma join to inner by default (set default value cross_to_inner_join_rewrite = 2). To have old behavior set cross_to_inner_join_rewrite = 1. #39326 (Vladimir C).
  • Disable format_csv_allow_single_quotes by default. #37096. #39423 (Kruglov Pavel).

New Feature

  • Add new direct join algorithm for RocksDB, ref #33582. #35363 (Vladimir C).
  • Add a setting zstd_window_log_max to configure max memory usage on zstd decoding when importing external files. Closes #35693. #37015 (wuxiaobai24).
  • Implement NatsStorage - table engine, which allows to pub/sub to NATS. Closes #32388. #37171 (tchepavel).
  • Implement table function MongoDB. Allow writes into MongoDB storage / table function. #37213 (aaapetrenko).
  • clickhouse-keeper new feature: add support for real-time digest calculation and verification. #37555 (Antonio Andelic).
  • In #17202 was reported that host_regexp was being tested against only one of the possible PTR responses. This PR makes the necessary changes so that host_regexp is applied against all possible PTR responses and validate if any matches. #37827 (Arthur Passos).
  • Support hadoop secure rpc transfer(hadoop.rpc.protection=privacy and hadoop.rpc.protection=integrity). #37852 (Peng Liu).
  • Add struct type support in StorageHive. #38118 (lgbo).
  • Added Base58 encoding/decoding. #38159 (Andrey Zvonov).
  • Add chart visualization to Play UI. #38197 (Alexey Milovidov).
  • support alter command on StorageHive table. #38214 (lgbo).
  • Added CREATE TABLE ... EMPTY AS SELECT query. It automatically deduces table structure from the SELECT query, but does not fill the table after creation. Resolves #38049. #38272 (Alexander Tokmakov).
  • Adds new setting implicit_transaction to run standalone queries inside a transaction. It handles both creation and closing (via COMMIT if the query succeeded or ROLLBACK if it didn't) of the transaction automatically. #38344 (Raúl Marín).
  • Allow trailing comma in columns list. closes #38425. #38440 (chen).
  • Compress clickhouse into self-extracting executable (path programs/self-extracting). New build target 'self-extracting' is added. #38447 (Yakov Olkhovskiy).
  • Introduced settings additional_table_filters. Using this setting, you can specify additional filtering condition for a table which will be applied directly after reading. Example: select number, x, y from (select number from system.numbers limit 5) f any left join (select x, y from table_1) s on f.number = s.x settings additional_table_filters={'system.numbers : 'number != 3', 'table_1' : 'x != 2'}. Introduced setting additional_result_filter which specifies additional filtering condition for query result. Closes #37918. #38475 (Nikolai Kochetov).
  • Add SQLInsert output format. Closes #38441. #38477 (Kruglov Pavel).
  • Downloadable clickhouse executable is compressed self-extracting. #38653 (Yakov Olkhovskiy).
  • Support isNullable function. This function checks whether it's argument is nullable and return true(1) or false(0). Closes #38611. #38841 (lokax).
  • Add functions translate(string, from_string, to_string) and translateUTF8(string, from_string, to_string). #38935 (Nikolay Degterinsky).
  • Add compatibility setting and system.settings_changes system table that contains information about changes in settings through ClickHouse versions. Closes #35972. #38957 (Kruglov Pavel).
  • Add the 3rd parameter to the tupleElement function and return it if tuple doesn't have a member. Only works if the 2nd parameter is of type String. Closes #38872. #38989 (lokax).
  • Support parseTimedelta function. It can be used like ```sql # ' ', ';', '-', '+', ',', ':' can be used as separators, eg. "1yr-2mo", "2m:6s" SELECT parseTimeDelta('1yr-2mo-4w + 12 days, 3 hours : 1 minute ; 33 seconds');. #39071 (jiahui-97).
  • Added options to limit IO operations with remote storage: max_remote_read_network_bandwidth_for_server and max_remote_write_network_bandwidth_for_server. #39095 (Sergei Trifonov).
  • Add send_logs_source_regexp setting. Send server text logs with specified regexp to match log source name. Empty means all sources. #39161 (Amos Bird).
  • OpenTelemetry now collects traces without Processors spans by default. To enable Processors spans collection opentelemetry_trace_processors setting. #39170 (Ilya Yatsishin).

Performance Improvement

  • Add new local_filesystem_read_method method io_uring based on the asynchronous Linux io_uring subsystem, improving read performance almost universally compared to the default pread method. #36103 (Saulius Valatka).
  • Distinct optimization for sorted columns. Use specialized distinct transformation in case input stream is sorted by column(s) in distinct. Optimization can be applied to pre-distinct, final distinct, or both. Initial implementation by @dimarub2000. #37803 (Igor Nikonov).
  • Add VBMI optimized copyOverlap32Shuffle for LZ4 decompress. #37891 (Guo Wangyang).
  • Improve performance of ORDER BY, MergeTree merges, window functions using batch version of BinaryHeap. #38022 (Maksim Kita).
  • Fix significant join performance regression which was introduced in https://github.com/ClickHouse/ClickHouse/pull/35616 . It's interesting that common join queries such as ssb queries have been 10 times slower for almost 3 months while no one complains. #38052 (Amos Bird).
  • Migrate from the Intel hyperscan library to vectorscan, this speeds up many string matching on non-x86 platforms. #38171 (Robert Schulze).
  • Increased parallelism of query plan steps executed after aggregation. #38295 (Nikita Taranov).
  • Improve performance of insertion to columns of type JSON. #38320 (Anton Popov).
  • Optimized insertion and lookups in the HashTable. #38413 (Nikita Taranov).
  • Fix performance degradation from #32493. #38417 (Alexey Milovidov).
  • Improve performance of column vector replicate using SIMD instructions. Author @zzachimed. #38565 (Maksim Kita).
  • Norm and Distance functions for arrays speed up 1.2-2 times. #38740 (Alexander Gololobov).
  • A less efficient execution plan can be generated for query with ORDER BY (a, b) than for ORDER BY a, b. #38873 (Igor Nikonov).
  • Executable UDF, Executable Dictionary, Executable Storage poll subprocess fix 1 second subprocess wait during subprocess termination. #38929 (Constantine Peresypkin).
  • Optimize accesses to system.stack_trace. #39177 (Azat Khuzhin).

Improvement

  • Optimized processing of ORDER BY in window functions. #34632 (Vladimir Chebotarev).
  • Support SQL standard create index and drop index syntax. #35166 (Jianmei Zhang).
  • use simd to re-write the current column replicate funcion and got 2x performance boost in our unit benchmark test. #37235 (zzachimed).
  • Send profile events for INSERT queries (previously only SELECT was supported). #37391 (Azat Khuzhin).
  • Implement in order aggregation (optimize_aggregation_in_order) for fully materialized projections. #37469 (Azat Khuzhin).
  • Support expressions with window functions. Closes #19857. #37848 (Dmitry Novik).
  • S3 single objects are now removed with RemoveObjectRequest (sic). Fixed a bug with S3ObjectStorage on GCP which did not allow to use removeFileIfExists effectively breaking approximately half of remove functionality. Automatic detection for DeleteObjects S3 API, that is not supported by GCS. This will allow to use GCS without explicit support_batch_delete=0 in configuration. #37882 (Vladimir Chebotarev).
  • Fix refcnt for unused MergeTree parts in SELECT queries (may defer parts removal). #37913 (Azat Khuzhin).
  • Expose basic Keeper related monitoring data (via ProfileEvents and CurrentMetrics). #38072 (lingpeng0314).
  • Added kerberosInit function and corresponding KerberosInit class as a replacement for kinit executable. Replaced all calls of kinit in Kafka and HDFS code by call of kerberosInit function. Added new integration test. Closes #27651. #38105 (Roman Vasin).
    • Add setting multiple_joins_try_to_keep_original_names to not rewrite identifier name on multiple JOINs rewrite, close #34697. #38149 (Vladimir C).
  • improved trace-visualizer UX. #38169 (Sergei Trifonov).
  • Add ability to pass headers to url table function / storage via sql. Closes #37897. #38176 (Kseniia Sumarokova).
  • Enable trace collection for AArch64. #38181 (Maksim Kita).
  • Do not skip symlinks in user_defined directory during SQL user defined functions loading. Closes #38042. #38184 (Maksim Kita).
  • Improve the stability for hive storage integration test. Move the data prepare step into test.py. #38260 (lgbo).
  • Added background cleanup of subdirectories in store/. In some cases clickhouse-server might left garbage subdirectories in store/ (for example, on unsuccessful table creation) and those dirs were never been removed. Fixes #33710. #38265 (Alexander Tokmakov).
  • Add DESCRIBE CACHE query to show cache settings from config. Add SHOW CACHES query to show available filesystem caches list. #38279 (Kseniia Sumarokova).
  • Add access check for system drop fs cache. Support ON CLUSTER. #38319 (Kseniia Sumarokova).
  • Support auto_close option for postgres engine connection. Closes #31486. #38363 (Kseniia Sumarokova).
  • Fix PostgreSQL database engine incompatibility on upgrade from 21.3 to 22.3. Closes #36659. #38369 (Kseniia Sumarokova).
  • filesystemAvailable and similar functions now work in clickhouse-local. This closes #38423. #38424 (Alexey Milovidov).
  • Hardware benchmark now has support for automatic results uploading. #38427 (Alexey Milovidov).
  • The table system.asynchronous_metric_log is further optimized for storage space. This closes #38134. See the YouTube video. #38428 (Alexey Milovidov).
  • Functions multiMatchAny(), multiMatchAnyIndex(), multiMatchAllIndices() and their fuzzy variants now accept non-const pattern array argument. #38485 (Robert Schulze).
  • Added L2 Squared distance and norm for both arrays and tuples. #38545 (Julian Gilyadov).
  • Add revision() function. #38555 (Azat Khuzhin).
  • Add group_by_use_nulls setting to make aggregation key columns nullable in the case of ROLLUP, CUBE and GROUPING SETS. Closes #37359. #38642 (Dmitry Novik).
  • Fix GCS via proxy tunnel usage. #38726 (Azat Khuzhin).
  • Support \i file in clickhouse client / local (similar to psql \i). #38813 (Kseniia Sumarokova).
  • Allow null modifier in columns declaration for table functions. #38816 (Kruglov Pavel).
    • Deactivate mutations_finalizing_task before shutdown to avoid TABLE_IS_READ_ONLY errors. #38851 (Raúl Marín).
  • Fix waiting of shared lock after exclusive lock failure. #38864 (Azat Khuzhin).
  • Add the ability to specify compression level during data export. #38907 (Nikolay Degterinsky).
  • New option rewrite in EXPLAIN AST. If enabled, it shows AST after it's rewritten, otherwise AST of original query. Disabled by default. #38910 (Igor Nikonov).
    • Stop reporting Zookeeper "Node exists" exceptions in system.errors when they are expected. #38961 (Raúl Marín).
  • Allow to specify globs * or {expr1, expr2, expr3} inside a key for clickhouse-extract-from-config tool. #38966 (Nikita Mikhaylov).
  • Add option enabling that SELECT from the system database requires grant. Details:. #38970 (Vitaly Baranov).
    • clearOldLogs: Don't report KEEPER_EXCEPTION on concurrent deletes. #39016 (Raúl Marín).
  • clickhouse-keeper improvement: persist metainformation about keeper servers to disk. #39069 (Antonio Andelic).
  • Continue without exception when running out of disk space when using filesystem cache. #39106 (Kseniia Sumarokova).
  • Handling SIGTERM signals from k8s. #39130 (Timur Solodovnikov).
  • SQL function multiStringAllPositions() now accepts non-const needle arguments. #39167 (Robert Schulze).
  • Add merge_algorithm (Undecided, Horizontal, Vertical) to system.part_log. #39181 (Azat Khuzhin).
  • Improve isNullable/isConstant/isNull/isNotNull performance for LowCardinality argument. #39192 (Kruglov Pavel).
  • Metric result_bytes for INSERT queries in system.query_log shows number of bytes inserted. Previously value was incorrect and stored the same value as result_rows. #39225 (Ilya Yatsishin).
  • The CPU usage metric in clickhouse-client will be displayed in a better way. Fixes #38756. #39280 (Sergei Trifonov).
  • Rethrow exception on filesystem cache initialisation on server startup, better error message. #39386 (Kseniia Sumarokova).
  • Support milliseconds, microseconds and nanoseconds in parseTimeDelta function. #39447 (Kruglov Pavel).

Bug Fix

Build/Testing/Packaging Improvement

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

  • Fix projection exception when aggregation keys are wrapped inside other functions. This fixes #37151. #37155 (Amos Bird).
  • Fix possible logical error ... with argument with type Nothing and default implementation for Nothing is expected to return result with type Nothing, got ... in some functions. Closes: #37610 Closes: #37741. #37759 (Kruglov Pavel).
  • Fix incorrect columns order in subqueries of UNION (in case of duplicated columns in subselects may produce incorrect result). #37887 (Azat Khuzhin).
  • Fix incorrect work of MODIFY ALTER Column with column names that contain dots. Closes #37907. #37971 (Kruglov Pavel).
  • Fix reading of sparse columns from MergeTree tables that store their data in S3. #37978 (Anton Popov).
  • Fix rounding for Decimal128/Decimal256 with more than 19-digits long scale. #38027 (Igor Nikonov).
  • Fix possible crash in Distributed async insert in case of removing a replica from config. #38029 (Nikolai Kochetov).
  • Fix "Missing columns" for GLOBAL JOIN with CTE w/o alias. #38056 (Azat Khuzhin).
  • Rewrite tuple functions as literals in backwards-compatibility mode. #38096 (Anton Kozlov).
    • Fix redundant memory reservation for output block during ORDER BY. #38127 (iyupeng).
  • Fix possible logical error Bad cast from type DB::IColumn* to DB::ColumnNullable* in array mapped functions. Closes #38006. #38132 (Kruglov Pavel).
  • With table SQL CREATE TABLE nested_name_tuples ( `a` Tuple(x String, y Tuple(i Int32, j String)) ) ENGINE = Memory; . #38136 (lgbo).
  • Fix bug with nested short-circuit functions that led to execution of arguments even if condition is false. Closes #38040. #38173 (Kruglov Pavel).
  • (Window View is a experimental feature) Fix LOGICAL_ERROR for WINDOW VIEW with incorrect structure. #38205 (Azat Khuzhin).
  • Update librdkafka submodule to fix crash when an OAUTHBEARER refresh callback is set. #38225 (Rafael Acevedo).
  • Do not allow recursive usage of OvercommitTracker during logging. Fixes #37794 cc @tavplubix @davenger. #38246 (Dmitry Novik).
  • Fix INSERT into Distributed hung due to ProfileEvents. #38307 (Azat Khuzhin).
  • Fix retries in PostgreSQL engine. #38310 (Kseniia Sumarokova).
  • Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result). #38324 (Azat Khuzhin).
  • Fix RabbitMQ with formats based on PeekableReadBuffer. Closes #38061. #38356 (Kseniia Sumarokova).
  • Fix possible Invalid number of rows in Chunk in materialised pg. Closes #37323. #38360 (Kseniia Sumarokova).
  • Fix RabbitMQ configuration with connection string setting. Closes #36531. #38365 (Kseniia Sumarokova).
  • Fix PostgreSQL engine not using PostgreSQL schema when retrieving array dimension size. Closes #36755. Closes #36772. #38366 (Kseniia Sumarokova).
  • Fix incorrect result of distributed queries with DISTINCT and LIMIT. Fixes #38282. #38371 (Anton Popov).
  • fix: expose new CH keeper port in Dockerfile clickhouse/clickhouse-keeper fix: use correct KEEPER_CONFIG filename in clickhouse/clickhouse-keeper docker image. #38462 (Evgeny Kruglov).
  • Fix parts removal (will be left forever if they had not been removed on server shutdown) after incorrect server shutdown. #38486 (Azat Khuzhin).
  • Fixes #38498 Current Implementation is similar to what shell does mentiond by @rschu1ze here. #38502 (Heena Bansal).
  • Fix table creation to avoid replication issues with pre-22.4 replicas. #38541 (Raúl Marín).
  • Fix crash for mapUpdate, mapFilter functions when using with constant map argument. Closes #38547. #38553 (hexiaoting).
  • Fix wrong results of countSubstrings() & position() on patterns with 0-bytes. #38589 (Robert Schulze).
  • Now it's possible to start a clickhouse-server and attach/detach tables even for tables with the incorrect values of IPv4/IPv6 representation. Proper fix for issue #35156. #38590 (alesapin).
  • Adapt some more nodes to avoid issues with pre-22.4 replicas. #38627 (Raúl Marín).
  • Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes #38333. #38675 (Amos Bird).
  • rankCorr function will work correctly if some arguments are NaNs. This closes #38396. #38722 (Alexey Milovidov).
  • Fix parallel_view_processing=1 with optimize_trivial_insert_select=1. Fix max_insert_threads while pushing to views. #38731 (Azat Khuzhin).
  • Fix use-after-free for Map combinator that leads to incorrect result. #38748 (Azat Khuzhin).
  • Fix throwing exception for seekable read from s3 (exception was not thrown). #38773 (Kseniia Sumarokova).
  • Fix checking whether s3 storage support parallel writes. It resulted in s3 parallel writes not working. #38792 (chen).
  • Fix s3 seekable reads with parallel read buffer. (Affected memory usage during query). Closes #38258. #38802 (Kseniia Sumarokova).
  • Update simdjson. This fixes #38621. #38838 (Alexey Milovidov).
  • MergeTree fix possible logical error for Vertical merges. #38859 (Maksim Kita).
  • Fix incorrect partition pruning when there is a nullable partition. This fixes #38941. #38946 (Amos Bird).
  • Fix fsync_part_directory for fetches. #38993 (Azat Khuzhin).
  • Functions multiMatchFuzzy() no throw a logical error if the needle argument is empty. #39012 (Robert Schulze).
  • Any allocations inside OvercommitTracker may lead to deadlock. Logging was not very informative so it's easier just to remove logging. Fixes #37794. #39030 (Dmitry Novik).
  • Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis). This fixes #38333. #39037 (Amos Bird).
  • Fix bug in filesystem cache that could happen in some corner case which coincided with cache capacity hitting the limit. Closes #39066. #39070 (Kseniia Sumarokova).
  • Fix ActionsDAG construction for arguments of window expressions. Fixes #38538 Allow using of higher-order functions in window expressions. #39112 (Dmitry Novik).
  • Keep LowCardinality type in tuple() function. Previously LowCardinality type was dropped and elements of created tuple had underlying type of LowCardinality. #39113 (Anton Popov).
  • Fix error Block structure mismatch which could happen for INSERT into table with attached MATERIALIZED VIEW and enabled setting extremes = 1. Closes #29759 and #38729. #39125 (Nikolai Kochetov).
  • Fix unexpected query result when both optimize_trivial_count_query and empty_result_for_aggregation_by_empty_set are set to true. This fixes #39140. #39155 (Amos Bird).
  • Fixed error Not found column Type in block in selects with PREWHERE and read-in-order optimizations. #39157 (Yakov Olkhovskiy).
  • Fix extremely rare race condition in during hardnlinks for remote fs. The only way to reproduce it is concurrent run of backups. #39190 (alesapin).
  • Fix fetch of in-memory part with allow_remote_fs_zero_copy_replication. #39214 (Azat Khuzhin).
  • Fix NOEXCEPT_SCOPE (before it calls std::terminate and looses the exception). #39229 (Azat Khuzhin).
  • Declare RabbitMQ queue without default arguments x-max-length and x-overflow. #39259 (rnbondarenko).
  • Fix segmentation fault in MaterializedPostgreSQL database engine, which could happen if some exception occurred at replication initialisation. Closes #36939. #39272 (Kseniia Sumarokova).
  • Fix incorrect fetch postgresql tables query fro PostgreSQL database engine. Closes #33502. #39283 (Kseniia Sumarokova).
  • Fix possible UB in MergeTreeBackgroundExecutor (leads to SIGSEGV on race with DROP/DETACH). #39342 (Azat Khuzhin).
  • Avoid possible abort() in CapnProto on exception descruction. Closes #30706. #39365 (Kruglov Pavel).
  • Fix behaviour of dictHas for direct dictionaries when multiple lookups to the same key are made in a single action. #39385 (James Morrison).
  • Fix crash which may happen while reading from dictionary with DateTime64 attribute. Fixes #38930. #39391 (Nikolai Kochetov).
  • Fix WriteBuffer finalize in destructor when cacnel query that could lead to stuck query or even terminate. Closes #38199. #39396 (Kruglov Pavel).
  • Fix UB (stack-use-after-scope) in extactAll(). #39397 (Azat Khuzhin).
  • Fix incorrect query result when trivial count optimization is in effect with array join. This fixes #39431. #39444 (Amos Bird).

Bug Fix (user-visible misbehaviour in official stable release)

  • Disable send_logs_level for INSERT into Distributed to avoid possible hung. #35075 (Azat Khuzhin).

NO CL ENTRY

  • NO CL ENTRY: 'Revert "Add a setting to use more memory for zstd decompression"'. #38194 (alesapin).
  • NO CL ENTRY: 'Revert "Revert "Add a setting to use more memory for zstd decompression""'. #38196 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "ClickHouse's boringssl module updated to the official version of the FIPS compliant."'. #38201 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)"'. #38361 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Add support for io_uring read method"'. #38377 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Revert "Fix optimization in PartialSortingTransform (SIGSEGV and possible incorrect result)""'. #38449 (Maksim Kita).
  • NO CL ENTRY: 'Don't spoil return code of integration tests runner with redundant tee'. #38548 (Vladimir Chebotarev).
  • NO CL ENTRY: 'Revert "Non Negative Derivative window function"'. #38551 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Upload to S3 compressed self-extracting clickhouse"'. #38788 (Mikhail f. Shiryaev).
  • NO CL ENTRY: 'Revert "Smallish updates of dev guide"'. #38848 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Fix toHour() monotonicity which can lead to incorrect query result (incorrect index analysis)"'. #39001 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Fix WriteBuffer finalize in destructor when cacnel query"'. #39433 (Kruglov Pavel).
  • NO CL ENTRY: 'Revert "[RFC] Fix LSan by fixing getauxval()"'. #39434 (Alexey Milovidov).
  • NO CL ENTRY: 'Revert "Remove broken optimisation in Direct dictionary dictHas implementation"'. #39461 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Fix trivial count optimization with array join"'. #39466 (Nikolai Kochetov).

NOT FOR CHANGELOG / INSIGNIFICANT

Performance optimization and Bug Fix

  • Enabled pread_threadpool read method by default. It will increase read performance. Bug fix: if direct IO is enabled and the number of threads is large and pread_threadpool is used, it may cause a logical error. #33653 (Alexey Milovidov).