ClickHouse/docs/changelogs/v22.1.1.2542-prestable.md
2022-06-23 12:13:08 +02:00

82 KiB

sidebar_position sidebar_label
1 2022

2022 Changelog

ClickHouse release v22.1.1.2542-prestable FIXME as compared to v21.12.1.9017-prestable

Backward Incompatible Change

  • Change ZooKeeper path for zero-copy marks for shared data. Fix for remove marks in ZooKeeper for renamed parts. #32061 (ianton-ru).
    • Account for scalar subqueries. With this change, rows read in scalar subqueries are now reported in the query_log. If the scalar subquery is cached (repeated or called for several rows) the rows read are only counted once. This change allows KILLing queries and reporting progress while they are executing scalar subqueries. #32271 (Raúl Marín).
  • Add left, right, leftUTF8, rightUTF8 functions. Fix error in implementation of substringUTF8 function with negative offset (offset from the end of string). The functions left and right were previously implemented in parser. Upgrade notes: distributed queries with left or right functions without aliases may throw exception if cluster contains different versions of clickhouse-server. If you are upgrading your cluster and encounter this error, you should finish upgrading your cluster to ensure all nodes have the same version. Also you can add aliases (AS something) to the columns in your queries to avoid this issue. #33407 (Alexey Milovidov).

New Feature

  • Implemented sparse serialization. It can reduce usage of disk space and improve performance of some queries for columns, which contain a lot of default (zero) values. It can be enabled by setting ratio_of_defaults_for_sparse_serialization. Sparse serialization will be chosen dynamically for column, if it has ratio of number of default values to number of all values above that threshold. Serialization (default or sparse) will be fixed for every column in part, but may varies between parts. #22535 (Anton Popov).
  • add grouping sets function, like GROUP BY grouping sets (a, b, (a, b)). #26869 (taylor12805).
  • Added an ability to read from all replicas within a shard during distributed query. To enable this, set allow_experimental_parallel_reading_from_replicas=true and max_parallel_replicas to any number. This closes #26748. #29279 (Nikita Mikhaylov).
  • Start and stop servers when hosts and ports configuration changes. #30549 (Kevin Michel).
  • Create any kind of view with comment. ... #31062 (Vasily Nemkov).
  • Implement hive table engine to access apache hive from clickhouse. Related RFC: #29245. #31104 (李扬).
  • Adding support for disks backed by Azure Blob Storage, in a similar way it has been done for disks backed by AWS S3. Current implementation allows for all the basic disk operations. #31505 (Jakub Kuklis).
    • Add "TABLE OVERRIDE" feature for customizing MaterializedMySQL table schemas. #32325 (Stig Bakken).
  • Implement data schema inference for input formats. Allow to skip structure (or write just auto) in table functions file, url, s3, hdfs and in parameters of clickhouse-local . Allow to skip structure in create query for table engines File, HDFS, S3, URL, Merge, Buffer, Distributed and ReplicatedMergeTree (if we add new replicas). #32455 (Kruglov Pavel).
  • Support TABLE OVERRIDE clause for MaterializedPostgreSQL. RFC: #31480. #32749 (Kseniia Sumarokova).
  • Add EXPLAIN TABLE OVERRIDE query. #32836 (Stig Bakken).
  • If an invalid setting is defined using the SET query or using the query parameters in the HTTP request, error message will contain suggestions that are similar to the invalid setting string (if any exists). #32946 (Antonio Andelic).
  • Add aggregate functions cramersV, cramersVBiasCorrected, theilsU and contingency. These functions calculate dependency (measure of association) between categorial values. All these functions are using cross-tab (histogram on pairs) for implementation. You can imagine it like a correlation coefficient but for any discrete values (not necessary numbers). #33366 (Alexey Milovidov).
  • Added function arrayLast. Closes #33390. #33415 (Maksim Kita).
  • Add MONTHNAME function. #33436 (usurai).
  • Auto detect file extension. Close #30918. #33443 (zhongyuankai).
  • Added function arrayLastIndex. #33465 (Maksim Kita).
  • Add new h3 miscellaneous functions: h3DegsToRads, h3RadsToDegs, h3HexAreaKm2, h3CellAreaM2, h3CellAreaRads2. #33479 (Bharat Nallan).
  • Detect format by file extension in file/hdfs/s3/url table functions and HDFS/S3/URL table engines. #33565 (Kruglov Pavel).

Performance Improvement

  • Non significant change. In extremely rare cases when data part is lost on every replica, after merging of some data parts, the subsequent queries may skip less amount of partitions during partition pruning. This hardly affects anything. #32220 (Azat Khuzhin).
  • Slight performance improvement of reinterpret function. #32587 (Alexey Milovidov).
  • Reduce allocated memory for dictionaries with string attributes. #33466 (Maksim Kita).
  • Avoid exponential backtracking in parser. This closes #20158. #33481 (Alexey Milovidov).

Improvement

  • Now date time conversion functions that generates time before 1970-01-01 00:00:00 will be saturated to zero instead of overflow. #29953 (Amos Bird).
  • Added settings command_read_timeout, command_write_timeout for StorageExecutable, StorageExecutablePool, ExecutableDictionary, ExecutablePoolDictionary, ExecutableUserDefinedFunctions. Setting command_read_timeout controls timeout for reading data from command stdout in milliseconds. Setting command_write_timeout timeout for writing data to command stdin in milliseconds. Added settings command_termination_timeout for ExecutableUserDefinedFunction, ExecutableDictionary, StorageExecutable. Added setting execute_direct for ExecutableUserDefinedFunction, by default true. Added setting execute_direct for ExecutableDictionary, ExecutablePoolDictionary, by default false. #30957 (Maksim Kita).
  • Optimize single part projection materialization. This closes #31669. #31885 (Amos Bird).
  • Enable comparison between Decimal and Float. Closes #22626. #31966 (flynn).
    • Ignore parse failure of opentelemetry's traceparent header. #32116 (Frank Chen).
  • Improve keeper writing performance by optimization the size calculation logic. #32366 (zhanglistar).
  • Allows to connect to mongodb 5.0. Closes #31483,. #32416 (Kseniia Sumarokova).
  • Disable some optimizations for window functions. Closes #31535. Closes #31620. #32453 (Kseniia Sumarokova).
  • Allow empty credentials for mongo engine. Closes #26267. #32460 (Kseniia Sumarokova).
  • Improve handling nested structures with missing columns while reading protobuf. Follow-up to https://github.com/ClickHouse/ClickHouse/pull/31988. #32531 (Vitaly Baranov).
  • Events clause support for window view watch query. #32607 (vxider).
  • Add settings max_concurrent_select_queries and max_concurrent_insert_queries for control concurrent queries by query kind. Close #3575. #32609 (SuperDJY).
  • support Date32 for genarateRandom engine. #32643 (nauta).
  • Support authSource option for storage MongoDB. Closes #32594. #32702 (Kseniia Sumarokova).
  • Allow to control connection timeouts for mysql (previously was supported only for dictionary source). Closes #16669. Previously default connect_timeout was rather small, now it is configurable. #32734 (Kseniia Sumarokova).
  • Flush all In-Memory data parts when WAL is not enabled while shutdown server or detaching table. #32742 (nauta).
  • Improve gRPC compression support for #28671. #32747 (Vitaly Baranov).
  • Added support for specifying subquery as SQL user defined function. Example: CREATE FUNCTION test AS () -> (SELECT 1). Closes #30755. #32758 (Maksim Kita).
  • Support hints for clickhouse-client and clickhouse-local. Closes #32237,. #32841 (凌涛).
    • Do not prepend THREADS_COUNT with -j to avoid additional prepending in subprocesses. #32844 (Ernest Zaslavsky).
  • Added support for BIT data type in MaterializedMySQL. Closes #15182, #32233. #32900 (zzsmdfj).
  • More efficient handling of globs for url storage. Closes #32866. #32907 (Kseniia Sumarokova).
  • This only happens in unofficial builds. Fixed segfault when inserting data into compressed Decimal, String, FixedString and Array columns. This closes #32939. #32940 (N. Kolotov).
  • Dictionaries added Date32 date type support. Closes #32913. #32971 (Maksim Kita).
  • Short circuit evaluation function throwIf support. Closes #32969. #32973 (Maksim Kita).
  • Improve Bool type serialization and deserialization. #32984 (Kruglov Pavel).
  • Send profile info in clickhouse-local. Closes #33093. #33097 (Kseniia Sumarokova).
  • Inject git information into clickhouse binary file. So we can get source code revision easily from clickhouse binary file. #33124 (李扬).
  • Validate config keys for external dictionaries. #33095#issuecomment-1000577517. #33130 (Kseniia Sumarokova).
  • Add retry for Postgres connect in case nothing has been fetched yet. Closes #33199. #33209 (Kseniia Sumarokova).
  • Correctly prevent nullable primary keys if necessary. This is for #32780. #33218 (Amos Bird).
  • If storage supports SETTINGS allow to pass them as key value or via config. Add this support for mysql. #33231 (Kseniia Sumarokova).
  • Allow to cancel formats Arrow / Parquet / ORC which failed to be cancelled it case of big files and setting input_format_allow_seeks as false. Closes #29678. #33238 (Kseniia Sumarokova).
  • Always display resource usage (total CPU usage, total RAM usage and max RAM usage per host) in client. #33271 (Alexey Milovidov).
  • parseDateTimeBestEffort support Unix Timestamp with Milliseconds. #33276 (Ben).
  • Support moving conditions to PREWHERE (setting optimize_move_to_prewhere) for tables of Merge engine if its all underlying tables supports PREWHERE. #33300 (Anton Popov).
  • Pressing Ctrl+C twice will terminate clickhouse-benchmark immediately without waiting for in-flight queries. This closes #32586. #33303 (Alexey Milovidov).
  • support <secure/> in cluster configuration. Close #33270. #33330 (SuperDJY).
  • LineAsString can be used as output format. This closes #30919. #33331 (Sergei Trifonov).
  • Allow negative intervals in function intervalLengthSum. Their length will be added as well. This closes #33323. #33335 (Alexey Milovidov).
  • clickhouse-local: track memory under --max_memory_usage_in_client option. #33341 (Azat Khuzhin).
  • Make installation script working on FreeBSD. This closes #33384. #33418 (Alexey Milovidov).
  • Add some building options in system.build_options. #33431 (李扬).
  • Abuse of untuple function was leading to exponential complexity of query analysis (found by fuzzer). This closes #33297. #33445 (Alexey Milovidov).
  • Add config to enable ipv4 or ipv6. This close #33381. #33450 (Wu Xueyang).
  • add function decodeURLFormComponent. Close #10298. #33451 (SuperDJY).
  • Implement Materialized view getVirtuals function. Close #11210. #33482 (zhongyuankai).

Bug Fix

  • Quota limit was not reached, but the limit was exceeded. This PR fixes #31174. #31656 (sunny).
    • ClickHouse Keeper handler should remove operation when response sent. #32988 (JackyWoo).
  • Fix null pointer dereference in low cardinality data when deserializing LowCardinality data in the Native format. #33021 (Harry Lee).
  • Specifically crafted input data for Native format may lead to reading uninitialized memory or crash. This is relevant if clickhouse-server is open for write access to adversary. #33050 (Heena Bansal).

Build/Testing/Packaging Improvement

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

  • Fixed CAST from String to IPv4 or IPv6 and back. Fixed error message in case of failed conversion. #29224 (Dmitry Novik).
  • Fix base64Encode adding trailing bytes on small strings. #31797 (Kevin Michel).
  • Fix unexpected projection removal when detaching parts. #32067 (Amos Bird).
  • Fix 'APPLY lambda' parsing which could lead to client/server crash. #32138 (Kruglov Pavel).
  • Some replication queue entries might hang for temporary_directories_lifetime (1 day by default) with Directory tmp_merge_<part_name> or Part ... (state Deleting) already exists, but it will be deleted soon or similar error. It's fixed. Fixes #29616. #32201 (Alexander Tokmakov).
  • Fix unix timestamp Millisecond convert to DateTime64, fractional part calc reversed. #32240 (Ben).
  • Fix broken select query when there are more than 2 row policies on same column, begin at second queries on the same session. #31606. #32291 (SuperDJY).
  • Fix failures in queries that are trying to use skipping indices, which are not materialized yet. Fixes #32292 and #30343. #32359 (Anton Popov).
  • Fix issue with ambiguous query formatting in distributed queries that led to errors when some table columns were named ALL or DISTINCT. This closes #32391. #32490 (Alexey Milovidov).
  • The Proper handle of the case with apache arrow column duplication. #32507 (Dmitriy Mokhnatkin).
  • Fix crash in JoinCommon::removeColumnNullability, close #32458. #32508 (Vladimir C).
  • fix groupBitmapAnd function on distributed table. #32529 (minhthucdao).
  • Fix async inserts with formats CustomSeparated, Template, Regexp, MsgPack and JSONAsString. Previousely async inserts with these formats didn't read any data. #32530 (Kruglov Pavel).
  • Fix sparse_hashed dict performance with sequential keys (wrong hash function). #32536 (Azat Khuzhin).
  • Fix table lifetime (i.e. possible use-after-free) in case of parallel DROP TABLE and INSERT. #32572 (Azat Khuzhin).
  • Fix possible exception at RabbitMQ storage startup by delaying channel creation. #32584 (Kseniia Sumarokova).
  • Fix surprisingly bad code in function file. #32640 (Alexey Milovidov).
  • close #32504. #32649 (Vladimir C).
  • Fix LOGICAL_ERROR when the target of a materialized view is a JOIN or a SET table. #32669 (Raúl Marín).
  • Fix optimize_read_in_order optimization in case when table engine is Distributed or Merge and its underlying MergeTree tables have monotonous function in prefix of sorting key. #32670 (Anton Popov).
  • Fix ALTER TABLE ... MATERIALIZE TTL query with TTL ... DELETE WHERE ... and TTL ... GROUP BY ... modes. #32695 (Anton Popov).
  • Fix error Column is not under aggregate function in case of MV with GROUP BY (list of columns) (which is pared as GROUP BY tuple(...)) over Kafka/RabbitMQ. Fixes #32668 and #32744. #32751 (Nikolai Kochetov).
  • fix crash when used fuzzBits with multiply same FixedString, Close #32737. #32755 (SuperDJY).
  • Fix --database option for clickhouse-local. #32797 (Kseniia Sumarokova).
  • Server might fail to start if database with MySQL engine cannot connect to MySQL server, it's fixed. Fixes #14441. #32802 (Alexander Tokmakov).
  • Avoid reusing the scalar subquery cache when processing MV blocks. This fixes a bug when the scalar query reference the source table but it means that all subscalar queries in the MV definition will be calculated for each block. #32811 (Raúl Marín).
  • MergeTree table engine might silently skip some mutations if there are too many running mutations or in case of high memory consumption, it's fixed. Fixes #17882. #32814 (Alexander Tokmakov).
  • Fix optimization with lazy seek for async reads from remote fs. Closes #32803. #32835 (Kseniia Sumarokova).
  • Fixed --echo option is not used by clickhouse-client in batch mode with single query. #32843 (N. Kolotov).
  • Fix MV query with multiple chunk result. Fixes #31419. #32862 (Nikolai Kochetov).
  • Close #32487. #32914 (Vladimir C).
  • Fix ORC stripe reading. #32929 (Ernest Zaslavsky).
  • Fix a regression in replaceRegexpAll function. The function worked incorrectly when matched substring was empty. This closes #32777. This closes #30245. #32945 (Alexey Milovidov).
  • Fix UB in case of unexpected EOF during filling a set from HTTP query (i.e. if the client interrupted in the middle, i.e. timeout 0.15s curl -Ss -F 's=@t.csv;' 'http://127.0.0.1:8123/?s_structure=key+Int&query=SELECT+dummy+IN+s' and with large enough t.csv). #32955 (Azat Khuzhin).
  • Fix throwing exception like positional argument out of bounds for non-positional arguments. Closes #31173#event-5789668239. #32961 (Kseniia Sumarokova).
  • Fix hdfs url check that didn't allow using HA namenode address. Bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/31042. #32976 (Kruglov Pavel).
  • Fix wrong tuple output in CSV format in case of custom csv delimiter. #32981 (Kruglov Pavel).
  • Fix Context leak in case of cancel_http_readonly_queries_on_client_close (i.e. leaking of external tables that had been uploaded the the server and other resources). #32982 (Azat Khuzhin).
  • Remove obsolete code from ConfigProcessor. Yandex specific code is not used anymore. The code contained one minor defect. This defect was reported by Mallik Hassan in #33032. This closes #33032. #33026 (Alexey Milovidov).
  • Fix s3 table function reading empty file. Closes #33008. #33037 (Kseniia Sumarokova).
  • Fix error Invalid version for SerializationLowCardinality key column in case of reading from LowCardinality column with local_filesystem_read_prefetch or remote_filesystem_read_prefetch enabled. #33046 (Nikolai Kochetov).
  • Don't allow to write into S3 if path contains globs. #33142 (Kruglov Pavel).
  • fix incorrect metric: StorageBufferBytes. #33159 (xuyatian).
  • Fix MaterializedPostreSQL detach/attach (removing / adding to replication) tables with non-default schema. Found in #29535. #33179 (Kseniia Sumarokova).
  • Make sure unused replication slots are always removed. Found in #26952,. #33187 (Kseniia Sumarokova).
  • Fix ddl validation. Fix setting materialized_postgresql_allow_automatic_update. Closes #29535. #33200 (Kseniia Sumarokova).
  • Fix WHERE 1=0 for external databases query. Closes #33152. #33214 (Kseniia Sumarokova).
  • Add asynchronous inserts (with enabled setting async_insert) to query log. Previously such queries didn't appear in query log. #33239 (Anton Popov).
  • Fix ACLMap num, because acl_to_num will erase. #33246 (小路).
  • Fix ACL with explicit digit hash in clickhouse-keeper: now the behavior consistent with zookeeper and generated digest is always accepted. #33249 (小路).
  • Fix when COMMENT for dictionaries does not appear in system.tables, system.dictionaries. Allow to modify comment for Dictionary engine. Closes #33251. #33261 (Maksim Kita).
  • The commands SYSTEM SUSPEND and SYSTEM ... THREAD FUZZER missed access control. It is fixed. Author: Kevin Michel. #33333 (Alexey Milovidov).
  • Do not try to read pass EOF (to workaround a bug in a kernel), this bug can be reproduced on kernels (3.14..5.9), and requires index_granularity_bytes=0 (i.e. turn off adaptive index granularity). #33372 (Azat Khuzhin).
  • Fix possible use-after-free for INSERT into MV with concurrent DROP (#32572 significantly reduce the race window, this one should completely eliminate it). #33386 (Azat Khuzhin).
  • Fix query cancellation in case of allow_experimental_parallel_reading_from_replicas. #33456 (Azat Khuzhin).
  • Fix DROP MaterializedPostgreSQL database. #33468 (Kseniia Sumarokova).
  • Fix dictionary expressions for RangeHashedDictionary range min and range max attributes when created using DDL. Closes #30809. #33478 (Maksim Kita).
  • Fix an exception Block structure mismatch which may happen during insertion into table with default nested LowCardinality column. Fixes #33028. #33504 (Nikolai Kochetov).
  • Close issue: #33289 Fix bug when query view with setting offset and limit. #33518 (hexiaoting).
  • Fix parsing incorrect queries with FROM INFILE statement. #33521 (Kruglov Pavel).
  • Fix segfault in arrowSchemaToCHHeader if schema contains Dictionary type. Closes #33507. #33529 (Kruglov Pavel).
  • session_id_counter poniter to next slot. #33555 (小路).
  • Fix segfault in Avro that appears after the second insert into file. #33566 (Kruglov Pavel).
  • Fix wrong database for JOIN w/o explicit database in distributed queries (Fixes: #10471). #33611 (Azat Khuzhin).

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

Bug Fix (v21.9.4.35-stable)

NO CL CATEGORY

  • Fix Regular Expression while key path search. #33023 (mreddy017).
    • Allow to split GraphiteMergeTree rollup rules for plain/tagged metrics (optional rule_type field). #33494 (Michail Safronov).

NO CL ENTRY

NOT FOR CHANGELOG / INSIGNIFICANT

New Feature / New Tool