ClickHouse release v21.9, 2021-09-09

Backward Incompatible Change

  • Do not output trailing zeros in text representation of Decimal types. Example: 1.23 will be printed instead of 1.230000 for decimal with scale 6. This closes #15794. It may introduce slight incompatibility if your applications somehow relied on the trailing zeros. Serialization in output formats can be controlled with the setting output_format_decimal_trailing_zeros. Implementation of toString and casting to String is changed unconditionally. #27680 (alexey-milovidov).
  • Do not allow to apply parametric aggregate function with -Merge combinator to aggregate function state if state was produced by aggregate function with different parameters. For example, state of fooState(42)(x) cannot be finalized with fooMerge(s) or fooMerge(123)(s), parameters must be specified explicitly like fooMerge(42)(s) and must be equal. It does not affect some special aggregate functions like quantile and sequence* that use parameters for finalization only. #26847 (tavplubix).
  • Under clickhouse-local, always treat local addresses with a port as remote. #26736 (Raúl Marín).
  • Fix the issue that in case of some sophisticated query with column aliases identical to the names of expressions, bad cast may happen. This fixes #25447. This fixes #26914. This fix may introduce backward incompatibility: if there are different expressions with identical names, exception will be thrown. It may break some rare cases when enable_optimize_predicate_expression is set. #26639 (alexey-milovidov).
  • Now, scalar subquery always returns Nullable result if it’s type can be Nullable. It is needed because in case of empty subquery it’s result should be Null. Previously, it was possible to get error about incompatible types (type deduction does not execute scalar subquery, and it could use not-nullable type). Scalar subquery with empty result which can’t be converted to Nullable (like Array or Tuple) now throws error. Fixes #25411. #26423 (Nikolai Kochetov).

New Feature

  • Implementation of short circuit function evaluation, closes #12587. Add settings short_circuit_function_evaluation to configure short circuit function evaluation. #23367 (Kruglov Pavel).
  • Add support for INTERSECT, EXCEPT, ANY, ALL operators. #24757 (Kirill Ershov). (Kseniia Sumarokova).
  • Add support for encryption at the virtual file system level (data encryption at rest) using AES-CTR algorithm. #24206 (Latysheva Alexandra). (Vitaly Baranov) #26733 #26377 #26465.
  • Added natural language processing (NLP) functions for tokenization, stemming, lemmatizing and search in synonyms extensions. #24997 (Nikolay Degterinsky).
  • Added integration with S2 geometry library. #24980 (Andr0901). (Nikita Mikhaylov).
  • Add SQLite table engine, table function, database engine. #24194 (Arslan Gumerov). (Kseniia Sumarokova).
  • Added support for custom query for MySQL, PostgreSQL, ClickHouse, JDBC, Cassandra dictionary source. Closes #1270. #26995 (Maksim Kita).
  • Introduce syntax for here documents. Example SELECT $doc$ VALUE $doc$. #26671 (Maksim Kita).
  • Add shared (replicated) storage of user, roles, row policies, quotas and settings profiles through ZooKeeper. #27426 (Kevin Michel).
  • Add compression for INTO OUTFILE that automatically choose compression algorithm. Closes #3473. #27134 (Filatenkov Artur).
  • Add INSERT ... FROM INFILE similarly to SELECT ... INTO OUTFILE. #27655 (Filatenkov Artur).
  • Added complex_key_range_hashed dictionary. Closes #22029. #27629 (Maksim Kita).
  • Support expressions in JOIN ON section. Close #21868. #24420 (Vladimir C).
  • When client connects to server, it receives information about all warnings that are already were collected by server. (It can be disabled by using option --no-warnings). Add system.warnings table to collect warnings about server configuration. #26246 (Filatenkov Artur). #26282 (Filatenkov Artur).
  • Allow using constant expressions from with and select in aggregate function parameters. Close #10945. #27531 (abel-cheng).
  • Add tupleToNameValuePairs, a function that turns a named tuple into an array of pairs. #27505 (Braulio Valdivielso Martínez).
  • Add support for bzip2 compression method for import/export. Closes #22428. #27377 (Nikolay Degterinsky).
  • Added bitmapSubsetOffsetLimit(bitmap, offset, cardinality_limit) function. It creates a subset of bitmap limit the results to cardinality_limit with offset of offset. #27234 (DHBin).
  • Add column default_database to system.users. #27054 (kevin wan).
  • Supported cluster macros inside table functions ‘cluster’ and ‘clusterAllReplicas’. #26913 (polyprogrammist).
  • Add new functions currentRoles(), enabledRoles(), defaultRoles(). #26780 (Vitaly Baranov).
  • New functions currentProfiles(), enabledProfiles(), defaultProfiles(). #26714 (Vitaly Baranov).
  • Add functions that return (initial_)query_id of the current query. This closes #23682. #26410 (Alexey Boykov).
  • Add REPLACE GRANT feature. #26384 (Caspian).
  • EXPLAIN query now has EXPLAIN ESTIMATE ... mode that will show information about read rows, marks and parts from MergeTree tables. Closes #23941. #26131 (fastio).
  • Added system.zookeeper_log table. All actions of ZooKeeper client are logged into this table. Implements #25449. #26129 (tavplubix).
  • Zero-copy replication for ReplicatedMergeTree over HDFS storage. #25918 (Zhichang Yu).
  • Allow to insert Nested type as array of structs in Arrow, ORC and Parquet input format. #25902 (Kruglov Pavel).
  • Add a new datatype Date32 (store data as Int32), support date range same with DateTime64 support load parquet date32 to ClickHouse Date32 Add new function toDate32 like toDate. #25774 (LiuNeng).
  • Allow setting default database for users. #25268. #25687 (kevin wan).
  • Add an optional parameter to MongoDB engine to accept connection string options and support SSL connection. Closes #21189. Closes #21041. #22045 (Omar Bazaraa).

Experimental Feature

  • Added a compression codec AES_128_GCM_SIV which encrypts columns instead of compressing them. #19896 (PHO). Will be rewritten, do not use.
  • Rename MaterializeMySQL to MaterializedMySQL. #26822 (tavplubix).

Performance Improvement

  • Improve the performance of fast queries when max_execution_time = 0 by reducing the number of clock_gettime system calls. #27325 (filimonov).
  • Specialize date time related comparison to achieve better performance. This fixes #27083 . #27122 (Amos Bird).
  • Share file descriptors in concurrent reads of the same files. There is no noticeable performance difference on Linux. But the number of opened files will be significantly (10..100 times) lower on typical servers and it makes operations easier. See #26214. #26768 (alexey-milovidov).
  • Improve latency of short queries, that require reading from tables with large number of columns. #26371 (Anton Popov).
  • Don’t build sets for indices when analyzing a query. #26365 (Raúl Marín).
  • Vectorize the SUM of Nullable integer types with native representation (David Manzanares, Raúl Marín). #26248 (Raúl Marín).
  • Compile expressions involving columns with Enum types. #26237 (Maksim Kita).
  • Compile aggregate functions groupBitOr, groupBitAnd, groupBitXor. #26161 (Maksim Kita).
  • Improved memory usage with better block size prediction when reading empty DEFAULT columns. Closes #17317. #25917 (Vladimir Chebotarev).
  • Reduce memory usage and number of read rows in queries with ORDER BY primary_key. #25721 (Anton Popov).
  • Enable distributed_push_down_limit by default. #27104 (Azat Khuzhin).
  • Make toTimeZone monotonicity when timeZone is a constant value to support partition puring when use sql like:. #26261 (huangzhaowei).

Improvement

  • Mark window functions as ready for general use. Remove the allow_experimental_window_functions setting. #27184 (Alexander Kuzmenkov).
  • Improve compatibility with non-whole-minute timezone offsets. #27080 (Raúl Marín).
  • If file descriptor in File table is regular file - allow to read multiple times from it. It allows clickhouse-local to read multiple times from stdin (with multiple SELECT queries or subqueries) if stdin is a regular file like clickhouse-local --query "SELECT * FROM table UNION ALL SELECT * FROM table" ... < file. This closes #11124. Co-authored with (alexey-milovidov). #25960 (BoloniniD).
  • Remove duplicate index analysis and avoid possible invalid limit checks during projection analysis. #27742 (Amos Bird).
  • Enable query parameters to be passed in the body of HTTP requests. #27706 (Hermano Lustosa).
  • Disallow arrayJoin on partition expressions. #27648 (Raúl Marín).
  • Log client IP address if authentication fails. #27514 (Misko Lee).
  • Use bytes instead of strings for binary data in the GRPC protocol. #27431 (Vitaly Baranov).
  • Send response with error message if HTTP port is not set and user tries to send HTTP request to TCP port. #27385 (Braulio Valdivielso Martínez).
  • Add _CAST function for internal usage, which will not preserve type nullability, but non-internal cast will preserve according to setting cast_keep_nullable. Closes #12636. #27382 (Kseniia Sumarokova).
  • Add setting log_formatted_queries to log additional formatted query into system.query_log. It’s useful for normalized query analysis because functions like normalizeQuery and normalizeQueryKeepNames don’t parse/format queries in order to achieve better performance. #27380 (Amos Bird).
  • Add two settings max_hyperscan_regexp_length and max_hyperscan_regexp_total_length to prevent huge regexp being used in hyperscan related functions, such as multiMatchAny. #27378 (Amos Bird).
  • Memory consumed by bitmap aggregate functions now is taken into account for memory limits. This closes #26555. #27252 (alexey-milovidov).
  • Add new index data skipping minmax index format for proper Nullable support. #27250 (Azat Khuzhin).
  • Add 10 seconds cache for S3 proxy resolver. #27216 (ianton-ru).
  • Split global mutex into individual regexp construction. This helps avoid huge regexp construction blocking other related threads. #27211 (Amos Bird).
  • Support schema for PostgreSQL database engine. Closes #27166. #27198 (Kseniia Sumarokova).
  • Track memory usage in clickhouse-client. #27191 (Filatenkov Artur).
  • Try recording query_kind in system.query_log even when query fails to start. #27182 (Amos Bird).
  • Added columns replica_is_active that maps replica name to is replica active status to table system.replicas. Closes #27138. #27180 (Maksim Kita).
  • Allow to pass query settings via server URI in Web UI. #27177 (kolsys).
  • Add a new metric called MaxPushedDDLEntryID which is the maximum ddl entry id that current node push to zookeeper. #27174 (Fuwang Hu).
  • Improved the existence condition judgment and empty string node judgment when clickhouse-keeper creates znode. #27125 (小路).
  • Merge JOIN correctly handles empty set in the right. #27078 (Vladimir C).
  • Now functions can be shard-level constants, which means if it’s executed in the context of some distributed table, it generates a normal column, otherwise it produces a constant value. Notable functions are: hostName(), tcpPort(), version(), buildId(), uptime(), etc. #27020 (Amos Bird).
  • Updated extractAllGroupsHorizontal - upper limit on the number of matches per row can be set via optional third argument. #26961 (Vasily Nemkov).
  • Expose RocksDB statistics via system.rocksdb table. Read rocksdb options from ClickHouse config (rocksdb... keys). NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. #26821 (Azat Khuzhin).
  • Less verbose internal RocksDB logs. NOTE: ClickHouse does not rely on RocksDB, it is just one of the additional integration storage engines. This closes #26252. #26789 (alexey-milovidov).
  • Changing default roles affects new sessions only. #26759 (Vitaly Baranov).
  • Watchdog is disabled in docker by default. Fix for not handling ctrl+c. #26757 (Mikhail f. Shiryaev).
  • SET PROFILE now applies constraints too if they’re set for a passed profile. #26730 (Vitaly Baranov).
  • Improve handling of KILL QUERY requests. #26675 (Raúl Marín).
  • mapPopulatesSeries function supports Map type. #26663 (Ildus Kurbangaliev).
  • Fix excessive (x2) connect attempts with skip_unavailable_shards. #26658 (Azat Khuzhin).
  • Avoid hanging clickhouse-benchmark if connection fails (i.e. on EMFILE). #26656 (Azat Khuzhin).
  • Allow more threads to be used by the Kafka engine. #26642 (feihengye).
  • Add round-robin support for clickhouse-benchmark (it does not differ from the regular multi host/port run except for statistics report). #26607 (Azat Khuzhin).
  • Executable dictionaries (executable, executable_pool) enable creation with DDL query using clickhouse-local. Closes #22355. #26510 (Maksim Kita).
  • Set client query kind for mysql and postgresql compatibility protocol handlers. #26498 (anneji-dev).
  • Apply LIMIT on the shards for queries like SELECT * FROM dist ORDER BY key LIMIT 10 w/ distributed_push_down_limit=1. Avoid running Distinct/LIMIT BY steps for queries like SELECT DISTINCT shading_key FROM dist ORDER BY key. Now distributed_push_down_limit is respected by optimize_distributed_group_by_sharding_key optimization. #26466 (Azat Khuzhin).
  • Updated protobuf to 3.17.3. Changelogs are available on https://github.com/protocolbuffers/protobuf/releases. #26424 (Ilya Yatsishin).
  • Enable use_hedged_requests setting that allows to mitigate tail latencies on large clusters. #26380 (alexey-milovidov).
  • Improve behaviour with non-existing host in user allowed host list. #26368 (ianton-ru).
  • Add ability to set Distributed directory monitor settings via CREATE TABLE (i.e. CREATE TABLE dist (key Int) Engine=Distributed(cluster, db, table) SETTINGS monitor_batch_inserts=1 and similar). #26336 (Azat Khuzhin).
  • Save server address in history URLs in web UI if it differs from the origin of web UI. This closes #26044. #26322 (alexey-milovidov).
  • Add events to profile calls to sleep / sleepEachRow. #26320 (Raúl Marín).
  • Allow to reuse connections of shards among different clusters. It also avoids creating new connections when using cluster table function. #26318 (Amos Bird).
  • Control the execution period of clear old temporary directories by parameter with default value. #26212. #26313 (fastio).
  • Add a setting function_range_max_elements_in_block to tune the safety threshold for data volume generated by function range. This closes #26303. #26305 (alexey-milovidov).
  • Check hash function at table creation, not at sampling. Add settings for MergeTree, if someone create a table with incorrect sampling column but sampling never be used, disable this settings for starting the server without exception. #26256 (zhaoyu).
  • Added output_format_avro_string_column_pattern setting to put specified String columns to Avro as string instead of default bytes. Implements #22414. #26245 (Ilya Golshtein).
  • Add information about column sizes in system.columns table for Log and TinyLog tables. This closes #9001. #26241 (Nikolay Degterinsky).
  • Don’t throw exception when querying system.detached_parts table if there is custom disk configuration and detached directory does not exist on some disks. This closes #26078. #26236 (alexey-milovidov).
  • Check for non-deterministic functions in keys, including constant expressions like now(), today(). This closes #25875. This closes #11333. #26235 (alexey-milovidov).
  • convert timestamp and timestamptz data types to DateTime64 in PostgreSQL table engine. #26234 (jasine).
  • Apply aggressive IN index analysis for projections so that better projection candidate can be selected. #26218 (Amos Bird).
  • Remove GLOBAL keyword for IN when scalar function is passed. In previous versions, if user specified GLOBAL IN f(x) exception was thrown. #26217 (Amos Bird).
  • Add error id (like BAD_ARGUMENTS) to exception messages. This closes #25862. #26172 (alexey-milovidov).
  • Fix incorrect output with —progress option for clickhouse-local. Progress bar will be cleared once it gets to 100% - same as it is done for clickhouse-client. Closes #17484. #26128 (Kseniia Sumarokova).
  • Add merge_selecting_sleep_ms setting. #26120 (lthaooo).
  • Remove complicated usage of Linux AIO with one block readahead and replace it with plain simple synchronous IO with O_DIRECT. In previous versions, the setting min_bytes_to_use_direct_io may not work correctly if max_threads is greater than one. Reading with direct IO (that is disabled by default for queries and enabled by default for large merges) will work in less efficient way. This closes #25997. #26003 (alexey-milovidov).
  • Flush Distributed table on REPLACE TABLE query. Resolves #24566 - Do not replace (or create) table on [CREATE OR] REPLACE TABLE ... AS SELECT query if insertion into new table fails. Resolves #23175. #25895 (tavplubix).
  • Add views column to system.query_log containing the names of the (materialized or live) views executed by the query. Adds a new log table (system.query_views_log) that contains information about each view executed during a query. Modifies view execution: When an exception is thrown while executing a view, any view that has already startedwill continue running until it finishes. This used to be the behaviour under parallel_view_processing=true and now it’s always the same behaviour. - Dependent views now report reading progress to the context. #25714 (Raúl Marín).
  • Do connection draining asynchonously upon finishing executing distributed queries. A new server setting is added max_threads_for_connection_collector which specifies the number of workers to recycle connections in background. If the pool is full, connection will be drained synchronously but a bit different than before: It’s drained after we send EOS to client, query will succeed immediately after receiving enough data, and any exception will be logged instead of throwing to the client. Added setting drain_timeout (3 seconds by default). Connection draining will disconnect upon timeout. #25674 (Amos Bird).
  • Support for multiple includes in configuration. It is possible to include users configuration, remote servers configuration from multiple sources. Simply place <include /> element with from_zk, from_env or incl attribute and it will be replaced with the substitution. #24404 (nvartolomei).
  • Fix multiple block insertion into distributed table with insert_distributed_one_random_shard = 1. This is a marginal feature. Mark as improvement. #23140 (Amos Bird).
  • Support LowCardinality and FixedString keys/values for Map type. #21543 (hexiaoting).
  • Enable reloading of local disk config. #19526 (taiyang-li).
  • Now KeyConditions can correctly skip nullable keys, including isNull and isNotNull. https://github.com/ClickHouse/ClickHouse/pull/12433. #12455 (Amos Bird).

Bug Fix

  • Fix a couple of bugs that may cause replicas to diverge. #27808 (tavplubix).
  • Fix a rare bug in DROP PART which can lead to the error Unexpected merged part intersects drop range. #27807 (alesapin).
  • Prevent crashes for some formats when NULL (tombstone) message was coming from Kafka. Closes #19255. #27794 (filimonov).
  • Fix column filtering with union distinct in subquery. Closes #27578. #27689 (Kseniia Sumarokova).
  • Fix bad type cast when functions like arrayHas are applied to arrays of LowCardinality of Nullable of different non-numeric types like DateTime and DateTime64. In previous versions bad cast occurs. In new version it will lead to exception. This closes #26330. #27682 (alexey-milovidov).
  • Fix postgresql table function resulting in non-closing connections. Closes #26088. #27662 (Kseniia Sumarokova).
  • Fixed another case of Unexpected merged part ... intersecting drop range ... error. #27656 (tavplubix).
  • Fix an error with aliased column in Distributed table. #27652 (Vladimir C).
  • After setting max_memory_usage* to non-zero value it was not possible to reset it back to 0 (unlimited). It’s fixed. #27638 (tavplubix).
  • Fixed underflow of the time value when constructing it from components. Closes #27193. #27605 (Vasily Nemkov).
  • Fix crash during projection materialization when some parts contain missing columns. This fixes #27512. #27528 (Amos Bird).
  • fix metric BackgroundMessageBrokerSchedulePoolTask, maybe mistyped. #27452 (Ben).
  • Fix distributed queries with zero shards and aggregation. #27427 (Azat Khuzhin).
  • Compatibility when /proc/meminfo does not contain KB suffix. #27361 (Mike Kot).
  • Fix incorrect result for query with row-level security, PREWHERE and LowCardinality filter. Fixes #27179. #27329 (Nikolai Kochetov).
  • Fixed incorrect validation of partition id for MergeTree tables that created with old syntax. #27328 (tavplubix).
  • Fix MySQL protocol when using parallel formats (CSV / TSV). #27326 (Raúl Marín).
  • Fix Cannot find column error for queries with sampling. Was introduced in #24574. Fixes #26522. #27301 (Nikolai Kochetov).
  • Fix errors like Expected ColumnLowCardinality, gotUInt8 or Bad cast from type DB::ColumnVector<char8_t> to DB::ColumnLowCardinality for some queries with LowCardinality in PREWHERE. And more importantly, fix the lack of whitespace in the error message. Fixes #23515. #27298 (Nikolai Kochetov).
  • Fix distributed_group_by_no_merge = 2 with distributed_push_down_limit = 1 or optimize_distributed_group_by_sharding_key = 1 with LIMIT BY and LIMIT OFFSET. #27249 (Azat Khuzhin). These are obscure combination of settings that no one is using.
  • Fix mutation stuck on invalid partitions in non-replicated MergeTree. #27248 (Azat Khuzhin).
  • In case of ambiguity, lambda functions prefer its arguments to other aliases or identifiers. #27235 (Raúl Marín).
  • Fix column structure in merge join, close #27091. #27217 (Vladimir C).
  • In rare cases system.detached_parts table might contain incorrect information for some parts, it’s fixed. Fixes #27114. #27183 (tavplubix).
  • Fix uninitialized memory in functions multiSearch* with empty array, close #27169. #27181 (Vladimir C).
  • Fix synchronization in GRPCServer. This PR fixes #27024. #27064 (Vitaly Baranov).
  • Fixed cache, complex_key_cache, ssd_cache, complex_key_ssd_cache configuration parsing. Options allow_read_expired_keys, max_update_queue_size, update_queue_push_timeout_milliseconds, query_wait_timeout_milliseconds were not parsed for dictionaries with non cache type. #27032 (Maksim Kita).
  • Fix possible mutation stack due to race with DROP_RANGE. #27002 (Azat Khuzhin).
  • Now partition ID in queries like ALTER TABLE ... PARTITION ID xxx validates for correctness. Fixes #25718. #26963 (alesapin).
  • Fix “Unknown column name” error with multiple JOINs in some cases, close #26899. #26957 (Vladimir C).
  • Fix reading of custom TLDs (stops processing with lower buffer or bigger file). #26948 (Azat Khuzhin).
  • Fix error Missing columns: 'xxx' when DEFAULT column references other non materialized column without DEFAULT expression. Fixes #26591. #26900 (alesapin).
  • Fix loading of dictionary keys in library-bridge for library dictionary source. #26834 (Kseniia Sumarokova).
  • Aggregate function parameters might be lost when applying some combinators causing exceptions like Conversion from AggregateFunction(topKArray, Array(String)) to AggregateFunction(topKArray(10), Array(String)) is not supported. It’s fixed. Fixes #26196 and #26433. #26814 (tavplubix).
  • Add event_time_microseconds value for REMOVE_PART in system.part_log. In previous versions is was not set. #26720 (Azat Khuzhin).
  • Do not remove data on ReplicatedMergeTree table shutdown to avoid creating data to metadata inconsistency. #26716 (nvartolomei).
  • Sometimes SET ROLE could work incorrectly, this PR fixes that. #26707 (Vitaly Baranov).
  • Some fixes for parallel formatting (https://github.com/ClickHouse/ClickHouse/issues/26694). #26703 (Raúl Marín).
  • Fix potential nullptr dereference in window functions. This fixes #25276. #26668 (Alexander Kuzmenkov).
  • Fix clickhouse-client history file conversion (when upgrading from the format of 3 years old version of clickhouse-client) if file is empty. #26589 (Azat Khuzhin).
  • Fix incorrect function names of groupBitmapAnd/Or/Xor (can be displayed in some occasions). This fixes. #26557 (Amos Bird).
  • Update chown cmd check in clickhouse-server docker entrypoint. It fixes the bug that cluster pod restart failed (or timeout) on kubernetes. #26545 (Ky Li).
  • Fix crash in RabbitMQ shutdown in case RabbitMQ setup was not started. Closes #26504. #26529 (Kseniia Sumarokova).
  • Fix issues with CREATE DICTIONARY query if dictionary name or database name was quoted. Closes #26491. #26508 (Maksim Kita).
  • Fix broken column name resolution after rewriting column aliases. This fixes #26432. #26475 (Amos Bird).
  • Fix some fuzzed msan crash. Fixes #22517. #26428 (Nikolai Kochetov).
  • Fix infinite non joined block stream in partial_merge_join close #26325. #26374 (Vladimir C).
  • Fix possible crash when login as dropped user. This PR fixes #26073. #26363 (Vitaly Baranov).
  • Fix optimize_distributed_group_by_sharding_key for multiple columns (leads to incorrect result w/ optimize_skip_unused_shards=1/allow_nondeterministic_optimize_skip_unused_shards=1 and multiple columns in sharding key expression). #26353 (Azat Khuzhin).
  • Fixed rare bug in lost replica recovery that may cause replicas to diverge. #26321 (tavplubix).
  • Fix zstd decompression (for import/export in zstd framing format that is unrelated to tables data) in case there are escape sequences at the end of internal buffer. Closes #26013. #26314 (Kseniia Sumarokova).
  • Fix logical error on join with totals, close #26017. #26250 (Vladimir C).
  • Remove excessive newline in thread_name column in system.stack_trace table. This fixes #24124. #26210 (alexey-milovidov).
  • Fix potential crash if more than one untuple expression is used. #26179 (alexey-milovidov).
  • Don’t throw exception in toString for Nullable Enum if Enum does not have a value for zero, close #25806. #26123 (Vladimir C).
  • Fixed incorrect sequence_id in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes #21184. #26051 (tavplubix).
  • Fix for the case that cutToFirstSignificantSubdomainCustom()/cutToFirstSignificantSubdomainCustomWithWWW()/firstSignificantSubdomainCustom() returns incorrect type for consts, and hence optimize_skip_unused_shards does not work:. #26041 (Azat Khuzhin).
  • Fix possible mismatched header when using normal projection with prewhere. This fixes #26020. #26038 (Amos Bird).
  • Fix sharding_key from column w/o function for remote() (before select * from remote('127.1', system.one, dummy) leads to Unknown column: dummy, there are only columns . error). #25824 (Azat Khuzhin).
  • Fixed Not found column ... and Missing column ... errors when selecting from MaterializeMySQL. Fixes #23708, #24830, #25794. #25822 (tavplubix).
  • Fix optimize_skip_unused_shards_rewrite_in for non-UInt64 types (may select incorrect shards eventually or throw Cannot infer type of an empty tuple or Function tuple requires at least one argument). #25798 (Azat Khuzhin).

Build/Testing/Packaging Improvement

  • Now we ran stateful and stateless tests in random timezones. Fixes #12439. Reading String as DateTime and writing DateTime as String in Protobuf format now respect timezone. Reading UInt16 as DateTime in Arrow and Parquet formats now treat it as Date and then converts to DateTime with respect to DateTime’s timezone, because Date is serialized in Arrow and Parquet as UInt16. GraphiteMergeTree now respect time zone for rounding of times. Fixes #5098. Author: @alexey-milovidov. #15408 (alesapin).
  • clickhouse-test supports SQL tests with Jinja2 templates. #26579 (Vladimir C).
  • Add support for build with clang-13. This closes #27705. #27714 (alexey-milovidov). #27777 (Sergei Semin)
  • Add CMake options to build with or without specific CPU instruction set. This is for #17469 and #27509. #27508 (alexey-milovidov).
  • Fix linking of auxiliar programs when using dynamic libraries. #26958 (Raúl Marín).
  • Update RocksDB to 2021-07-16 master. #26411 (alexey-milovidov).

ClickHouse release v21.8, 2021-08-12

Upgrade Notes

  • New version is using Map data type for system logs tables (system.query_log, system.query_thread_log, system.processes, system.opentelemetry_span_log). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes #18698. #23934, #25773 (hexiaoting, sundy-li, Maksim Kita). If you want to downgrade from version 21.8 to older versions, you will need to cleanup system tables with logs manually. Look at /var/lib/clickhouse/data/system/*_log.

New Features

  • Add support for a part of SQL/JSON standard. #24148 (l1tsolaiki, Kseniia Sumarokova).
  • Collect common system metrics (in system.asynchronous_metrics and system.asynchronous_metric_log) on CPU usage, disk usage, memory usage, IO, network, files, load average, CPU frequencies, thermal sensors, EDAC counters, system uptime; also added metrics about the scheduling jitter and the time spent collecting the metrics. It works similar to atop in ClickHouse and allows access to monitoring data even if you have no additional tools installed. Close #9430. #24416 (alexey-milovidov, Yegor Levankov).
  • Add MaterializedPostgreSQL table engine and database engine. This database engine allows replicating a whole database or any subset of database tables. #20470 (Kseniia Sumarokova).
  • Add new functions leftPad(), rightPad(), leftPadUTF8(), rightPadUTF8(). #26075 (Vitaly Baranov).
  • Add the FIRST keyword to the ADD INDEX command to be able to add the index at the beginning of the indices list. #25904 (xjewer).
  • Introduce system.data_skipping_indices table containing information about existing data skipping indices. Close #7659. #25693 (Dmitry Novik).
  • Add bin/unbin functions. #25609 (zhaoyu).
  • Support Map and UInt128, Int128, UInt256, Int256 types in mapAdd and mapSubtract functions. #25596 (Ildus Kurbangaliev).
  • Support DISTINCT ON (columns) expression, close #25404. #25589 (Zijie Lu).
  • Add an ability to reset a custom setting to default and remove it from the table’s metadata. It allows rolling back the change without knowing the system/config’s default. Closes #14449. #17769 (xjewer).
  • Render pipelines as graphs in Web UI if EXPLAIN PIPELINE graph = 1 query is submitted. #26067 (alexey-milovidov).

Performance Improvements

  • Compile aggregate functions. Use option compile_aggregate_expressions to enable it. #24789 (Maksim Kita).
  • Improve latency of short queries that require reading from tables with many columns. #26371 (Anton Popov).

Improvements

  • Use Map data type for system logs tables (system.query_log, system.query_thread_log, system.processes, system.opentelemetry_span_log). These tables will be auto-created with new data types. Virtual columns are created to support old queries. Closes #18698. #23934, #25773 (hexiaoting, sundy-li, Maksim Kita).
  • For a dictionary with a complex key containing only one attribute, allow not wrapping the key expression in tuple for functions dictGet, dictHas. #26130 (Maksim Kita).
  • Implement function bin/hex from AggregateFunction states. #26094 (zhaoyu).
  • Support arguments of UUID type for empty and notEmpty functions. UUID is empty if it is all zeros (nil UUID). Closes #3446. #25974 (zhaoyu).
  • Add support for SET SQL_SELECT_LIMIT in MySQL protocol. Closes #17115. #25972 (Kseniia Sumarokova).
  • More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends. Added missing documentation. Close #5897. #25962 (alexey-milovidov).
  • Add setting optimize_move_to_prewhere_if_final. If query has FINAL, the optimization move_to_prewhere will be enabled only if both optimize_move_to_prewhere and optimize_move_to_prewhere_if_final are enabled. Closes #8684. #25940 (Kseniia Sumarokova).
  • Allow complex quoted identifiers of JOINed tables. Close #17861. #25924 (alexey-milovidov).
  • Add support for Unicode (e.g. Chinese, Cyrillic) components in Nested data types. Close #25594. #25923 (alexey-milovidov).
  • Allow quantiles* functions to work with aggregate_functions_null_for_empty. Close #25892. #25919 (alexey-milovidov).
  • Allow parameters for parametric aggregate functions to be arbitrary constant expressions (e.g., 1 + 2), not just literals. It also allows using the query parameters (in parameterized queries like {param:UInt8}) inside parametric aggregate functions. Closes #11607. #25910 (alexey-milovidov).
  • Correctly throw the exception on the attempt to parse an invalid Date. Closes #6481. #25909 (alexey-milovidov).
  • Support for multiple includes in configuration. It is possible to include users configuration, remote server configuration from multiple sources. Simply place <include /> element with from_zk, from_env or incl attribute, and it will be replaced with the substitution. #24404 (nvartolomei).
  • Support for queries with a column named "null" (it must be specified in back-ticks or double quotes) and ON CLUSTER. Closes #24035. #25907 (alexey-milovidov).
  • Support LowCardinality, Decimal, and UUID for JSONExtract. Closes #24606. #25900 (Kseniia Sumarokova).
  • Convert history file from readline format to replxx format. #25888 (Azat Khuzhin).
  • Fix an issue which can lead to intersecting parts after DROP PART or background deletion of an empty part. #25884 (alesapin).
  • Better handling of lost parts for ReplicatedMergeTree tables. Fixes rare inconsistencies in ReplicationQueue. Fixes #10368. #25820 (alesapin).
  • Allow starting clickhouse-client with unreadable working directory. #25817 (ianton-ru).
  • Fix “No available columns” error for Merge storage. #25801 (Azat Khuzhin).
  • MySQL Engine now supports the exchange of column comments between MySQL and ClickHouse. #25795 (Storozhuk Kostiantyn).
  • Fix inconsistent behaviour of GROUP BY constant on empty set. Closes #6842. #25786 (Kseniia Sumarokova).
  • Cancel already running merges in partition on DROP PARTITION and TRUNCATE for ReplicatedMergeTree. Resolves #17151. #25684 (tavplubix).
  • Support ENUM` data type for MaterializeMySQL. #25676 (Storozhuk Kostiantyn).
  • Support materialized and aliased columns in JOIN, close #13274. #25634 (Vladimir C).
  • Fix possible logical race condition between ALTER TABLE ... DETACH and background merges. #25605 (Azat Khuzhin).
  • Make NetworkReceiveElapsedMicroseconds metric to correctly include the time spent waiting for data from the client to INSERT. Close #9958. #25602 (alexey-milovidov).
  • Support TRUNCATE TABLE for S3 and HDFS. Close #25530. #25550 (Kseniia Sumarokova).
  • Support for dynamic reloading of config to change number of threads in pool for background jobs execution (merges, mutations, fetches). #25548 (Nikita Mikhaylov).
  • Allow extracting of non-string element as string using JSONExtract. This is for #25414. #25452 (Amos Bird).
  • Support regular expression in Database argument for StorageMerge. Close #776. #25064 (flynn).
  • Web UI: if the value looks like a URL, automatically generate a link. #25965 (alexey-milovidov).
  • Make sudo service clickhouse-server start to work on systems with systemd like Centos 8. Close #14298. Close #17799. #25921 (alexey-milovidov).

Bug Fixes

  • Fix incorrect SET ROLE in some cases. #26707 (Vitaly Baranov).
  • Fix potential nullptr dereference in window functions. Fix #25276. #26668 (Alexander Kuzmenkov).
  • Fix incorrect function names of groupBitmapAnd/Or/Xor. Fix #26557 (Amos Bird).
  • Fix crash in RabbitMQ shutdown in case RabbitMQ setup was not started. Closes #26504. #26529 (Kseniia Sumarokova).
  • Fix issues with CREATE DICTIONARY query if dictionary name or database name was quoted. Closes #26491. #26508 (Maksim Kita).
  • Fix broken name resolution after rewriting column aliases. Fix #26432. #26475 (Amos Bird).
  • Fix infinite non-joined block stream in partial_merge_join close #26325. #26374 (Vladimir C).
  • Fix possible crash when login as dropped user. Fix #26073. #26363 (Vitaly Baranov).
  • Fix optimize_distributed_group_by_sharding_key for multiple columns (leads to incorrect result w/ optimize_skip_unused_shards=1/allow_nondeterministic_optimize_skip_unused_shards=1 and multiple columns in sharding key expression). #26353 (Azat Khuzhin).
  • CAST from Date to DateTime (or DateTime64) was not using the timezone of the DateTime type. It can also affect the comparison between Date and DateTime. Inference of the common type for Date and DateTime also was not using the corresponding timezone. It affected the results of function if and array construction. Closes #24128. #24129 (Maksim Kita).
  • Fixed rare bug in lost replica recovery that may cause replicas to diverge. #26321 (tavplubix).
  • Fix zstd decompression in case there are escape sequences at the end of internal buffer. Closes #26013. #26314 (Kseniia Sumarokova).
  • Fix logical error on join with totals, close #26017. #26250 (Vladimir C).
  • Remove excessive newline in thread_name column in system.stack_trace table. Fix #24124. #26210 (alexey-milovidov).
  • Fix joinGet with LowCarinality columns, close #25993. #26118 (Vladimir C).
  • Fix possible crash in pointInPolygon if the setting validate_polygons is turned off. #26113 (alexey-milovidov).
  • Fix throwing exception when iterate over non-existing remote directory. #26087 (ianton-ru).
  • Fix rare server crash because of abort in ZooKeeper client. Fixes #25813. #26079 (alesapin).
  • Fix wrong thread count estimation for right subquery join in some cases. Close #24075. #26052 (Vladimir C).
  • Fixed incorrect sequence_id in MySQL protocol packets that ClickHouse sends on exception during query execution. It might cause MySQL client to reset connection to ClickHouse server. Fixes #21184. #26051 (tavplubix).
  • Fix possible mismatched header when using normal projection with PREWHERE. Fix #26020. #26038 (Amos Bird).
  • Fix formatting of type Map with integer keys to JSON. #25982 (Anton Popov).
  • Fix possible deadlock during query profiler stack unwinding. Fix #25968. #25970 (Maksim Kita).
  • Fix crash on call dictGet() with bad arguments. #25913 (Vitaly Baranov).
  • Fixed scram-sha-256 authentication for PostgreSQL engines. Closes #24516. #25906 (Kseniia Sumarokova).
  • Fix extremely long backoff for background tasks when the background pool is full. Fixes #25836. #25893 (alesapin).
  • Fix ARM exception handling with non default page size. Fixes #25512, #25044, #24901, #23183, #20221, #19703, #19028, #18391, #18121, #17994, #12483. #25854 (Maksim Kita).
  • Fix sharding_key from column w/o function for remote() (before select * from remote('127.1', system.one, dummy) leads to Unknown column: dummy, there are only columns . error). #25824 (Azat Khuzhin).
  • Fixed Not found column ... and Missing column ... errors when selecting from MaterializeMySQL. Fixes #23708, #24830, #25794. #25822 (tavplubix).
  • Fix optimize_skip_unused_shards_rewrite_in for non-UInt64 types (may select incorrect shards eventually or throw Cannot infer type of an empty tuple or Function tuple requires at least one argument). #25798 (Azat Khuzhin).
  • Fix rare bug with DROP PART query for ReplicatedMergeTree tables which can lead to error message Unexpected merged part intersecting drop range. #25783 (alesapin).
  • Fix bug in TTL with GROUP BY expression which refuses to execute TTL after first execution in part. #25743 (alesapin).
  • Allow StorageMerge to access tables with aliases. Closes #6051. #25694 (Kseniia Sumarokova).
  • Fix slow dict join in some cases, close #24209. #25618 (Vladimir C).
  • Fix ALTER MODIFY COLUMN of columns, which participates in TTL expressions. #25554 (Anton Popov).
  • Fix assertion in PREWHERE with non-UInt8 type, close #19589. #25484 (Vladimir C).
  • Fix some fuzzed msan crash. Fixes #22517. #26428 (Nikolai Kochetov).
  • Update chown cmd check in clickhouse-server docker entrypoint. It fixes error ‘cluster pod restart failed (or timeout)’ on kubernetes. #26545 (Ky Li).

ClickHouse release v21.7, 2021-07-09

Backward Incompatible Change

  • Improved performance of queries with explicitly defined large sets. Added compatibility setting legacy_column_name_of_tuple_literal. It makes sense to set it to true, while doing rolling update of cluster from version lower than 21.7 to any higher version. Otherwise distributed queries with explicitly defined sets at IN clause may fail during update. #25371 (Anton Popov).
  • Forward/backward incompatible change of maximum buffer size in clickhouse-keeper (an experimental alternative to ZooKeeper). Better to do it now (before production), than later. #25421 (alesapin).

New Feature

  • Support configuration in YAML format as alternative to XML. This closes #3607. #21858 (BoloniniD).
  • Provides a way to restore replicated table when the data is (possibly) present, but the ZooKeeper metadata is lost. Resolves #13458. #13652 (Mike Kot).
  • Support structs and maps in Arrow/Parquet/ORC and dictionaries in Arrow input/output formats. Present new setting output_format_arrow_low_cardinality_as_dictionary. #24341 (Kruglov Pavel).
  • Added support for Array type in dictionaries. #25119 (Maksim Kita).
  • Added function bitPositionsToArray. Closes #23792. Author [Kevin Wan] (@MaxWk). #25394 (Maksim Kita).
  • Added function dateName to return names like ‘Friday’ or ‘April’. Author [Daniil Kondratyev] (@dankondr). #25372 (Maksim Kita).
  • Add toJSONString function to serialize columns to their JSON representations. #25164 (Amos Bird).
  • Now query_log has two new columns: initial_query_start_time, initial_query_start_time_microsecond that record the starting time of a distributed query if any. #25022 (Amos Bird).
  • Add aggregate function segmentLengthSum. #24250 (flynn).
  • Add a new boolean setting prefer_global_in_and_join which defaults all IN/JOIN as GLOBAL IN/JOIN. #23434 (Amos Bird).
  • Support ALTER DELETE queries for Join table engine. #23260 (foolchi).
  • Add quantileBFloat16 aggregate function as well as the corresponding quantilesBFloat16 and medianBFloat16. It is very simple and fast quantile estimator with relative error not more than 0.390625%. This closes #16641. #23204 (Ivan Novitskiy).
  • Implement sequenceNextNode() function useful for flow analysis. #19766 (achimbab).

Experimental Feature

Performance Improvement

  • Added optimization that transforms some functions to reading of subcolumns to reduce amount of read data. E.g., statement col IS NULL is transformed to reading of subcolumn col.null. Optimization can be enabled by setting optimize_functions_to_subcolumns which is currently off by default. #24406 (Anton Popov).
  • Rewrite more columns to possible alias expressions. This may enable better optimization, such as projections. #24405 (Amos Bird).
  • Index of type bloom_filter can be used for expressions with hasAny function with constant arrays. This closes: #24291. #24900 (Vasily Nemkov).
  • Add exponential backoff to reschedule read attempt in case RabbitMQ queues are empty. (ClickHouse has support for importing data from RabbitMQ). Closes #24340. #24415 (Kseniia Sumarokova).

Improvement

  • Allow to limit bandwidth for replication. Add two Replicated*MergeTree settings: max_replicated_fetches_network_bandwidth and max_replicated_sends_network_bandwidth which allows to limit maximum speed of replicated fetches/sends for table. Add two server-wide settings (in default user profile): max_replicated_fetches_network_bandwidth_for_server and max_replicated_sends_network_bandwidth_for_server which limit maximum speed of replication for all tables. The settings are not followed perfectly accurately. Turned off by default. Fixes #1821. #24573 (alesapin).
  • Resource constraints and isolation for ODBC and Library bridges. Use separate clickhouse-bridge group and user for bridge processes. Set oom_score_adj so the bridges will be first subjects for OOM killer. Set set maximum RSS to 1 GiB. Closes #23861. #25280 (Kseniia Sumarokova).
  • Add standalone clickhouse-keeper symlink to the main clickhouse binary. Now it’s possible to run coordination without the main clickhouse server. #24059 (alesapin).
  • Use global settings for query to VIEW. Fixed the behavior when queries to VIEW use local settings, that leads to errors if setting on CREATE VIEW and SELECT were different. As for now, VIEW won’t use these modified settings, but you can still pass additional settings in SETTINGS section of CREATE VIEW query. Close #20551. #24095 (Vladimir).
  • On server start, parts with incorrect partition ID would not be ever removed, but always detached. #25070. #25166 (Nikolai Kochetov).
  • Increase size of background schedule pool to 128 (background_schedule_pool_size setting). It allows avoiding replication queue hung on slow zookeeper connection. #25072 (alesapin).
  • Add merge tree setting max_parts_to_merge_at_once which limits the number of parts that can be merged in the background at once. Doesn’t affect OPTIMIZE FINAL query. Fixes #1820. #24496 (alesapin).
  • Allow NOT IN operator to be used in partition pruning. #24894 (Amos Bird).
  • Recognize IPv4 addresses like 127.0.1.1 as local. This is controversial and closes #23504. Michael Filimonov will test this feature. #24316 (alexey-milovidov).
  • ClickHouse database created with MaterializeMySQL (it is an experimental feature) now contains all column comments from the MySQL database that materialized. #25199 (Storozhuk Kostiantyn).
  • Add settings (connection_auto_close/connection_max_tries/connection_pool_size) for MySQL storage engine. #24146 (Azat Khuzhin).
  • Improve startup time of Distributed engine. #25663 (Azat Khuzhin).
  • Improvement for Distributed tables. Drop replicas from dirname for internal_replication=true (allows INSERT into Distributed with cluster from any number of replicas, before only 15 replicas was supported, everything more will fail with ENAMETOOLONG while creating directory for async blocks). #25513 (Azat Khuzhin).
  • Added support Interval type for LowCardinality. It is needed for intermediate values of some expressions. Closes #21730. #25410 (Vladimir).
  • Add == operator on time conditions for sequenceMatch and sequenceCount functions. For eg: sequenceMatch(‘(?1)(?t==1)(?2)’)(time, data = 1, data = 2). #25299 (Christophe Kalenzaga).
  • Add settings http_max_fields, http_max_field_name_size, http_max_field_value_size. #25296 (Ivan).
  • Add support for function if with Decimal and Int types on its branches. This closes #20549. This closes #10142. #25283 (alexey-milovidov).
  • Update prompt in clickhouse-client and display a message when reconnecting. This closes #10577. #25281 (alexey-milovidov).
  • Correct memory tracking in aggregate function topK. This closes #25259. #25260 (alexey-milovidov).
  • Fix topLevelDomain for IDN hosts (i.e. example.рф), before it returns empty string for such hosts. #25103 (Azat Khuzhin).
  • Detect Linux kernel version at runtime (for worked nested epoll, that is required for async_socket_for_remote/use_hedged_requests, otherwise remote queries may stuck). #25067 (Azat Khuzhin).
  • For distributed query, when optimize_skip_unused_shards=1, allow to skip shard with condition like (sharding key) IN (one-element-tuple). (Tuples with many elements were supported. Tuple with single element did not work because it is parsed as literal). #24930 (Amos Bird).
  • Improved log messages of S3 errors, no more double whitespaces in case of empty keys and buckets. #24897 (Vladimir Chebotarev).
  • Some queries require multi-pass semantic analysis. Try reusing built sets for IN in this case. #24874 (Amos Bird).
  • Respect max_distributed_connections for insert_distributed_sync (otherwise for huge clusters and sync insert it may run out of max_thread_pool_size). #24754 (Azat Khuzhin).
  • Avoid hiding errors like Limit for rows or bytes to read exceeded for scalar subqueries. #24545 (nvartolomei).
  • Make String-to-Int parser stricter so that toInt64('+') will throw. #24475 (Amos Bird).
  • If SSD_CACHE is created with DDL query, it can be created only inside user_files directory. #24466 (Maksim Kita).
  • PostgreSQL support for specifying non default schema for insert queries. Closes #24149. #24413 (Kseniia Sumarokova).
  • Fix IPv6 addresses resolving (i.e. fixes select * from remote('[::1]', system.one)). #24319 (Azat Khuzhin).
  • Fix trailing whitespaces in FROM clause with subqueries in multiline mode, and also changes the output of the queries slightly in a more human friendly way. #24151 (Azat Khuzhin).
  • Improvement for Distributed tables. Add ability to split distributed batch on failures (i.e. due to memory limits, corruptions), under distributed_directory_monitor_split_batch_on_failure (OFF by default). #23864 (Azat Khuzhin).
  • Handle column name clashes for Join table engine. Closes #20309. #23769 (Vladimir).
  • Display progress for File table engine in clickhouse-local and on INSERT query in clickhouse-client when data is passed to stdin. Closes #18209. #23656 (Kseniia Sumarokova).
  • Bugfixes and improvements of clickhouse-copier. Allow to copy tables with different (but compatible schemas). Closes #9159. Added test to copy ReplacingMergeTree. Closes #22711. Support TTL on columns and Data Skipping Indices. It simply removes it to create internal Distributed table (underlying table will have TTL and skipping indices). Closes #19384. Allow to copy MATERIALIZED and ALIAS columns. There are some cases in which it could be helpful (e.g. if this column is in PRIMARY KEY). Now it could be allowed by setting allow_to_copy_alias_and_materialized_columns property to true in task configuration. Closes #9177. Closes [#11007] (https://github.com/ClickHouse/ClickHouse/issues/11007). Closes #9514. Added a property allow_to_drop_target_partitions in task configuration to drop partition in original table before moving helping tables. Closes #20957. Get rid of OPTIMIZE DEDUPLICATE query. This hack was needed, because ALTER TABLE MOVE PARTITION was retried many times and plain MergeTree tables don’t have deduplication. Closes #17966. Write progress to ZooKeeper node on path task_path + /status in JSON format. Closes #20955. Support for ReplicatedTables without arguments. Closes #24834 .#23518 (Nikita Mikhaylov).
  • Added sleep with backoff between read retries from S3. #23461 (Vladimir Chebotarev).
  • Respect insert_allow_materialized_columns (allows materialized columns) for INSERT into Distributed table. #23349 (Azat Khuzhin).
  • Add ability to push down LIMIT for distributed queries. #23027 (Azat Khuzhin).
  • Fix zero-copy replication with several S3 volumes (Fixes #22679). #22864 (ianton-ru).
  • Resolve the actual port number bound when a user requests any available port from the operating system to show it in the log message. #25569 (bnaecker).
  • Fixed case, when sometimes conversion of postgres arrays resulted in String data type, not n-dimensional array, because attndims works incorrectly in some cases. Closes #24804. #25538 (Kseniia Sumarokova).
  • Fix convertion of DateTime with timezone for MySQL, PostgreSQL, ODBC. Closes #5057. #25528 (Kseniia Sumarokova).
  • Distinguish KILL MUTATION for different tables (fixes unexpected Cancelled mutating parts error). #25025 (Azat Khuzhin).
  • Allow to declare S3 disk at root of bucket (S3 virtual filesystem is an experimental feature under development). #24898 (Vladimir Chebotarev).
  • Enable reading of subcolumns (e.g. components of Tuples) for distributed tables. #24472 (Anton Popov).
  • A feature for MySQL compatibility protocol: make user function to return correct output. Closes #25697. #25697 (sundyli).

Bug Fix

  • Improvement for backward compatibility. Use old modulo function version when used in partition key. Closes #23508. #24157 (Kseniia Sumarokova).
  • Fix extremely rare bug on low-memory servers which can lead to the inability to perform merges without restart. Possibly fixes #24603. #24872 (alesapin).
  • Fix extremely rare error Tagging already tagged part in replication queue during concurrent alter move/replace partition. Possibly fixes #22142. #24961 (alesapin).
  • Fix potential crash when calculating aggregate function states by aggregation of aggregate function states of other aggregate functions (not a practical use case). See #24523. #25015 (alexey-milovidov).
  • Fixed the behavior when query SYSTEM RESTART REPLICA or SYSTEM SYNC REPLICA does not finish. This was detected on server with extremely low amount of RAM. #24457 (Nikita Mikhaylov).
  • Fix bug which can lead to ZooKeeper client hung inside clickhouse-server. #24721 (alesapin).
  • If ZooKeeper connection was lost and replica was cloned after restoring the connection, its replication queue might contain outdated entries. Fixed failed assertion when replication queue contains intersecting virtual parts. It may rarely happen if some data part was lost. Print error in log instead of terminating. #24777 (tavplubix).
  • Fix lost WHERE condition in expression-push-down optimization of query plan (setting query_plan_filter_push_down = 1 by default). Fixes #25368. #25370 (Nikolai Kochetov).
  • Fix bug which can lead to intersecting parts after merges with TTL: Part all_40_40_0 is covered by all_40_40_1 but should be merged into all_40_41_1. This shouldn't happen often.. #25549 (alesapin).
  • On ZooKeeper connection loss ReplicatedMergeTree table might wait for background operations to complete before trying to reconnect. It’s fixed, now background operations are stopped forcefully. #25306 (tavplubix).
  • Fix error Key expression contains comparison between inconvertible types for queries with ARRAY JOIN in case if array is used in primary key. Fixes #8247. #25546 (Anton Popov).
  • Fix wrong totals for query WITH TOTALS and WITH FILL. Fixes #20872. #25539 (Anton Popov).
  • Fix data race when querying system.clusters while reloading the cluster configuration at the same time. #25737 (Amos Bird).
  • Fixed No such file or directory error on moving Distributed table between databases. Fixes #24971. #25667 (tavplubix).
  • REPLACE PARTITION might be ignored in rare cases if the source partition was empty. It’s fixed. Fixes #24869. #25665 (tavplubix).
  • Fixed a bug in Replicated database engine that might rarely cause some replica to skip enqueued DDL query. #24805 (tavplubix).
  • Fix null pointer dereference in EXPLAIN AST without query. #25631 (Nikolai Kochetov).
  • Fix waiting of automatic dropping of empty parts. It could lead to full filling of background pool and stuck of replication. #23315 (Anton Popov).
  • Fix restore of a table stored in S3 virtual filesystem (it is an experimental feature not ready for production). #25601 (ianton-ru).
  • Fix nullptr dereference in Arrow format when using Decimal256. Add Decimal256 support for Arrow format. #25531 (Kruglov Pavel).
  • Fix excessive underscore before the names of the preprocessed configuration files. #25431 (Vitaly Baranov).
  • A fix for clickhouse-copier tool: Fix segfault when sharding_key is absent in task config for copier. #25419 (Nikita Mikhaylov).
  • Fix REPLACE column transformer when used in DDL by correctly quoting the formated query. This fixes #23925. #25391 (Amos Bird).
  • Fix the possibility of non-deterministic behaviour of the quantileDeterministic function and similar. This closes #20480. #25313 (alexey-milovidov).
  • Support SimpleAggregateFunction(LowCardinality) for SummingMergeTree. Fixes #25134. #25300 (Nikolai Kochetov).
  • Fix logical error with exception message “Cannot sum Array/Tuple in min/maxMap”. #25298 (Kruglov Pavel).
  • Fix error Bad cast from type DB::ColumnLowCardinality to DB::ColumnVector<char8_t> for queries where LowCardinality argument was used for IN (this bug appeared in 21.6). Fixes #25187. #25290 (Nikolai Kochetov).
  • Fix incorrect behaviour of joinGetOrNull with not-nullable columns. This fixes #24261. #25288 (Amos Bird).
  • Fix incorrect behaviour and UBSan report in big integers. In previous versions CAST(1e19 AS UInt128) returned zero. #25279 (alexey-milovidov).
  • Fixed an error which occurred while inserting a subset of columns using CSVWithNames format. Fixes #25129. #25169 (Nikita Mikhaylov).
  • Do not use table’s projection for SELECT with FINAL. It is not supported yet. #25163 (Amos Bird).
  • Fix possible parts loss after updating up to 21.5 in case table used UUID in partition key. (It is not recommended to use UUID in partition key). Fixes #25070. #25127 (Nikolai Kochetov).
  • Fix crash in query with cross join and joined_subquery_requires_alias = 0. Fixes #24011. #25082 (Nikolai Kochetov).
  • Fix bug with constant maps in mapContains function that lead to error empty column was returned by function mapContains. Closes #25077. #25080 (Kruglov Pavel).
  • Remove possibility to create tables with columns referencing themselves like a UInt32 ALIAS a + 1 or b UInt32 MATERIALIZED b. Fixes #24910, #24292. #25059 (alesapin).
  • Fix wrong result when using aggregate projection with not empty GROUP BY key to execute query with GROUP BY by empty key. #25055 (Amos Bird).
  • Fix serialization of splitted nested messages in Protobuf format. This PR fixes #24647. #25000 (Vitaly Baranov).
  • Fix limit/offset settings for distributed queries (ignore on the remote nodes). #24940 (Azat Khuzhin).
  • Fix possible heap-buffer-overflow in Arrow format. #24922 (Kruglov Pavel).
  • Fixed possible error ‘Cannot read from istream at offset 0’ when reading a file from DiskS3 (S3 virtual filesystem is an experimental feature under development that should not be used in production). #24885 (Pavel Kovalenko).
  • Fix “Missing columns” exception when joining Distributed Materialized View. #24870 (Azat Khuzhin).
  • Allow NULL values in postgresql compatibility protocol. Closes #22622. #24857 (Kseniia Sumarokova).
  • Fix bug when exception Mutation was killed can be thrown to the client on mutation wait when mutation not loaded into memory yet. #24809 (alesapin).
  • Fixed bug in deserialization of random generator state with might cause some data types such as AggregateFunction(groupArraySample(N), T)) to behave in a non-deterministic way. #24538 (tavplubix).
  • Disallow building uniqXXXXStates of other aggregation states. #24523 (Raúl Marín). Then allow it back by actually eliminating the root cause of the related issue. (alexey-milovidov).
  • Fix usage of tuples in CREATE .. AS SELECT queries. #24464 (Anton Popov).
  • Fix computation of total bytes in Buffer table. In current ClickHouse version total_writes.bytes counter decreases too much during the buffer flush. It leads to counter overflow and totalBytes return something around 17.44 EB some time after the flush. #24450 (DimasKovas).
  • Fix incorrect information about the monotonicity of toWeek function. This fixes #24422 . This bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/5212 , and was exposed later by smarter partition pruner. #24446 (Amos Bird).
  • When user authentication is managed by LDAP. Fixed potential deadlock that can happen during LDAP role (re)mapping, when LDAP group is mapped to a nonexistent local role. #24431 (Denis Glazachev).
  • In “multipart/form-data” message consider the CRLF preceding a boundary as part of it. Fixes #23905. #24399 (Ivan).
  • Fix drop partition with intersect fake parts. In rare cases there might be parts with mutation version greater than current block number. #24321 (Amos Bird).
  • Fixed a bug in moving Materialized View from Ordinary to Atomic database (RENAME TABLE query). Now inner table is moved to new database together with Materialized View. Fixes #23926. #24309 (tavplubix).
  • Allow empty HTTP headers. Fixes #23901. #24285 (Ivan).
  • Correct processing of mutations (ALTER UPDATE/DELETE) in Memory tables. Closes #24274. #24275 (flynn).
  • Make column LowCardinality property in JOIN output the same as in the input, close #23351, close #20315. #24061 (Vladimir).
  • A fix for Kafka tables. Fix the bug in failover behavior when Engine = Kafka was not able to start consumption if the same consumer had an empty assignment previously. Closes #21118. #21267 (filimonov).

Build/Testing/Packaging Improvement

  • Add darwin-aarch64 (Mac M1 / Apple Silicon) builds in CI #25560 (Ivan) and put the links to the docs and website (alexey-milovidov).
  • Adds cross-platform embedding of binary resources into executables. It works on Illumos. #25146 (bnaecker).
  • Add join related options to stress tests to improve fuzzing. #25200 (Vladimir).
  • Enable build with s3 module in osx #25217. #25218 (kevin wan).
  • Add integration test cases to cover JDBC bridge. #25047 (Zhichun Wu).
  • Integration tests configuration has special treatment for dictionaries. Removed remaining dictionaries manual setup. #24728 (Ilya Yatsishin).
  • Add libfuzzer tests for YAMLParser class. #24480 (BoloniniD).
  • Ubuntu 20.04 is now used to run integration tests, docker-compose version used to run integration tests is updated to 1.28.2. Environment variables now take effect on docker-compose. Rework test_dictionaries_all_layouts_separate_sources to allow parallel run. #20393 (Ilya Yatsishin).
  • Fix TOCTOU error in installation script. #25277 (alexey-milovidov).

ClickHouse release 21.6, 2021-06-05

Upgrade Notes

  • zstd compression library is updated to v1.5.0. You may get messages about “checksum does not match” in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour.
  • The setting compile_expressions is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off.
  • Values of UUID type cannot be compared with integer. For example, instead of writing uuid != 0 type uuid != '00000000-0000-0000-0000-000000000000'.

New Feature

  • Add Postgres-like cast operator (::). E.g.: [1, 2]::Array(UInt8), 0.1::Decimal(4, 4), number::UInt16. #23871 (Anton Popov).
  • Make big integers production ready. Add support for UInt128 data type. Fix known issues with the Decimal256 data type. Support big integers in dictionaries. Support gcd/lcm functions for big integers. Support big integers in array search and conditional functions. Support LowCardinality(UUID). Support big integers in generateRandom table function and clickhouse-obfuscator. Fix error with returning UUID from scalar subqueries. This fixes #7834. This fixes #23936. This fixes #4176. This fixes #24018. Backward incompatible change: values of UUID type cannot be compared with integer. For example, instead of writing uuid != 0 type uuid != '00000000-0000-0000-0000-000000000000'. #23631 (alexey-milovidov).
  • Support Array data type for inserting and selecting data in Arrow, Parquet and ORC formats. #21770 (taylor12805).
  • Implement table comments. Closes #23225. #23548 (flynn).
  • Support creating dictionaries with DDL queries in clickhouse-local. Closes #22354. Added support for DETACH DICTIONARY PERMANENTLY. Added support for EXCHANGE DICTIONARIES for Atomic database engine. Added support for moving dictionaries between databases using RENAME DICTIONARY. #23436 (Maksim Kita).
  • Add aggregate function uniqTheta to support Theta Sketch in ClickHouse. #23894. #22609 (Ping Yu).
  • Add function splitByRegexp. #24077 (abel-cheng).
  • Add function arrayProduct which accept an array as the parameter, and return the product of all the elements in array. Closes #21613. #23782 (Maksim Kita).
  • Add thread_name column in system.stack_trace. This closes #23256. #24124 (abel-cheng).
  • If insert_null_as_default = 1, insert default values instead of NULL in INSERT ... SELECT and INSERT ... SELECT ... UNION ALL ... queries. Closes #22832. #23524 (Kseniia Sumarokova).
  • Add support for progress indication in clickhouse-local with --progress option. #23196 (Egor Savin).
  • Add support for HTTP compression (determined by Content-Encoding HTTP header) in http dictionary source. This fixes #8912. #23946 (FArthur-cmd).
  • Added SYSTEM QUERY RELOAD MODEL, SYSTEM QUERY RELOAD MODELS. Closes #18722. #23182 (Maksim Kita).
  • Add setting json (boolean, 0 by default) for EXPLAIN PLAN query. When enabled, query output will be a single JSON row. It is recommended to use TSVRaw format to avoid unnecessary escaping. #23082 (Nikolai Kochetov).
  • Add setting indexes (boolean, disabled by default) to EXPLAIN PIPELINE query. When enabled, shows used indexes, number of filtered parts and granules for every index applied. Supported for MergeTree* tables. #22352 (Nikolai Kochetov).
  • LDAP: implemented user DN detection functionality to use when mapping Active Directory groups to ClickHouse roles. #22228 (Denis Glazachev).
  • New aggregate function deltaSumTimestamp for summing the difference between consecutive rows while maintaining ordering during merge by storing timestamps. #21888 (Russ Frank).
  • Added less secure IMDS credentials provider for S3 which works under docker correctly. #21852 (Vladimir Chebotarev).
  • Add back indexHint function. This is for #21238. This reverts #9542. This fixes #9540. #21304 (Amos Bird).

Experimental Feature

Performance Improvement

  • Enable compile_expressions setting by default. When this setting enabled, compositions of simple functions and operators will be compiled to native code with LLVM at runtime. #8482 (Maksim Kita, alexey-milovidov). Note: if you feel in trouble, turn this option off.
  • Update re2 library. Performance of regular expressions matching is improved. Also this PR adds compatibility with gcc-11. #24196 (Raúl Marín).
  • ORC input format reading by stripe instead of reading entire table into memory by once which is cost memory when file size is huge. #23102 (Chao Ma).
  • Fusion of aggregate functions sum, count and avg in a query into single aggregate function. The optimization is controlled with the optimize_fuse_sum_count_avg setting. This is implemented with a new aggregate function sumCount. This function returns a tuple of two fields: sum and count. #21337 (hexiaoting).
  • Update zstd to v1.5.0. The performance of compression is improved for single digits percentage. #24135 (Raúl Marín). Note: you may get messages about “checksum does not match” in replication. These messages are expected due to update of compression algorithm and you can ignore them.
  • Improved performance of Buffer tables: do not acquire lock for total_bytes/total_rows for Buffer engine. #24066 (Azat Khuzhin).
  • Preallocate support for hashed/sparse_hashed dictionaries is returned. #23979 (Azat Khuzhin).
  • Enable async_socket_for_remote by default (lower amount of threads in querying Distributed tables with large fanout). #23683 (Nikolai Kochetov).

Improvement

  • Add _partition_value virtual column to MergeTree table family. It can be used to prune partition in a deterministic way. It’s needed to implement partition matcher for mutations. #23673 (Amos Bird).
  • Added region parameter for S3 storage and disk. #23846 (Vladimir Chebotarev).
  • Allow configuring different log levels for different logging channels. Closes #19569. #23857 (filimonov).
  • Keep default timezone on DateTime operations if it was not provided explicitly. For example, if you add one second to a value of DateTime type without timezone it will remain DateTime without timezone. In previous versions the value of default timezone was placed to the returned data type explicitly so it becomes DateTime(‘something’). This closes #4854. #23392 (alexey-milovidov).
  • Allow user to specify empty string instead of database name for MySQL storage. Default database will be used for queries. In previous versions it was working for SELECT queries and not support for INSERT was also added. This closes #19281. This can be useful working with Sphinx or other MySQL-compatible foreign databases. #23319 (alexey-milovidov).
  • Fixed quantile(s)TDigest. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. #23314 (Vladimir Chebotarev).
  • Function now64 now supports optional timezone argument. #24091 (Vasily Nemkov).
  • Fix the case when a progress bar in interactive mode in clickhouse-client that appear in the middle of the data may rewrite some parts of visible data in terminal. This closes #19283. #23050 (alexey-milovidov).
  • Fix crash when memory allocation fails in simdjson. https://github.com/simdjson/simdjson/pull/1567 . Mark as improvement because it’s a very rare bug. #24147 (Amos Bird).
  • Preserve dictionaries until storage shutdown (this will avoid possible external dictionary 'DICT' not found errors at server shutdown during final flush of the Buffer engine). #24068 (Azat Khuzhin).
  • Flush Buffer tables before shutting down tables (within one database), to avoid discarding blocks due to underlying table had been already detached (and Destination table default.a_data_01870 doesn't exist. Block of data is discarded error in the log). #24067 (Azat Khuzhin).
  • Now prefer_column_name_to_alias = 1 will also favor column names for group by, having and order by. This fixes #23882. #24022 (Amos Bird).
  • Add support for ORDER BY WITH FILL with DateTime64. #24016 (kevin wan).
  • Enable DateTime64 to be a version column in ReplacingMergeTree. #23992 (kevin wan).
  • Log information about OS name, kernel version and CPU architecture on server startup. #23988 (Azat Khuzhin).
  • Support specifying table schema for postgresql dictionary source. Closes #23958. #23980 (Kseniia Sumarokova).
  • Add hints for names of Enum elements (suggest names in case of typos). Closes #17112. #23919 (flynn).
  • Measure found rate (the percentage for which the value was found) for dictionaries (see found_rate in system.dictionaries). #23916 (Azat Khuzhin).
  • Allow to add specific queue settings via table settng rabbitmq_queue_settings_list. (Closes #23737 and #23918). Allow user to control all RabbitMQ setup: if table setting rabbitmq_queue_consume is set to 1 - RabbitMQ table engine will only connect to specified queue and will not perform any RabbitMQ consumer-side setup like declaring exchange, queues, bindings. (Closes #21757). Add proper cleanup when RabbitMQ table is dropped - delete queues, which the table has declared and all bound exchanges - if they were created by the table. #23887 (Kseniia Sumarokova).
  • Add broken_data_files/broken_data_compressed_bytes into system.distribution_queue. Add metric for number of files for asynchronous insertion into Distributed tables that has been marked as broken (BrokenDistributedFilesToInsert). #23885 (Azat Khuzhin).
  • Querying system.tables does not go to ZooKeeper anymore. #23793 (Fuwang Hu).
  • Respect lock_acquire_timeout_for_background_operations for OPTIMIZE queries. #23623 (Azat Khuzhin).
  • Possibility to change S3 disk settings in runtime via new SYSTEM RESTART DISK SQL command. #23429 (Pavel Kovalenko).
  • If user applied a misconfiguration by mistakenly setting max_distributed_connections to value zero, every query to a Distributed table will throw exception with a message containing “logical error”. But it’s really an expected behaviour, not a logical error, so the exception message was slightly incorrect. It also triggered checks in our CI enviroment that ensures that no logical errors ever happen. Instead we will treat max_distributed_connections misconfigured to zero as the minimum possible value (one). #23348 (Azat Khuzhin).
  • Disable min_bytes_to_use_mmap_io by default. #23322 (Azat Khuzhin).
  • Support LowCardinality nullability with join_use_nulls, close #15101. #23237 (vdimir).
  • Added possibility to restore MergeTree parts to detached directory for S3 disk. #23112 (Pavel Kovalenko).
  • Retries on HTTP connection drops in S3. #22988 (Vladimir Chebotarev).
  • Add settings external_storage_max_read_rows and external_storage_max_read_rows for MySQL table engine, dictionary source and MaterializeMySQL minor data fetches. #22697 (TCeason).
  • MaterializeMySQL (experimental feature): Previously, MySQL 5.7.9 was not supported due to SQL incompatibility. Now leave MySQL parameter verification to the MaterializeMySQL. #23413 (TCeason).
  • Enable reading of subcolumns for distributed tables. #24472 (Anton Popov).
  • Fix usage of tuples in CREATE .. AS SELECT queries. #24464 (Anton Popov).
  • Support for Parquet format in Kafka tables. #23412 (Chao Ma).

Bug Fix

  • Use old modulo function version when used in partition key and primary key. Closes #23508. #24157 (Kseniia Sumarokova). It was a source of backward incompatibility in previous releases.
  • Fixed the behavior when query SYSTEM RESTART REPLICA or SYSTEM SYNC REPLICA is being processed infinitely. This was detected on server with extremely little amount of RAM. #24457 (Nikita Mikhaylov).
  • Fix incorrect monotonicity of toWeek function. This fixes #24422 . This bug was introduced in #5212, and was exposed later by smarter partition pruner. #24446 (Amos Bird).
  • Fix drop partition with intersect fake parts. In rare cases there might be parts with mutation version greater than current block number. #24321 (Amos Bird).
  • Fixed a bug in moving Materialized View from Ordinary to Atomic database (RENAME TABLE query). Now inner table is moved to new database together with Materialized View. Fixes #23926. #24309 (tavplubix).
  • Allow empty HTTP headers in client requests. Fixes #23901. #24285 (Ivan).
  • Set max_threads = 1 to fix mutation fail of Memory tables. Closes #24274. #24275 (flynn).
  • Fix typo in implementation of Memory tables, this bug was introduced at #15127. Closes #24192. #24193 (张中南).
  • Fix abnormal server termination due to HDFS becoming not accessible during query execution. Closes #24117. #24191 (Kseniia Sumarokova).
  • Fix crash on updating of Nested column with const condition. #24183 (hexiaoting).
  • Fix race condition which could happen in RBAC under a heavy load. This PR fixes #24090, #24134,. #24176 (Vitaly Baranov).
  • Fix a rare bug that could lead to a partially initialized table that can serve write requests (insert/alter/so on). Now such tables will be in readonly mode. #24122 (alesapin).
  • Fix an issue: EXPLAIN PIPELINE with SELECT xxx FINAL showed a wrong pipeline. (hexiaoting).
  • Fixed using const DateTime value vs DateTime64 column in WHERE. #24100 (Vasily Nemkov).
  • Fix crash in merge JOIN, closes #24010. #24013 (vdimir).
  • Some ALTER PARTITION queries might cause Part A intersects previous part B and Unexpected merged part C intersecting drop range D errors in replication queue. It’s fixed. Fixes #23296. #23997 (tavplubix).
  • Fix SIGSEGV for external GROUP BY and overflow row (i.e. queries like SELECT FROM GROUP BY WITH TOTALS SETTINGS max_bytes_before_external_group_by>0, max_rows_to_group_by>0, group_by_overflow_mode='any', totals_mode='before_having'). #23962 (Azat Khuzhin).
  • Fix keys metrics accounting for CACHE dictionary with duplicates in the source (leads to DictCacheKeysRequestedMiss overflows). #23929 (Azat Khuzhin).
  • Fix implementation of connection pool of PostgreSQL engine. Closes #23897. #23909 (Kseniia Sumarokova).
  • Fix distributed_group_by_no_merge = 2 with GROUP BY and aggregate function wrapped into regular function (had been broken in #23546). Throw exception in case of someone trying to use distributed_group_by_no_merge = 2 with window functions. Disable optimize_distributed_group_by_sharding_key for queries with window functions. #23906 (Azat Khuzhin).
  • A fix for s3 table function: better handling of HTTP errors. Response bodies of HTTP errors were being ignored earlier. #23844 (Vladimir Chebotarev).
  • A fix for s3 table function: better handling of URI’s. Fixed an incompatibility with URLs containing + symbol, data with such keys could not be read previously. #23822 (Vladimir Chebotarev).
  • Fix error Can't initialize pipeline with empty pipe for queries with GLOBAL IN/JOIN and use_hedged_requests. Fixes #23431. #23805 (Nikolai Kochetov).
  • Fix CLEAR COLUMN does not work when it is referenced by materialized view. Close #23764. #23781 (flynn).
  • Fix heap use after free when reading from HDFS if Values format is used. #23761 (Kseniia Sumarokova).
  • Avoid possible “Cannot schedule a task” error (in case some exception had been occurred) on INSERT into Distributed. #23744 (Azat Khuzhin).
  • Fixed a bug in recovery of staled ReplicatedMergeTree replica. Some metadata updates could be ignored by staled replica if ALTER query was executed during downtime of the replica. #23742 (tavplubix).
  • Fix a bug with Join and WITH TOTALS, close #17718. #23549 (vdimir).
  • Fix possible Block structure mismatch error for queries with UNION which could possibly happen after filter-pushdown optimization. Fixes #23029. #23359 (Nikolai Kochetov).
  • Add type conversion when the setting optimize_skip_unused_shards_rewrite_in is enabled. This fixes MSan report. #23219 (Azat Khuzhin).
  • Add a missing check when updating nested subcolumns, close issue: #22353. #22503 (hexiaoting).

Build/Testing/Packaging Improvement

  • Support building on Illumos. #24144. Adds support for building on Solaris-derived operating systems. #23746 (bnaecker).
  • Add more benchmarks for hash tables, including the Swiss Table from Google (that appeared to be slower than ClickHouse hash map in our specific usage scenario). #24111 (Maksim Kita).
  • Update librdkafka 1.6.0-RC3 to 1.6.1. #23874 (filimonov).
  • Always enable asynchronous-unwind-tables explicitly. It may fix query profiler on AArch64. #23602 (alexey-milovidov).
  • Avoid possible build dependency on locale and filesystem order. This allows reproducible builds. #23600 (alexey-milovidov).
  • Remove a source of nondeterminism from build. Now builds at different point of time will produce byte-identical binaries. Partially addressed #22113. #23559 (alexey-milovidov).
  • Add simple tool for benchmarking (Zoo)Keeper. #23038 (alesapin).

ClickHouse release 21.5, 2021-05-20

Backward Incompatible Change

  • Change comparison of integers and floating point numbers when integer is not exactly representable in the floating point data type. In new version comparison will return false as the rounding error will occur. Example: 9223372036854775808.0 != 9223372036854775808, because the number 9223372036854775808 is not representable as floating point number exactly (and 9223372036854775808.0 is rounded to 9223372036854776000.0). But in previous version the comparison will return as the numbers are equal, because if the floating point number 9223372036854776000.0 get converted back to UInt64, it will yield 9223372036854775808. For the reference, the Python programming language also treats these numbers as equal. But this behaviour was dependend on CPU model (different results on AMD64 and AArch64 for some out-of-range numbers), so we make the comparison more precise. It will treat int and float numbers equal only if int is represented in floating point type exactly. #22595 (alexey-milovidov).
  • Remove support for argMin and argMax for single Tuple argument. The code was not memory-safe. The feature was added by mistake and it is confusing for people. These functions can be reintroduced under different names later. This fixes #22384 and reverts #17359. #23393 (alexey-milovidov).

New Feature

  • Added functions dictGetChildren(dictionary, key), dictGetDescendants(dictionary, key, level). Function dictGetChildren return all children as an array if indexes. It is a inverse transformation for dictGetHierarchy. Function dictGetDescendants return all descendants as if dictGetChildren was applied level times recursively. Zero level value is equivalent to infinity. Improved performance of dictGetHierarchy, dictIsIn functions. Closes #14656. #22096 (Maksim Kita).
  • Added function dictGetOrNull. It works like dictGet, but return Null in case key was not found in dictionary. Closes #22375. #22413 (Maksim Kita).
  • Added a table function s3Cluster, which allows to process files from s3 in parallel on every node of a specified cluster. #22012 (Nikita Mikhaylov).
  • Added support for replicas and shards in MySQL/PostgreSQL table engine / table function. You can write SELECT * FROM mysql('host{1,2}-{1|2}', ...). Closes #20969. #22217 (Kseniia Sumarokova).
  • Added ALTER TABLE ... FETCH PART ... query. It’s similar to FETCH PARTITION, but fetches only one part. #22706 (turbo jason).
  • Added a setting max_distributed_depth that limits the depth of recursive queries to Distributed tables. Closes #20229. #21942 (flynn).

Performance Improvement

  • Improved performance of intDiv by dynamic dispatch for AVX2. This closes #22314. #23000 (alexey-milovidov).
  • Improved performance of reading from ArrowStream input format for sources other then local file (e.g. URL). #22673 (nvartolomei).
  • Disabled compression by default when interacting with localhost (with clickhouse-client or server to server with distributed queries) via native protocol. It may improve performance of some import/export operations. This closes #22234. #22237 (alexey-milovidov).
  • Exclude values that does not belong to the shard from right part of IN section for distributed queries (under optimize_skip_unused_shards_rewrite_in, enabled by default, since it still requires optimize_skip_unused_shards). #21511 (Azat Khuzhin).
  • Improved performance of reading a subset of columns with File-like table engine and column-oriented format like Parquet, Arrow or ORC. This closes #issue:20129. #21302 (keenwolf).
  • Allow to move more conditions to PREWHERE as it was before version 21.1 (adjustment of internal heuristics). Insufficient number of moved condtions could lead to worse performance. #23397 (Anton Popov).
  • Improved performance of ODBC connections and fixed all the outstanding issues from the backlog. Using nanodbc library instead of Poco::ODBC. Closes #9678. Add support for DateTime64 and Decimal* for ODBC table engine. Closes #21961. Fixed issue with cyrillic text being truncated. Closes #16246. Added connection pools for odbc bridge. #21972 (Kseniia Sumarokova).

Improvement

  • Increase max_uri_size (the maximum size of URL in HTTP interface) to 1 MiB by default. This closes #21197. #22997 (alexey-milovidov).
  • Set background_fetches_pool_size to 8 that is better for production usage with frequent small insertions or slow ZooKeeper cluster. #22945 (alexey-milovidov).
  • FlatDictionary added initial_array_size, max_array_size options. #22521 (Maksim Kita).
  • Add new setting non_replicated_deduplication_window for non-replicated MergeTree inserts deduplication. #22514 (alesapin).
  • Update paths to the CatBoost model configs in config reloading. #22434 (Kruglov Pavel).
  • Added Decimal256 type support in dictionaries. Decimal256 is experimental feature. Closes #20979. #22960 (Maksim Kita).
  • Enabled async_socket_for_remote by default (using less amount of OS threads for distributed queries). #23683 (Nikolai Kochetov).
  • Fixed quantile(s)TDigest. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. #23314 (Vladimir Chebotarev).
  • Make function name unhex case insensitive for compatibility with MySQL. #23229 (alexey-milovidov).
  • Implement functions arrayHasAny, arrayHasAll, has, indexOf, countEqual for generic case when types of array elements are different. In previous versions the functions arrayHasAny, arrayHasAll returned false and has, indexOf, countEqual thrown exception. Also add support for Decimal and big integer types in functions has and similar. This closes #20272. #23044 (alexey-milovidov).
  • Raised the threshold on max number of matches in result of the function extractAllGroupsHorizontal. #23036 (Vasily Nemkov).
  • Do not perform optimize_skip_unused_shards for cluster with one node. #22999 (Azat Khuzhin).
  • Added ability to run clickhouse-keeper (experimental drop-in replacement to ZooKeeper) with SSL. Config settings keeper_server.tcp_port_secure can be used for secure interaction between client and keeper-server. keeper_server.raft_configuration.secure can be used to enable internal secure communication between nodes. #22992 (alesapin).
  • Added ability to flush buffer only in background for Buffer tables. #22986 (Azat Khuzhin).
  • When selecting from MergeTree table with NULL in WHERE condition, in rare cases, exception was thrown. This closes #20019. #22978 (alexey-milovidov).
  • Fix error handling in Poco HTTP Client for AWS. #22973 (kreuzerkrieg).
  • Respect max_part_removal_threads for ReplicatedMergeTree. #22971 (Azat Khuzhin).
  • Fix obscure corner case of MergeTree settings inactive_parts_to_throw_insert = 0 with inactive_parts_to_delay_insert > 0. #22947 (Azat Khuzhin).
  • dateDiff now works with DateTime64 arguments (even for values outside of DateTime range) #22931 (Vasily Nemkov).
  • MaterializeMySQL (experimental feature): added an ability to replicate MySQL databases containing views without failing. This is accomplished by ignoring the views. #22760 (Christian).
  • Allow RBAC row policy via postgresql protocol. Closes #22658. PostgreSQL protocol is enabled in configuration by default. #22755 (Kseniia Sumarokova).
  • Add metric to track how much time is spend during waiting for Buffer layer lock. #22725 (Azat Khuzhin).
  • Allow to use CTE in VIEW definition. This closes #22491. #22657 (Amos Bird).
  • Clear the rest of the screen and show cursor in clickhouse-client if previous program has left garbage in terminal. This closes #16518. #22634 (alexey-milovidov).
  • Make round function to behave consistently on non-x86_64 platforms. Rounding half to nearest even (Banker’s rounding) is used. #22582 (alexey-milovidov).
  • Correctly check structure of blocks of data that are sending by Distributed tables. #22325 (Azat Khuzhin).
  • Allow publishing Kafka errors to a virtual column of Kafka engine, controlled by the kafka_handle_error_mode setting. #21850 (fastio).
  • Add aliases simpleJSONExtract/simpleJSONHas to visitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}. Fixes #21383. #21519 (fastio).
  • Add clickhouse-library-bridge for library dictionary source. Closes #9502. #21509 (Kseniia Sumarokova).
  • Forbid to drop a column if it’s referenced by materialized view. Closes #21164. #21303 (flynn).
  • Support dynamic interserver credentials (rotating credentials without downtime). #14113 (johnskopis).
  • Add support for Kafka storage with Arrow and ArrowStream format messages. #23415 (Chao Ma).
  • Fixed missing semicolon in exception message. The user may find this exception message unpleasant to read. #23208 (alexey-milovidov).
  • Fixed missing whitespace in some exception messages about LowCardinality type. #23207 (alexey-milovidov).
  • Some values were formatted with alignment in center in table cells in Markdown format. Not anymore. #23096 (alexey-milovidov).
  • Remove non-essential details from suggestions in clickhouse-client. This closes #22158. #23040 (alexey-milovidov).
  • Correct calculation of bytes_allocated field in system.dictionaries for sparse_hashed dictionaries. #22867 (Azat Khuzhin).
  • Fixed approximate total rows accounting for reverse reading from MergeTree. #22726 (Azat Khuzhin).
  • Fix the case when it was possible to configure dictionary with clickhouse source that was looking to itself that leads to infinite loop. Closes #14314. #22479 (Maksim Kita).

Bug Fix

  • Multiple fixes for hedged requests. Fixed an error Can't initialize pipeline with empty pipe for queries with GLOBAL IN/JOIN when the setting use_hedged_requests is enabled. Fixes #23431. #23805 (Nikolai Kochetov). Fixed a race condition in hedged connections which leads to crash. This fixes #22161. #22443 (Kruglov Pavel). Fix possible crash in case if unknown packet was received from remote query (with async_socket_for_remote enabled). Fixes #21167. #23309 (Nikolai Kochetov).
  • Fixed the behavior when disabling input_format_with_names_use_header setting discards all the input with CSVWithNames format. This fixes #22406. #23202 (Nikita Mikhaylov).
  • Fixed remote JDBC bridge timeout connection issue. Closes #9609. #23771 (Maksim Kita, alexey-milovidov).
  • Fix the logic of initial load of complex_key_hashed if update_field is specified. Closes #23800. #23824 (Maksim Kita).
  • Fixed crash when PREWHERE and row policy filter are both in effect with empty result. #23763 (Amos Bird).
  • Avoid possible “Cannot schedule a task” error (in case some exception had been occurred) on INSERT into Distributed. #23744 (Azat Khuzhin).
  • Added an exception in case of completely the same values in both samples in aggregate function mannWhitneyUTest. This fixes #23646. #23654 (Nikita Mikhaylov).
  • Fixed server fault when inserting data through HTTP caused an exception. This fixes #23512. #23643 (Nikita Mikhaylov).
  • Fixed misinterpretation of some LIKE expressions with escape sequences. #23610 (alexey-milovidov).
  • Fixed restart / stop command hanging. Closes #20214. #23552 (filimonov).
  • Fixed COLUMNS matcher in case of multiple JOINs in select query. Closes #22736. #23501 (Maksim Kita).
  • Fixed a crash when modifying column’s default value when a column itself is used as ReplacingMergeTree‘s parameter. #23483 (hexiaoting).
  • Fixed corner cases in vertical merges with ReplacingMergeTree. In rare cases they could lead to fails of merges with exceptions like Incomplete granules are not allowed while blocks are granules size. #23459 (Anton Popov).
  • Fixed bug that does not allow cast from empty array literal, to array with dimensions greater than 1, e.g. CAST([] AS Array(Array(String))). Closes #14476. #23456 (Maksim Kita).
  • Fixed a bug when deltaSum aggregate function produced incorrect result after resetting the counter. #23437 (Russ Frank).
  • Fixed Cannot unlink file error on unsuccessful creation of ReplicatedMergeTree table with multidisk configuration. This closes #21755. #23433 (tavplubix).
  • Fixed incompatible constant expression generation during partition pruning based on virtual columns. This fixes https://github.com/ClickHouse/ClickHouse/pull/21401#discussion_r611888913. #23366 (Amos Bird).
  • Fixed a crash when setting join_algorithm is set to ‘auto’ and Join is performed with a Dictionary. Close #23002. #23312 (Vladimir).
  • Don’t relax NOT conditions during partition pruning. This fixes #23305 and #21539. #23310 (Amos Bird).
  • Fixed very rare race condition on background cleanup of old blocks. It might cause a block not to be deduplicated if it’s too close to the end of deduplication window. #23301 (tavplubix).
  • Fixed very rare (distributed) race condition between creation and removal of ReplicatedMergeTree tables. It might cause exceptions like node doesn't exist on attempt to create replicated table. Fixes #21419. #23294 (tavplubix).
  • Fixed simple key dictionary from DDL creation if primary key is not first attribute. Fixes #23236. #23262 (Maksim Kita).
  • Fixed reading from ODBC when there are many long column names in a table. Closes #8853. #23215 (Kseniia Sumarokova).
  • MaterializeMySQL (experimental feature): fixed Not found column error when selecting from MaterializeMySQL with condition on key column. Fixes #22432. #23200 (tavplubix).
  • Correct aliases handling if subquery was optimized to constant. Fixes #22924. Fixes #10401. #23191 (Maksim Kita).
  • Server might fail to start if data_type_default_nullable setting is enabled in default profile, it’s fixed. Fixes #22573. #23185 (tavplubix).
  • Fixed a crash on shutdown which happened because of wrong accounting of current connections. #23154 (Vitaly Baranov).
  • Fixed Table .inner_id... doesn't exist error when selecting from Materialized View after detaching it from Atomic database and attaching back. #23047 (tavplubix).
  • Fix error Cannot find column in ActionsDAG result which may happen if subquery uses untuple. Fixes #22290. #22991 (Nikolai Kochetov).
  • Fix usage of constant columns of type Map with nullable values. #22939 (Anton Popov).
  • fixed formatDateTime() on DateTime64 and “%C” format specifier fixed toDateTime64() for large values and non-zero scale. #22937 (Vasily Nemkov).
  • Fixed a crash when using mannWhitneyUTest and rankCorr with window functions. This fixes #22728. #22876 (Nikita Mikhaylov).
  • LIVE VIEW (experimental feature): fixed possible hanging in concurrent DROP/CREATE of TEMPORARY LIVE VIEW in TemporaryLiveViewCleaner, see. #22858 (Vitaly Baranov).
  • Fixed pushdown of HAVING in case, when filter column is used in aggregation. #22763 (Anton Popov).
  • Fixed possible hangs in Zookeeper requests in case of OOM exception. Fixes #22438. #22684 (Nikolai Kochetov).
  • Fixed wait for mutations on several replicas for ReplicatedMergeTree table engines. Previously, mutation/alter query may finish before mutation actually executed on other replicas. #22669 (alesapin).
  • Fixed exception for Log with nested types without columns in the SELECT clause. #22654 (Azat Khuzhin).
  • Fix unlimited wait for auxiliary AWS requests. #22594 (Vladimir Chebotarev).
  • Fixed a crash when client closes connection very early #22579. #22591 (nvartolomei).
  • Map data type (experimental feature): fixed an incorrect formatting of function map in distributed queries. #22588 (foolchi).
  • Fixed deserialization of empty string without newline at end of TSV format. This closes #20244. Possible workaround without version update: set input_format_null_as_default to zero. It was zero in old versions. #22527 (alexey-milovidov).
  • Fixed wrong cast of a column of LowCardinality type in Merge Join algorithm. Close #22386, close #22388. #22510 (Vladimir).
  • Buffer overflow (on read) was possible in tokenbf_v1 full text index. The excessive bytes are not used but the read operation may lead to crash in rare cases. This closes #19233. #22421 (alexey-milovidov).
  • Do not limit HTTP chunk size. Fixes #21907. #22322 (Ivan).
  • Fixed a bug, which leads to underaggregation of data in case of enabled optimize_aggregation_in_order and many parts in table. Slightly improve performance of aggregation with enabled optimize_aggregation_in_order. #21889 (Anton Popov).
  • Check if table function view is used as a column. This complements #20350. #21465 (Amos Bird).
  • Fix “unknown column” error for tables with Merge engine in queris with JOIN and aggregation. Closes #18368, close #22226. #21370 (Vladimir).
  • Fixed name clashes in pushdown optimization. It caused incorrect WHERE filtration after FULL JOIN. Close #20497. #20622 (Vladimir).
  • Fixed very rare bug when quorum insert with quorum_parallel=1 is not really “quorum” because of deduplication. #18215 (filimonov - reported, alesapin - fixed).

Build/Testing/Packaging Improvement

ClickHouse release 21.4

ClickHouse release 21.4.1 2021-04-12

Backward Incompatible Change

  • The toStartOfIntervalFunction will align hour intervals to the midnight (in previous versions they were aligned to the start of unix epoch). For example, toStartOfInterval(x, INTERVAL 11 HOUR) will split every day into three intervals: 00:00:00..10:59:59, 11:00:00..21:59:59 and 22:00:00..23:59:59. This behaviour is more suited for practical needs. This closes #9510. #22060 (alexey-milovidov).
  • Age and Precision in graphite rollup configs should increase from retention to retention. Now it’s checked and the wrong config raises an exception. #21496 (Mikhail f. Shiryaev).
  • Fix cutToFirstSignificantSubdomainCustom()/firstSignificantSubdomainCustom() returning wrong result for 3+ level domains present in custom top-level domain list. For input domains matching these custom top-level domains, the third-level domain was considered to be the first significant one. This is now fixed. This change may introduce incompatibility if the function is used in e.g. the sharding key. #21946 (Azat Khuzhin).
  • Column keys in table system.dictionaries was replaced to columns key.names and key.types. Columns key.names, key.types, attribute.names, attribute.types from system.dictionaries table does not require dictionary to be loaded. #21884 (Maksim Kita).
  • Now replicas that are processing the ALTER TABLE ATTACH PART[ITION] command search in their detached/ folders before fetching the data from other replicas. As an implementation detail, a new command ATTACH_PART is introduced in the replicated log. Parts are searched and compared by their checksums. #18978 (Mike Kot). Note:
  • ATTACH PART[ITION] queries may not work during cluster upgrade.
  • It’s not possible to rollback to older ClickHouse version after executing ALTER ... ATTACH query in new version as the old servers would fail to pass the ATTACH_PART entry in the replicated log.
  • In this version, empty <remote_url_allow_hosts></remote_url_allow_hosts> will block all access to remote hosts while in previous versions it did nothing. If you want to keep old behaviour and you have empty remote_url_allow_hosts element in configuration file, remove it. #20058 (Vladimir Chebotarev).

New Feature

  • Extended range of DateTime64 to support dates from year 1925 to 2283. Improved support of DateTime around zero date (1970-01-01). #9404 (alexey-milovidov, Vasily Nemkov). Not every time and date functions are working for extended range of dates.
  • Added support of Kerberos authentication for preconfigured users and HTTP requests (GSS-SPNEGO). #14995 (Denis Glazachev).
  • Add prefer_column_name_to_alias setting to use original column names instead of aliases. it is needed to be more compatible with common databases’ aliasing rules. This is for #9715 and #9887. #22044 (Amos Bird).
  • Added functions dictGetChildren(dictionary, key), dictGetDescendants(dictionary, key, level). Function dictGetChildren return all children as an array if indexes. It is a inverse transformation for dictGetHierarchy. Function dictGetDescendants return all descendants as if dictGetChildren was applied level times recursively. Zero level value is equivalent to infinity. Closes #14656. #22096 (Maksim Kita).
  • Added executable_pool dictionary source. Close #14528. #21321 (Maksim Kita).
  • Added table function dictionary. It works the same way as Dictionary engine. Closes #21560. #21910 (Maksim Kita).
  • Support Nullable type for PolygonDictionary attribute. #21890 (Maksim Kita).
  • Functions dictGet, dictHas use current database name if it is not specified for dictionaries created with DDL. Closes #21632. #21859 (Maksim Kita).
  • Added function dictGetOrNull. It works like dictGet, but return Null in case key was not found in dictionary. Closes #22375. #22413 (Maksim Kita).
  • Added async update in ComplexKeyCache, SSDCache, SSDComplexKeyCache dictionaries. Added support for Nullable type in Cache, ComplexKeyCache, SSDCache, SSDComplexKeyCache dictionaries. Added support for multiple attributes fetch with dictGet, dictGetOrDefault functions. Fixes #21517. #20595 (Maksim Kita).
  • Support dictHas function for RangeHashedDictionary. Fixes #6680. #19816 (Maksim Kita).
  • Add function timezoneOf that returns the timezone name of DateTime or DateTime64 data types. This does not close #9959. Fix inconsistencies in function names: add aliases timezone and timeZone as well as toTimezone and toTimeZone and timezoneOf and timeZoneOf. #22001 (alexey-milovidov).
  • Add new optional clause GRANTEES for CREATE/ALTER USER commands. It specifies users or roles which are allowed to receive grants from this user on condition this user has also all required access granted with grant option. By default GRANTEES ANY is used which means a user with grant option can grant to anyone. Syntax: CREATE USER ... GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]. #21641 (Vitaly Baranov).
  • Add new column slowdowns_count to system.clusters. When using hedged requests, it shows how many times we switched to another replica because this replica was responding slowly. Also show actual value of errors_count in system.clusters. #21480 (Kruglov Pavel).
  • Add _partition_id virtual column for MergeTree* engines. Allow to prune partitions by _partition_id. Add partitionID() function to calculate partition id string. #21401 (Amos Bird).
  • Add function isIPAddressInRange to test if an IPv4 or IPv6 address is contained in a given CIDR network prefix. #21329 (PHO).
  • Added new SQL command ALTER TABLE 'table_name' UNFREEZE [PARTITION 'part_expr'] WITH NAME 'backup_name'. This command is needed to properly remove ‘freezed’ partitions from all disks. #21142 (Pavel Kovalenko).
  • Supports implicit key type conversion for JOIN. #19885 (Vladimir).

Experimental Feature

  • Support RANGE OFFSET frame (for window functions) for floating point types. Implement lagInFrame/leadInFrame window functions, which are analogous to lag/lead, but respect the window frame. They are identical when the frame is between unbounded preceding and unbounded following. This closes #5485. #21895 (Alexander Kuzmenkov).
  • Zero-copy replication for ReplicatedMergeTree over S3 storage. #16240 (ianton-ru).
  • Added possibility to migrate existing S3 disk to the schema with backup-restore capabilities. #22070 (Pavel Kovalenko).

Performance Improvement

  • Supported parallel formatting in clickhouse-local and everywhere else. #21630 (Nikita Mikhaylov).
  • Support parallel parsing for CSVWithNames and TSVWithNames formats. This closes #21085. #21149 (Nikita Mikhaylov).
  • Enable read with mmap IO for file ranges from 64 MiB (the settings min_bytes_to_use_mmap_io). It may lead to moderate performance improvement. #22326 (alexey-milovidov).
  • Add cache for files read with min_bytes_to_use_mmap_io setting. It makes significant (2x and more) performance improvement when the value of the setting is small by avoiding frequent mmap/munmap calls and the consequent page faults. Note that mmap IO has major drawbacks that makes it less reliable in production (e.g. hung or SIGBUS on faulty disks; less controllable memory usage). Nevertheless it is good in benchmarks. #22206 (alexey-milovidov).
  • Avoid unnecessary data copy when using codec NONE. Please note that codec NONE is mostly useless - it’s recommended to always use compression (LZ4 is by default). Despite the common belief, disabling compression may not improve performance (the opposite effect is possible). The NONE codec is useful in some cases: - when data is uncompressable; - for synthetic benchmarks. #22145 (alexey-milovidov).
  • Faster GROUP BY with small max_rows_to_group_by and group_by_overflow_mode='any'. #21856 (Nikolai Kochetov).
  • Optimize performance of queries like SELECT ... FINAL ... WHERE. Now in queries with FINAL it’s allowed to move to PREWHERE columns, which are in sorting key. #21830 (foolchi).
  • Improved performance by replacing memcpy to another implementation. This closes #18583. #21520 (alexey-milovidov).
  • Improve performance of aggregation in order of sorting key (with enabled setting optimize_aggregation_in_order). #19401 (Anton Popov).

Improvement

  • Add connection pool for PostgreSQL table/database engine and dictionary source. Should fix #21444. #21839 (Kseniia Sumarokova).
  • Support non-default table schema for postgres storage/table-function. Closes #21701. #21711 (Kseniia Sumarokova).
  • Support replicas priority for postgres dictionary source. #21710 (Kseniia Sumarokova).
  • Introduce a new merge tree setting min_bytes_to_rebalance_partition_over_jbod which allows assigning new parts to different disks of a JBOD volume in a balanced way. #16481 (Amos Bird).
  • Added Grant, Revoke and System values of query_kind column for corresponding queries in system.query_log. #21102 (Vasily Nemkov).
  • Allow customizing timeouts for HTTP connections used for replication independently from other HTTP timeouts. #20088 (nvartolomei).
  • Better exception message in client in case of exception while server is writing blocks. In previous versions client may get misleading message like Data compressed with different methods. #22427 (alexey-milovidov).
  • Fix error Directory tmp_fetch_XXX already exists which could happen after failed fetch part. Delete temporary fetch directory if it already exists. Fixes #14197. #22411 (nvartolomei).
  • Fix MSan report for function range with UInt256 argument (support for large integers is experimental). This closes #22157. #22387 (alexey-milovidov).
  • Add current_database column to system.processes table. It contains the current database of the query. #22365 (Alexander Kuzmenkov).
  • Add case-insensitive history search/navigation and subword movement features to clickhouse-client. #22105 (Amos Bird).
  • If tuple of NULLs, e.g. (NULL, NULL) is on the left hand side of IN operator with tuples of non-NULLs on the right hand side, e.g. SELECT (NULL, NULL) IN ((0, 0), (3, 1)) return 0 instead of throwing an exception about incompatible types. The expression may also appear due to optimization of something like SELECT (NULL, NULL) = (8, 0) OR (NULL, NULL) = (3, 2) OR (NULL, NULL) = (0, 0) OR (NULL, NULL) = (3, 1). This closes #22017. #22063 (alexey-milovidov).
  • Update used version of simdjson to 0.9.1. This fixes #21984. #22057 (Vitaly Baranov).
  • Added case insensitive aliases for CONNECTION_ID() and VERSION() functions. This fixes #22028. #22042 (Eugene Klimov).
  • Add option strict_increase to windowFunnel function to calculate each event once (resolve #21835). #22025 (Vladimir).
  • If partition key of a MergeTree table does not include Date or DateTime columns but includes exactly one DateTime64 column, expose its values in the min_time and max_time columns in system.parts and system.parts_columns tables. Add min_time and max_time columns to system.parts_columns table (these was inconsistency to the system.parts table). This closes #18244. #22011 (alexey-milovidov).
  • Supported replication_alter_partitions_sync=1 setting in clickhouse-copier for moving partitions from helping table to destination. Decreased default timeouts. Fixes #21911. #21912 (turbo jason).
  • Show path to data directory of EmbeddedRocksDB tables in system tables. #21903 (tavplubix).
  • Add profile event HedgedRequestsChangeReplica, change read data timeout from sec to ms. #21886 (Kruglov Pavel).
  • DiskS3 (experimental feature under development). Fixed bug with the impossibility to move directory if the destination is not empty and cache disk is used. #21837 (Pavel Kovalenko).
  • Better formatting for Array and Map data types in Web UI. #21798 (alexey-milovidov).
  • Update clusters only if their configurations were updated. #21685 (Kruglov Pavel).
  • Propagate query and session settings for distributed DDL queries. Set distributed_ddl_entry_format_version to 2 to enable this. Added distributed_ddl_output_mode setting. Supported modes: none, throw (default), null_status_on_timeout and never_throw. Miscellaneous fixes and improvements for Replicated database engine. #21535 (tavplubix).
  • If PODArray was instantiated with element size that is neither a fraction or a multiple of 16, buffer overflow was possible. No bugs in current releases exist. #21533 (alexey-milovidov).
  • Add last_error_time/last_error_message/last_error_stacktrace/remote columns for system.errors. #21529 (Azat Khuzhin).
  • Add aliases simpleJSONExtract/simpleJSONHas to visitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}. Fixes #21383. #21519 (fastio).
  • Add setting optimize_skip_unused_shards_limit to limit the number of sharding key values for optimize_skip_unused_shards. #21512 (Azat Khuzhin).
  • Improve clickhouse-format to not throw exception when there are extra spaces or comment after the last query, and throw exception early with readable message when format ASTInsertQuery with data . #21311 (flynn).
  • Improve support of integer keys in data type Map. #21157 (Anton Popov).
  • MaterializeMySQL: attempt to reconnect to MySQL if the connection is lost. #20961 (Håvard Kvålen).
  • Support more cases to rewrite CROSS JOIN to INNER JOIN. #20392 (Vladimir).
  • Do not create empty parts on INSERT when optimize_on_insert setting enabled. Fixes #20304. #20387 (Kruglov Pavel).
  • MaterializeMySQL: add minmax skipping index for _version column. #20382 (Stig Bakken).
  • Add option --backslash for clickhouse-format, which can add a backslash at the end of each line of the formatted query. #21494 (flynn).
  • Now clickhouse will not throw LOGICAL_ERROR exception when we try to mutate the already covered part. Fixes #22013. #22291 (alesapin).

Bug Fix

  • Remove socket from epoll before cancelling packet receiver in HedgedConnections to prevent possible race. Fixes #22161. #22443 (Kruglov Pavel).
  • Add (missing) memory accounting in parallel parsing routines. In previous versions OOM was possible when the resultset contains very large blocks of data. This closes #22008. #22425 (alexey-milovidov).
  • Fix exception which may happen when SELECT has constant WHERE condition and source table has columns which names are digits. #22270 (LiuNeng).
  • Fix query cancellation with use_hedged_requests=0 and async_socket_for_remote=1. #22183 (Azat Khuzhin).
  • Fix uncaught exception in InterserverIOHTTPHandler. #22146 (Azat Khuzhin).
  • Fix docker entrypoint in case http_port is not in the config. #22132 (Ewout).
  • Fix error Invalid number of rows in Chunk in JOIN with TOTALS and arrayJoin. Closes #19303. #22129 (Vladimir).
  • Fix the background thread pool name which used to poll message from Kafka. The Kafka engine with the broken thread pool will not consume the message from message queue. #22122 (fastio).
  • Fix waiting for OPTIMIZE and ALTER queries for ReplicatedMergeTree table engines. Now the query will not hang when the table was detached or restarted. #22118 (alesapin).
  • Disable async_socket_for_remote/use_hedged_requests for buggy Linux kernels. #22109 (Azat Khuzhin).
  • Docker entrypoint: avoid chown of . in case when LOG_PATH is empty. Closes #22100. #22102 (filimonov).
  • The function decrypt was lacking a check for the minimal size of data encrypted in AEAD mode. This closes #21897. #22064 (alexey-milovidov).
  • In rare case, merge for CollapsingMergeTree may create granule with index_granularity + 1 rows. Because of this, internal check, added in #18928 (affects 21.2 and 21.3), may fail with error Incomplete granules are not allowed while blocks are granules size. This error did not allow parts to merge. #21976 (Nikolai Kochetov).
  • Reverted #15454 that may cause significant increase in memory usage while loading external dictionaries of hashed type. This closes #21935. #21948 (Maksim Kita).
  • Prevent hedged connections overlaps (Unknown packet 9 from server error). #21941 (Azat Khuzhin).
  • Fix reading the HTTP POST request with “multipart/form-data” content type in some cases. #21936 (Ivan).
  • Fix wrong ORDER BY results when a query contains window functions, and optimization for reading in primary key order is applied. Fixes #21828. #21915 (Alexander Kuzmenkov).
  • Fix deadlock in first catboost model execution. Closes #13832. #21844 (Kruglov Pavel).
  • Fix incorrect query result (and possible crash) which could happen when WHERE or HAVING condition is pushed before GROUP BY. Fixes #21773. #21841 (Nikolai Kochetov).
  • Better error handling and logging in WriteBufferFromS3. #21836 (Pavel Kovalenko).
  • Fix possible crashes in aggregate functions with combinator Distinct, while using two-level aggregation. This is a follow-up fix of #18365 . Can only reproduced in production env. #21818 (Amos Bird).
  • Fix scalar subquery index analysis. This fixes #21717 , which was introduced in #18896. #21766 (Amos Bird).
  • Fix bug for ReplicatedMerge table engines when ALTER MODIFY COLUMN query doesn’t change the type of Decimal column if its size (32 bit or 64 bit) doesn’t change. #21728 (alesapin).
  • Fix possible infinite waiting when concurrent OPTIMIZE and DROP are run for ReplicatedMergeTree. #21716 (Azat Khuzhin).
  • Fix function arrayElement with type Map for constant integer arguments. #21699 (Anton Popov).
  • Fix SIGSEGV on not existing attributes from ip_trie with access_to_key_from_attributes. #21692 (Azat Khuzhin).
  • Server now start accepting connections only after DDLWorker and dictionaries initialization. #21676 (Azat Khuzhin).
  • Add type conversion for keys of tables of type Join (previously led to SIGSEGV). #21646 (Azat Khuzhin).
  • Fix distributed requests cancellation (for example simple select from multiple shards with limit, i.e. select * from remote('127.{2,3}', system.numbers) limit 100) with async_socket_for_remote=1. #21643 (Azat Khuzhin).
  • Fix fsync_part_directory for horizontal merge. #21642 (Azat Khuzhin).
  • Remove unknown columns from joined table in WHERE for queries to external database engines (MySQL, PostgreSQL). close #14614, close #19288 (dup), close #19645 (dup). #21640 (Vladimir).
  • std::terminate was called if there is an error writing data into s3. #21624 (Vladimir).
  • Fix possible error Cannot find column when optimize_skip_unused_shards is enabled and zero shards are used. #21579 (Azat Khuzhin).
  • In case if query has constant WHERE condition, and setting optimize_skip_unused_shards enabled, all shards may be skipped and query could return incorrect empty result. #21550 (Amos Bird).
  • Fix table function clusterAllReplicas returns wrong _shard_num. close #21481. #21498 (flynn).
  • Fix that S3 table holds old credentials after config update. #21457 (Grigory Pervakov).
  • Fixed race on SSL object inside SecureSocket in Poco. #21456 (Nikita Mikhaylov).
  • Fix Avro format parsing for Kafka. Fixes #21437. #21438 (Ilya Golshtein).
  • Fix receive and send timeouts and non-blocking read in secure socket. #21429 (Kruglov Pavel).
  • force_drop_table flag didn’t work for MATERIALIZED VIEW, it’s fixed. Fixes #18943. #20626 (tavplubix).
  • Fix name clashes in PredicateRewriteVisitor. It caused incorrect WHERE filtration after full join. Close #20497. #20622 (Vladimir).

Build/Testing/Packaging Improvement

  • Add Jepsen tests for ClickHouse Keeper. #21677 (alesapin).
  • Run stateless tests in parallel in CI. Depends on #22181. #22300 (alesapin).
  • Enable status check for SQLancer CI run. #22015 (Ilya Yatsishin).
  • Multiple preparations for PowerPC builds: Enable the bundled openldap on ppc64le. #22487 (Kfir Itzhak). Enable compiling on ppc64le with Clang. #22476 (Kfir Itzhak). Fix compiling boost on ppc64le. #22474 (Kfir Itzhak). Fix CMake error about internal CMake variable CMAKE_ASM_COMPILE_OBJECT not set on ppc64le. #22469 (Kfir Itzhak). Fix Fedora/RHEL/CentOS not finding libclang_rt.builtins on ppc64le. #22458 (Kfir Itzhak). Enable building with jemalloc on ppc64le. #22447 (Kfir Itzhak). Fix ClickHouse’s config embedding and cctz’s timezone embedding on ppc64le. #22445 (Kfir Itzhak). Fixed compiling on ppc64le and use the correct instruction pointer register on ppc64le. #22430 (Kfir Itzhak).
  • Re-enable the S3 (AWS) library on aarch64. #22484 (Kfir Itzhak).
  • Add tzdata to Docker containers because reading ORC formats requires it. This closes #14156. #22000 (alexey-milovidov).
  • Introduce 2 arguments for clickhouse-server image Dockerfile: deb_location & single_binary_location. #21977 (filimonov).
  • Allow to use clang-tidy with release builds by enabling assertions if it is used. #21914 (alexey-milovidov).
  • Add llvm-12 binaries name to search in cmake scripts. Implicit constants conversions to mute clang warnings. Updated submodules to build with CMake 3.19. Mute recursion in macro expansion in readpassphrase library. Deprecated -fuse-ld changed to --ld-path for clang. #21597 (Ilya Yatsishin).
  • Updating docker/test/testflows/runner/dockerd-entrypoint.sh to use Yandex dockerhub-proxy, because Docker Hub has enabled very restrictive rate limits #21551 (vzakaznikov).
  • Fix macOS shared lib build. #20184 (nvartolomei).
  • Add ctime option to zookeeper-dump-tree. It allows to dump node creation time. #21842 (Ilya).

ClickHouse release 21.3 (LTS)

ClickHouse release v21.3, 2021-03-12

Backward Incompatible Change

  • Now it’s not allowed to create MergeTree tables in old syntax with table TTL because it’s just ignored. Attach of old tables is still possible. #20282 (alesapin).
  • Now all case-insensitive function names will be rewritten to their canonical representations. This is needed for projection query routing (the upcoming feature). #20174 (Amos Bird).
  • Fix creation of TTL in cases, when its expression is a function and it is the same as ORDER BY key. Now it’s allowed to set custom aggregation to primary key columns in TTL with GROUP BY. Backward incompatible: For primary key columns, which are not in GROUP BY and aren’t set explicitly now is applied function any instead of max, when TTL is expired. Also if you use TTL with WHERE or GROUP BY you can see exceptions at merges, while making rolling update. #15450 (Anton Popov).

New Feature

  • Add file engine settings: engine_file_empty_if_not_exists and engine_file_truncate_on_insert. #20620 (M0r64n).
  • Add aggregate function deltaSum for summing the differences between consecutive rows. #20057 (Russ Frank).
  • New event_time_microseconds column in system.part_log table. #20027 (Bharat Nallan).
  • Added timezoneOffset(datetime) function which will give the offset from UTC in seconds. This close #issue:19850. #19962 (keenwolf).
  • Add setting insert_shard_id to support insert data into specific shard from distributed table. #19961 (flynn).
  • Function reinterpretAs updated to support big integers. Fixes #19691. #19858 (Maksim Kita).
  • Added Server Side Encryption Customer Keys (the x-amz-server-side-encryption-customer-(key/md5) header) support in S3 client. See the link. Closes #19428. #19748 (Vladimir Chebotarev).
  • Added implicit_key option for executable dictionary source. It allows to avoid printing key for every record if records comes in the same order as the input keys. Implements #14527. #19677 (Maksim Kita).
  • Add quota type query_selects and query_inserts. #19603 (JackyWoo).
  • Add function extractTextFromHTML #19600 (zlx19950903), (alexey-milovidov).
  • Tables with MergeTree* engine now have two new table-level settings for query concurrency control. Setting max_concurrent_queries limits the number of concurrently executed queries which are related to this table. Setting min_marks_to_honor_max_concurrent_queries tells to apply previous setting only if query reads at least this number of marks. #19544 (Amos Bird).
  • Added file function to read file from user_files directory as a String. This is different from the file table function. This implements #issue:18851. #19204 (keenwolf).

Experimental feature

  • Add experimental Replicated database engine. It replicates DDL queries across multiple hosts. #16193 (tavplubix).
  • Introduce experimental support for window functions, enabled with allow_experimental_window_functions = 1. This is a preliminary, alpha-quality implementation that is not suitable for production use and will change in backward-incompatible ways in future releases. Please see the documentation for the list of supported features. #20337 (Alexander Kuzmenkov).
  • Add the ability to backup/restore metadata files for DiskS3. #18377 (Pavel Kovalenko).

Performance Improvement

  • Hedged requests for remote queries. When setting use_hedged_requests enabled (off by default), allow to establish many connections with different replicas for query. New connection is enabled in case existent connection(s) with replica(s) were not established within hedged_connection_timeout or no data was received within receive_data_timeout. Query uses the first connection which send non empty progress packet (or data packet, if allow_changing_replica_until_first_data_packet); other connections are cancelled. Queries with max_parallel_replicas > 1 are supported. #19291 (Kruglov Pavel). This allows to significantly reduce tail latencies on very large clusters.
  • Added support for PREWHERE (and enable the corresponding optimization) when tables have row-level security expressions specified. #19576 (Denis Glazachev).
  • The setting distributed_aggregation_memory_efficient is enabled by default. It will lower memory usage and improve performance of distributed queries. #20599 (alexey-milovidov).
  • Improve performance of GROUP BY multiple fixed size keys. #20472 (alexey-milovidov).
  • Improve performance of aggregate functions by more strict aliasing. #19946 (alexey-milovidov).
  • Speed up reading from Memory tables in extreme cases (when reading speed is in order of 50 GB/sec) by simplification of pipeline and (consequently) less lock contention in pipeline scheduling. #20468 (alexey-milovidov).
  • Partially reimplement HTTP server to make it making less copies of incoming and outgoing data. It gives up to 1.5 performance improvement on inserting long records over HTTP. #19516 (Ivan).
  • Add compress setting for Memory tables. If it’s enabled the table will use less RAM. On some machines and datasets it can also work faster on SELECT, but it is not always the case. This closes #20093. Note: there are reasons why Memory tables can work slower than MergeTree: (1) lack of compression (2) static size of blocks (3) lack of indices and prewhere… #20168 (alexey-milovidov).
  • Slightly better code in aggregation. #20978 (alexey-milovidov).
  • Add back intDiv/modulo specializations for better performance. This fixes #21293 . The regression was introduced in https://github.com/ClickHouse/ClickHouse/pull/18145 . #21307 (Amos Bird).
  • Do not squash blocks too much on INSERT SELECT if inserting into Memory table. In previous versions inefficient data representation was created in Memory table after INSERT SELECT. This closes #13052. #20169 (alexey-milovidov).
  • Fix at least one case when DataType parser may have exponential complexity (found by fuzzer). This closes #20096. #20132 (alexey-milovidov).
  • Parallelize SELECT with FINAL for single part with level > 0 when do_not_merge_across_partitions_select_final setting is 1. #19375 (Kruglov Pavel).
  • Fill only requested columns when querying system.parts and system.parts_columns. Closes #19570. #21035 (Anmol Arora).
  • Perform algebraic optimizations of arithmetic expressions inside avg aggregate function. close #20092. #20183 (flynn).

Improvement

  • Case-insensitive compression methods for table functions. Also fixed LZMA compression method which was checked in upper case. #21416 (Vladimir Chebotarev).
  • Add two settings to delay or throw error during insertion when there are too many inactive parts. This is useful when server fails to clean up parts quickly enough. #20178 (Amos Bird).
  • Provide better compatibility for mysql clients. 1. mysql jdbc 2. mycli. #21367 (Amos Bird).
  • Forbid to drop a column if it’s referenced by materialized view. Closes #21164. #21303 (flynn).
  • MySQL dictionary source will now retry unexpected connection failures (Lost connection to MySQL server during query) which sometimes happen on SSL/TLS connections. #21237 (Alexander Kazakov).
  • Usability improvement: more consistent DateTime64 parsing: recognize the case when unix timestamp with subsecond resolution is specified as scaled integer (like 1111111111222 instead of 1111111111.222). This closes #13194. #21053 (alexey-milovidov).
  • Do only merging of sorted blocks on initiator with distributed_group_by_no_merge. #20882 (Azat Khuzhin).
  • When loading config for mysql source ClickHouse will now randomize the list of replicas with the same priority to ensure the round-robin logics of picking mysql endpoint. This closes #20629. #20632 (Alexander Kazakov).
  • Function ‘reinterpretAs(x, Type)’ renamed into ‘reinterpret(x, Type)’. #20611 (Maksim Kita).
  • Support vhost for RabbitMQ engine #20576. #20596 (Kseniia Sumarokova).
  • Improved serialization for data types combined of Arrays and Tuples. Improved matching enum data types to protobuf enum type. Fixed serialization of the Map data type. Omitted values are now set by default. #20506 (Vitaly Baranov).
  • Fixed race between execution of distributed DDL tasks and cleanup of DDL queue. Now DDL task cannot be removed from ZooKeeper if there are active workers. Fixes #20016. #20448 (tavplubix).
  • Make FQDN and other DNS related functions work correctly in alpine images. #20336 (filimonov).
  • Do not allow early constant folding of explicitly forbidden functions. #20303 (Azat Khuzhin).
  • Implicit conversion from integer to Decimal type might succeeded if integer value doe not fit into Decimal type. Now it throws ARGUMENT_OUT_OF_BOUND. #20232 (tavplubix).
  • Lockless SYSTEM FLUSH DISTRIBUTED. #20215 (Azat Khuzhin).
  • Normalize count(constant), sum(1) to count(). This is needed for projection query routing. #20175 (Amos Bird).
  • Support all native integer types in bitmap functions. #20171 (Amos Bird).
  • Updated CacheDictionary, ComplexCacheDictionary, SSDCacheDictionary, SSDComplexKeyDictionary to use LRUHashMap as underlying index. #20164 (Maksim Kita).
  • The setting access_management is now configurable on startup by providing CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT, defaults to disabled (0) which was the prior value. #20139 (Marquitos).
  • Fix toDateTime64(toDate()/toDateTime()) for DateTime64 - Implement DateTime64 clamping to match DateTime behaviour. #20131 (Azat Khuzhin).
  • Quota improvements: SHOW TABLES is now considered as one query in the quota calculations, not two queries. SYSTEM queries now consume quota. Fix calculation of interval’s end in quota consumption. #20106 (Vitaly Baranov).
  • Supports path IN (set) expressions for system.zookeeper table. #20105 (小路).
  • Show full details of MaterializeMySQL tables in system.tables. #20051 (Stig Bakken).
  • Fix data race in executable dictionary that was possible only on misuse (when the script returns data ignoring its input). #20045 (alexey-milovidov).
  • The value of MYSQL_OPT_RECONNECT option can now be controlled by “opt_reconnect” parameter in the config section of mysql replica. #19998 (Alexander Kazakov).
  • If user calls JSONExtract function with Float32 type requested, allow inaccurate conversion to the result type. For example the number 0.1 in JSON is double precision and is not representable in Float32, but the user still wants to get it. Previous versions return 0 for non-Nullable type and NULL for Nullable type to indicate that conversion is imprecise. The logic was 100% correct but it was surprising to users and leading to questions. This closes #13962. #19960 (alexey-milovidov).
  • Add conversion of block structure for INSERT into Distributed tables if it does not match. #19947 (Azat Khuzhin).
  • Improvement for the system.distributed_ddl_queue table. Initialize MaxDDLEntryID to the last value after restarting. Before this PR, MaxDDLEntryID will remain zero until a new DDLTask is processed. #19924 (Amos Bird).
  • Show MaterializeMySQL tables in system.parts. #19770 (Stig Bakken).
  • Add separate config directive for Buffer profile. #19721 (Azat Khuzhin).
  • Move conditions that are not related to JOIN to WHERE clause. #18720. #19685 (hexiaoting).
  • Add ability to throttle INSERT into Distributed based on amount of pending bytes for async send (bytes_to_delay_insert/max_delay_to_insert and bytes_to_throw_insert settings for Distributed engine has been added). #19673 (Azat Khuzhin).
  • Fix some rare cases when write errors can be ignored in destructors. #19451 (Azat Khuzhin).
  • Print inline frames in stack traces for fatal errors. #19317 (Ivan).

Bug Fix

  • Fix redundant reconnects to ZooKeeper and the possibility of two active sessions for a single clickhouse server. Both problems introduced in #14678. #21264 (alesapin).
  • Fix error Bad cast from type ... to DB::ColumnLowCardinality while inserting into table with LowCardinality column from Values format. Fixes #21140 #21357 (Nikolai Kochetov).
  • Fix a deadlock in ALTER DELETE mutations for non replicated MergeTree table engines when the predicate contains the table itself. Fixes #20558. #21477 (alesapin).
  • Fix SIGSEGV for distributed queries on failures. #21434 (Azat Khuzhin).
  • Now ALTER MODIFY COLUMN queries will correctly affect changes in partition key, skip indices, TTLs, and so on. Fixes #13675. #21334 (alesapin).
  • Fix bug with join_use_nulls and joining TOTALS from subqueries. This closes #19362 and #21137. #21248 (vdimir).
  • Fix crash in EXPLAIN for query with UNION. Fixes #20876, #21170. #21246 (flynn).
  • Now mutations allowed only for table engines that support them (MergeTree family, Memory, MaterializedView). Other engines will report a more clear error. Fixes #21168. #21183 (alesapin).
  • Fixes #21112. Fixed bug that could cause duplicates with insert query (if one of the callbacks came a little too late). #21138 (Kseniia Sumarokova).
  • Fix input_format_null_as_default take effective when types are nullable. This fixes #21116 . #21121 (Amos Bird).
  • fix bug related to cast Tuple to Map. Closes #21029. #21120 (hexiaoting).
  • Fix the metadata leak when the Replicated*MergeTree with custom (non default) ZooKeeper cluster is dropped. #21119 (fastio).
  • Fix type mismatch issue when using LowCardinality keys in joinGet. This fixes #21114. #21117 (Amos Bird).
  • fix default_replica_path and default_replica_name values are useless on Replicated(*)MergeTree engine when the engine needs specify other parameters. #21060 (mxzlxy).
  • Out of bound memory access was possible when formatting specifically crafted out of range value of type DateTime64. This closes #20494. This closes #20543. #21023 (alexey-milovidov).
  • Block parallel insertions into storage join. #21009 (vdimir).
  • Fixed behaviour, when ALTER MODIFY COLUMN created mutation, that will knowingly fail. #21007 (Anton Popov).
  • Closes #9969. Fixed Brotli http compression error, which reproduced for large data sizes, slightly complicated structure and with json output format. Update Brotli to the latest version to include the “fix rare access to uninitialized data in ring-buffer”. #20991 (Kseniia Sumarokova).
  • Fix ‘Empty task was returned from async task queue’ on query cancellation. #20881 (Azat Khuzhin).
  • USE database; query did not work when using MySQL 5.7 client to connect to ClickHouse server, it’s fixed. Fixes #18926. #20878 (tavplubix).
  • Fix usage of -Distinct combinator with -State combinator in aggregate functions. #20866 (Anton Popov).
  • Fix subquery with union distinct and limit clause. close #20597. #20610 (flynn).
  • Fixed inconsistent behavior of dictionary in case of queries where we look for absent keys in dictionary. #20578 (Nikita Mikhaylov).
  • Fix the number of threads for scalar subqueries and subqueries for index (after #19007 single thread was always used). Fixes #20457, #20512. #20550 (Nikolai Kochetov).
  • Fix crash which could happen if unknown packet was received from remove query (was introduced in #17868). #20547 (Azat Khuzhin).
  • Add proper checks while parsing directory names for async INSERT (fixes SIGSEGV). #20498 (Azat Khuzhin).
  • Fix function transform does not work properly for floating point keys. Closes #20460. #20479 (flynn).
  • Fix infinite loop when propagating WITH aliases to subqueries. This fixes #20388. #20476 (Amos Bird).
  • Fix abnormal server termination when http client goes away. #20464 (Azat Khuzhin).
  • Fix LOGICAL_ERROR for join_use_nulls=1 when JOIN contains const from SELECT. #20461 (Azat Khuzhin).
  • Check if table function view is used in expression list and throw an error. This fixes #20342. #20350 (Amos Bird).
  • Avoid invalid dereference in RANGE_HASHED() dictionary. #20345 (Azat Khuzhin).
  • Fix null dereference with join_use_nulls=1. #20344 (Azat Khuzhin).
  • Fix incorrect result of binary operations between two constant decimals of different scale. Fixes #20283. #20339 (Maksim Kita).
  • Fix too often retries of failed background tasks for ReplicatedMergeTree table engines family. This could lead to too verbose logging and increased CPU load. Fixes #20203. #20335 (alesapin).
  • Restrict to DROP or RENAME version column of *CollapsingMergeTree and ReplacingMergeTree table engines. #20300 (alesapin).
  • Fixed the behavior when in case of broken JSON we tried to read the whole file into memory which leads to exception from the allocator. Fixes #19719. #20286 (Nikita Mikhaylov).
  • Fix exception during vertical merge for MergeTree table engines family which don’t allow to perform vertical merges. Fixes #20259. #20279 (alesapin).
  • Fix rare server crash on config reload during the shutdown. Fixes #19689. #20224 (alesapin).
  • Fix CTE when using in INSERT SELECT. This fixes #20187, fixes #20195. #20211 (Amos Bird).
  • Fixes #19314. #20156 (Ivan).
  • fix toMinute function to handle special timezone correctly. #20149 (keenwolf).
  • Fix server crash after query with if function with Tuple type of then/else branches result. Tuple type must contain Array or another complex type. Fixes #18356. #20133 (alesapin).
  • The MongoDB table engine now establishes connection only when it’s going to read data. ATTACH TABLE won’t try to connect anymore. #20110 (Vitaly Baranov).
  • Bugfix in StorageJoin. #20079 (vdimir).
  • Fix the case when calculating modulo of division of negative number by small divisor, the resulting data type was not large enough to accomodate the negative result. This closes #20052. #20067 (alexey-milovidov).
  • MaterializeMySQL: Fix replication for statements that update several tables. #20066 (Håvard Kvålen).
  • Prevent “Connection refused” in docker during initialization script execution. #20012 (filimonov).
  • EmbeddedRocksDB is an experimental storage. Fix the issue with lack of proper type checking. Simplified code. This closes #19967. #19972 (alexey-milovidov).
  • Fix a segfault in function fromModifiedJulianDay when the argument type is Nullable(T) for any integral types other than Int32. #19959 (PHO).
  • BloomFilter index crash fix. Fixes #19757. #19884 (Maksim Kita).
  • Deadlock was possible if system.text_log is enabled. This fixes #19874. #19875 (alexey-milovidov).
  • Fix starting the server with tables having default expressions containing dictGet(). Allow getting return type of dictGet() without loading dictionary. #19805 (Vitaly Baranov).
  • Fix clickhouse-client abort exception while executing only select. #19790 (taiyang-li).
  • Fix a bug that moving pieces to destination table may failed in case of launching multiple clickhouse-copiers. #19743 (madianjun).
  • Background thread which executes ON CLUSTER queries might hang waiting for dropped replicated table to do something. It’s fixed. #19684 (yiguolei).

Build/Testing/Packaging Improvement

  • Allow to build ClickHouse with AVX-2 enabled globally. It gives slight performance benefits on modern CPUs. Not recommended for production and will not be supported as official build for now. #20180 (alexey-milovidov).
  • Fix some of the issues found by Coverity. See #19964. #20010 (alexey-milovidov).
  • Allow to start up with modified binary under gdb. In previous version if you set up breakpoint in gdb before start, server will refuse to start up due to failed integrity check. #21258 (alexey-milovidov).
  • Add a test for different compression methods in Kafka. #21111 (filimonov).
  • Fixed port clash from test_storage_kerberized_hdfs test. #19974 (Ilya Yatsishin).
  • Print stdout and stderr to log when failed to start docker in integration tests. Before this PR there was a very short error message in this case which didn’t help to investigate the problems. #20631 (Vitaly Baranov).

ClickHouse release 21.2

ClickHouse release v21.2.2.8-stable, 2021-02-07

Backward Incompatible Change

  • Bitwise functions (bitAnd, bitOr, etc) are forbidden for floating point arguments. Now you have to do explicit cast to integer. #19853 (Azat Khuzhin).
  • Forbid lcm/gcd for floats. #19532 (Azat Khuzhin).
  • Fix memory tracking for OPTIMIZE TABLE/merges; account query memory limits and sampling for OPTIMIZE TABLE/merges. #18772 (Azat Khuzhin).
  • Disallow floating point column as partition key, see #18421. #18464 (hexiaoting).
  • Excessive parenthesis in type definitions no longer supported, example: Array((UInt8)).

New Feature

  • Added PostgreSQL table engine (both select/insert, with support for multidimensional arrays), also as table function. Added PostgreSQL dictionary source. Added PostgreSQL database engine. #18554 (Kseniia Sumarokova).
  • Data type Nested now supports arbitrary levels of nesting. Introduced subcolumns of complex types, such as size0 in Array, null in Nullable, names of Tuple elements, which can be read without reading of whole column. #17310 (Anton Popov).
  • Added Nullable support for FlatDictionary, HashedDictionary, ComplexKeyHashedDictionary, DirectDictionary, ComplexKeyDirectDictionary, RangeHashedDictionary. #18236 (Maksim Kita).
  • Adds a new table called system.distributed_ddl_queue that displays the queries in the DDL worker queue. #17656 (Bharat Nallan).
  • Added support of mapping LDAP group names, and attribute values in general, to local roles for users from ldap user directories. #17211 (Denis Glazachev).
  • Support insert into table function cluster, and for both table functions remote and cluster, support distributing data across nodes by specify sharding key. Close #16752. #18264 (flynn).
  • Add function decodeXMLComponent to decode characters for XML. Example: SELECT decodeXMLComponent('Hello,&quot;world&quot;!') #17659. #18542 (nauta).
  • Added functions parseDateTimeBestEffortUSOrZero, parseDateTimeBestEffortUSOrNull. #19712 (Maksim Kita).
  • Add sign math function. #19527 (flynn).
  • Add information about used features (functions, table engines, etc) into system.query_log. #18495. #19371 (Kseniia Sumarokova).
  • Function formatDateTime support the %Q modification to format date to quarter. #19224 (Jianmei Zhang).
  • Support MetaKey+Enter hotkey binding in play UI. #19012 (sundyli).
  • Add three functions for map data type: 1. mapContains(map, key) to check weather map.keys include the second parameter key. 2. mapKeys(map) return all the keys in Array format 3. mapValues(map) return all the values in Array format. #18788 (hexiaoting).
  • Add log_comment setting related to #18494. #18549 (Zijie Lu).
  • Add support of tuple argument to argMin and argMax functions. #17359 (Ildus Kurbangaliev).
  • Support EXISTS VIEW syntax. #18552 (Du Chuan).
  • Add SELECT ALL syntax. closes #18706. #18723 (flynn).

Performance Improvement

  • Faster parts removal by lowering the number of stat syscalls. This returns the optimization that existed while ago. More safe interface of IDisk. This closes #19065. #19086 (alexey-milovidov).
  • Aliases declared in WITH statement are properly used in index analysis. Queries like WITH column AS alias SELECT ... WHERE alias = ... may use index now. #18896 (Amos Bird).
  • Add optimize_alias_column_prediction (on by default), that will: - Respect aliased columns in WHERE during partition pruning and skipping data using secondary indexes; - Respect aliased columns in WHERE for trivial count queries for optimize_trivial_count; - Respect aliased columns in GROUP BY/ORDER BY for optimize_aggregation_in_order/optimize_read_in_order. #16995 (sundyli).
  • Speed up aggregate function sum. Improvement only visible on synthetic benchmarks and not very practical. #19216 (alexey-milovidov).
  • Update libc++ and use another ABI to provide better performance. #18914 (Danila Kutenin).
  • Rewrite sumIf() and sum(if()) function to countIf() function when logically equivalent. #17041 (flynn).
  • Use a connection pool for S3 connections, controlled by the s3_max_connections settings. #13405 (Vladimir Chebotarev).
  • Add support for zstd long option for better compression of string columns to save space. #17184 (ygrek).
  • Slightly improve server latency by removing access to configuration on every connection. #19863 (alexey-milovidov).
  • Reduce lock contention for multiple layers of the Buffer engine. #19379 (Azat Khuzhin).
  • Support splitting Filter step of query plan into Expression + Filter pair. Together with Expression + Expression merging optimization (#17458) it may delay execution for some expressions after Filter step. #19253 (Nikolai Kochetov).

Improvement

  • SELECT count() FROM table now can be executed if only one any column can be selected from the table. This PR fixes #10639. #18233 (Vitaly Baranov).
  • Set charset to utf8mb4 when interacting with remote MySQL servers. Fixes #19795. #19800 (alexey-milovidov).
  • S3 table function now supports auto compression mode (autodetect). This closes #18754. #19793 (Vladimir Chebotarev).
  • Correctly output infinite arguments for formatReadableTimeDelta function. In previous versions, there was implicit conversion to implementation specific integer value. #19791 (alexey-milovidov).
  • Table function S3 will use global region if the region can’t be determined exactly. This closes #10998. #19750 (Vladimir Chebotarev).
  • In distributed queries if the setting async_socket_for_remote is enabled, it was possible to get stack overflow at least in debug build configuration if very deeply nested data type is used in table (e.g. Array(Array(Array(...more...)))). This fixes #19108. This change introduces minor backward incompatibility: excessive parenthesis in type definitions no longer supported, example: Array((UInt8)). #19736 (alexey-milovidov).
  • Add separate pool for message brokers (RabbitMQ and Kafka). #19722 (Azat Khuzhin).
  • Fix rare max_number_of_merges_with_ttl_in_pool limit overrun (more merges with TTL can be assigned) for non-replicated MergeTree. #19708 (alesapin).
  • Dictionary: better error message during attribute parsing. #19678 (Maksim Kita).
  • Add an option to disable validation of checksums on reading. Should never be used in production. Please do not expect any benefits in disabling it. It may only be used for experiments and benchmarks. The setting only applicable for tables of MergeTree family. Checksums are always validated for other table engines and when receiving data over network. In my observations there is no performance difference or it is less than 0.5%. #19588 (alexey-milovidov).
  • Support constant result in function multiIf. #19533 (Maksim Kita).
  • Enable function length/empty/notEmpty for datatype Map, which returns keys number in Map. #19530 (taiyang-li).
  • Add --reconnect option to clickhouse-benchmark. When this option is specified, it will reconnect before every request. This is needed for testing. #19872 (alexey-milovidov).
  • Support using the new location of .debug file. This fixes #19348. #19520 (Amos Bird).
  • toIPv6 function parses IPv4 addresses. #19518 (Bharat Nallan).
  • Add http_referer field to system.query_log, system.processes, etc. This closes #19389. #19390 (alexey-milovidov).
  • Improve MySQL compatibility by making more functions case insensitive and adding aliases. #19387 (Daniil Kondratyev).
  • Add metrics for MergeTree parts (Wide/Compact/InMemory) types. #19381 (Azat Khuzhin).
  • Allow docker to be executed with arbitrary uid. #19374 (filimonov).
  • Fix wrong alignment of values of IPv4 data type in Pretty formats. They were aligned to the right, not to the left. This closes #19184. #19339 (alexey-milovidov).
  • Allow change max_server_memory_usage without restart. This closes #18154. #19186 (alexey-milovidov).
  • The exception when function bar is called with certain NaN argument may be slightly misleading in previous versions. This fixes #19088. #19107 (alexey-milovidov).
  • Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server images. #19096 (filimonov).
  • Fixed PeekableReadBuffer: Memory limit exceed error when inserting data with huge strings. Fixes #18690. #18979 (tavplubix).
  • Docker image: several improvements for clickhouse-server entrypoint. #18954 (filimonov).
  • Add normalizeQueryKeepNames and normalizedQueryHashKeepNames to normalize queries without masking long names with ?. This helps better analyze complex query logs. #18910 (Amos Bird).
  • Check per-block checksum of the distributed batch on the sender before sending (without reading the file twice, the checksums will be verified while reading), this will avoid stuck of the INSERT on the receiver (on truncated .bin file on the sender). Avoid reading .bin files twice for batched INSERT (it was required to calculate rows/bytes to take squashing into account, now this information included into the header, backward compatible is preserved). #18853 (Azat Khuzhin).
  • Fix issues with RIGHT and FULL JOIN of tables with aggregate function states. In previous versions exception about cloneResized method was thrown. #18818 (templarzq).
  • Added prefix-based S3 endpoint settings. #18812 (Vladimir Chebotarev).
  • Add [UInt8, UInt16, UInt32, UInt64] arguments types support for bitmapTransform, bitmapSubsetInRange, bitmapSubsetLimit, bitmapContains functions. This closes #18713. #18791 (sundyli).
  • Allow CTE (Common Table Expressions) to be further aliased. Propagate CSE (Common Subexpressions Elimination) to subqueries in the same level when enable_global_with_statement = 1. This fixes #17378 . This fixes https://github.com/ClickHouse/ClickHouse/pull/16575#issuecomment-753416235 . #18684 (Amos Bird).
  • Update librdkafka to v1.6.0-RC2. Fixes #18668. #18671 (filimonov).
  • In case of unexpected exceptions automatically restart background thread which is responsible for execution of distributed DDL queries. Fixes #17991. #18285 (徐炘).
  • Updated AWS C++ SDK in order to utilize global regions in S3. #17870 (Vladimir Chebotarev).
  • Added support for WITH ... [AND] [PERIODIC] REFRESH [interval_in_sec] clause when creating LIVE VIEW tables. #14822 (vzakaznikov).
  • Restrict MODIFY TTL queries for MergeTree tables created in old syntax. Previously the query succeeded, but actually it had no effect. #19064 (Anton Popov).

Bug Fix

  • Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes #18364. #18373 (Amos Bird).
  • Fix starting the server with tables having default expressions containing dictGet(). Allow getting return type of dictGet() without loading dictionary. #19805 (Vitaly Baranov).
  • Fix server crash after query with if function with Tuple type of then/else branches result. Tuple type must contain Array or another complex type. Fixes #18356. #20133 (alesapin).
  • MaterializeMySQL (experimental feature): Fix replication for statements that update several tables. #20066 (Håvard Kvålen).
  • Prevent “Connection refused” in docker during initialization script execution. #20012 (filimonov).
  • EmbeddedRocksDB is an experimental storage. Fix the issue with lack of proper type checking. Simplified code. This closes #19967. #19972 (alexey-milovidov).
  • Fix a segfault in function fromModifiedJulianDay when the argument type is Nullable(T) for any integral types other than Int32. #19959 (PHO).
  • The function greatCircleAngle returned inaccurate results in previous versions. This closes #19769. #19789 (alexey-milovidov).
  • Fix rare bug when some replicated operations (like mutation) cannot process some parts after data corruption. Fixes #19593. #19702 (alesapin).
  • Background thread which executes ON CLUSTER queries might hang waiting for dropped replicated table to do something. It’s fixed. #19684 (yiguolei).
  • Fix wrong deserialization of columns description. It makes INSERT into a table with a column named \ impossible. #19479 (alexey-milovidov).
  • Mark distributed batch as broken in case of empty data block in one of files. #19449 (Azat Khuzhin).
  • Fixed very rare bug that might cause mutation to hang after DROP/DETACH/REPLACE/MOVE PARTITION. It was partially fixed by #15537 for the most cases. #19443 (tavplubix).
  • Fix possible error Extremes transform was already added to pipeline. Fixes #14100. #19430 (Nikolai Kochetov).
  • Fix default value in join types with non-zero default (e.g. some Enums). Closes #18197. #19360 (vdimir).
  • Do not mark file for distributed send as broken on EOF. #19290 (Azat Khuzhin).
  • Fix leaking of pipe fd for async_socket_for_remote. #19153 (Azat Khuzhin).
  • Fix infinite reading from file in ORC format (was introduced in #10580). Fixes #19095. #19134 (Nikolai Kochetov).
  • Fix issue in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes #18913. #19123 (alesapin).
  • Fix startup bug when clickhouse was not able to read compression codec from LowCardinality(Nullable(...)) and throws exception Attempt to read after EOF. Fixes #18340. #19101 (alesapin).
  • Simplify the implementation of tupleHammingDistance. Support for tuples of any equal length. Fixes #19029. #19084 (Nikolai Kochetov).
  • Make sure groupUniqArray returns correct type for argument of Enum type. This closes #17875. #19019 (alexey-milovidov).
  • Fix possible error Expected single dictionary argument for function if use function ignore with LowCardinality argument. Fixes #14275. #19016 (Nikolai Kochetov).
  • Fix inserting of LowCardinality column to table with TinyLog engine. Fixes #18629. #19010 (Nikolai Kochetov).
  • Fix minor issue in JOIN: Join tries to materialize const columns, but our code waits for them in other places. #18982 (Nikita Mikhaylov).
  • Disable optimize_move_functions_out_of_any because optimization is not always correct. This closes #18051. This closes #18973. #18981 (alexey-milovidov).
  • Fix possible exception QueryPipeline stream: different number of columns caused by merging of query plan’s Expression steps. Fixes #18190. #18980 (Nikolai Kochetov).
  • Fixed very rare deadlock at shutdown. #18977 (tavplubix).
  • Fixed rare crashes when server run out of memory. #18976 (tavplubix).
  • Fix incorrect behavior when ALTER TABLE ... DROP PART 'part_name' query removes all deduplication blocks for the whole partition. Fixes #18874. #18969 (alesapin).
  • Fixed issue #18894 Add a check to avoid exception when long column alias(‘table.column’ style, usually auto-generated by BI tools like Looker) equals to long table name. #18968 (Daniel Qin).
  • Fix error Task was not found in task queue (possible only for remote queries, with async_socket_for_remote = 1). #18964 (Nikolai Kochetov).
  • Fix bug when mutation with some escaped text (like ALTER ... UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1') serialized incorrectly. Fixes #18878. #18944 (alesapin).
  • ATTACH PARTITION will reset mutations. #18804. #18935 (fastio).
  • Fix issue with bitmapOrCardinality that may lead to nullptr dereference. This closes #18911. #18912 (sundyli).
  • Fixed Attempt to read after eof error when trying to CAST NULL from Nullable(String) to Nullable(Decimal(P, S)). Now function CAST returns NULL when it cannot parse decimal from nullable string. Fixes #7690. #18718 (Winter Zhang).
  • Fix data type convert issue for MySQL engine. #18124 (bo zeng).
  • Fix clickhouse-client abort exception while executing only select. #19790 (taiyang-li).

Build/Testing/Packaging Improvement

ClickHouse release 21.1

ClickHouse release v21.1.3.32-stable, 2021-02-03

Bug Fix

  • BloomFilter index crash fix. Fixes #19757. #19884 (Maksim Kita).
  • Fix crash when pushing down predicates to union distinct subquery. This fixes #19855. #19861 (Amos Bird).
  • Fix filtering by UInt8 greater than 127. #19799 (Anton Popov).
  • In previous versions, unusual arguments for function arrayEnumerateUniq may cause crash or infinite loop. This closes #19787. #19788 (alexey-milovidov).
  • Fixed stack overflow when using accurate comparison of arithmetic type with string type. #19773 (tavplubix).
  • Fix crash when nested column name was used in WHERE or PREWHERE. Fixes #19755. #19763 (Nikolai Kochetov).
  • Fix a segmentation fault in bitmapAndnot function. Fixes #19668. #19713 (Maksim Kita).
  • Some functions with big integers may cause segfault. Big integers is experimental feature. This closes #19667. #19672 (alexey-milovidov).
  • Fix wrong result of function neighbor for LowCardinality argument. Fixes #10333. #19617 (Nikolai Kochetov).
  • Fix use-after-free of the CompressedWriteBuffer in Connection after disconnect. #19599 (Azat Khuzhin).
  • DROP/DETACH TABLE table ON CLUSTER cluster SYNC query might hang, it’s fixed. Fixes #19568. #19572 (tavplubix).
  • Query CREATE DICTIONARY id expression fix. #19571 (Maksim Kita).
  • Fix SIGSEGV with merge_tree_min_rows_for_concurrent_read/merge_tree_min_bytes_for_concurrent_read=0/UINT64_MAX. #19528 (Azat Khuzhin).
  • Buffer overflow (on memory read) was possible if addMonth function was called with specifically crafted arguments. This fixes #19441. This fixes #19413. #19472 (alexey-milovidov).
  • Uninitialized memory read was possible in encrypt/decrypt functions if empty string was passed as IV. This closes #19391. #19397 (alexey-milovidov).
  • Fix possible buffer overflow in Uber H3 library. See https://github.com/uber/h3/issues/392. This closes #19219. #19383 (alexey-milovidov).
  • Fix system.parts _state column (LOGICAL_ERROR when querying this column, due to incorrect order). #19346 (Azat Khuzhin).
  • Fixed possible wrong result or segfault on aggregation when Materialized View and its target table have different structure. Fixes #18063. #19322 (tavplubix).
  • Fix error Cannot convert column now64() because it is constant but values of constants are different in source and result. Continuation of #7156. #19316 (Nikolai Kochetov).
  • Fix bug when concurrent ALTER and DROP queries may hang while processing ReplicatedMergeTree table. #19237 (alesapin).
  • Fixed There is no checkpoint error when inserting data through http interface using Template or CustomSeparated format. Fixes #19021. #19072 (tavplubix).
  • Disable constant folding for subqueries on the analysis stage, when the result cannot be calculated. #18446 (Azat Khuzhin).
  • Mutation might hang waiting for some non-existent part after MOVE or REPLACE PARTITION or, in rare cases, after DETACH or DROP PARTITION. It’s fixed. #15537 (tavplubix).

ClickHouse release v21.1.2.15-stable 2021-01-18

Backward Incompatible Change

  • The setting input_format_null_as_default is enabled by default. #17525 (alexey-milovidov).
  • Check settings constraints for profile settings from config. Server will fail to start if users.xml contain settings that do not meet corresponding constraints. #18486 (tavplubix).
  • Restrict ALTER MODIFY SETTING from changing storage settings that affects data parts (write_final_mark and enable_mixed_granularity_parts). #18306 (Amos Bird).
  • Set insert_quorum_parallel to 1 by default. It is significantly more convenient to use than “sequential” quorum inserts. But if you rely to sequential consistency, you should set the setting back to zero. #17567 (alexey-milovidov).
  • Remove sumburConsistentHash function. This closes #18120. #18656 (alexey-milovidov).
  • Removed aggregate functions timeSeriesGroupSum, timeSeriesGroupRateSum because a friend of mine said they never worked. This fixes #16869. If you have luck using these functions, write a email to [email protected]. #17423 (alexey-milovidov).
  • Prohibit toUnixTimestamp(Date()) (before it just returns UInt16 representation of Date). #17376 (Azat Khuzhin).
  • Allow using extended integer types (Int128, Int256, UInt256) in avg and avgWeighted functions. Also allow using different types (integer, decimal, floating point) for value and for weight in avgWeighted function. This is a backward-incompatible change: now the avg and avgWeighted functions always return Float64 (as documented). Before this change the return type for Decimal arguments was also Decimal. #15419 (Mike).
  • Expression toUUID(N) no longer works. Replace with toUUID('00000000-0000-0000-0000-000000000000'). This change is motivated by non-obvious results of toUUID(N) where N is non zero.
  • SSL Certificates with incorrect “key usage” are rejected. In previous versions they are used to work. See #19262.
  • incl references to substitutions file (/etc/metrika.xml) were removed from the default config (<remote_servers>, <zookeeper>, <macros>, <compression>, <networks>). If you were using substitutions file and were relying on those implicit references, you should put them back manually and explicitly by adding corresponding sections with incl="..." attributes before the update. See #18740 (alexey-milovidov).

New Feature

  • Implement gRPC protocol in ClickHouse. #15111 (Vitaly Baranov).
  • Allow to use multiple zookeeper clusters. #17070 (fastio).
  • Implemented REPLACE TABLE and CREATE OR REPLACE TABLE queries. #18521 (tavplubix).
  • Implement UNION DISTINCT and treat the plain UNION clause as UNION DISTINCT by default. Add a setting union_default_mode that allows to treat it as UNION ALL or require explicit mode specification. #16338 (flynn).
  • Added function accurateCastOrNull. This closes #10290. Add type conversions in x IN (subquery) expressions. This closes #10266. #16724 (Maksim Kita).
  • IP Dictionary supports IPv4 / IPv6 types directly. #17571 (vdimir).
  • IP Dictionary supports key fetching. Resolves #18241. #18480 (vdimir).
  • Add *.zst compression/decompression support for data import and export. It enables using *.zst in file() function and Content-encoding: zstd in HTTP client. This closes #16791 . #17144 (Abi Palagashvili).
  • Added mannWitneyUTest, studentTTest and welchTTest aggregate functions. Refactored rankCorr a bit. #16883 (Nikita Mikhaylov).
  • Add functions countMatches/countMatchesCaseInsensitive. #17459 (Azat Khuzhin).
  • Implement countSubstrings()/countSubstringsCaseInsensitive()/countSubstringsCaseInsensitiveUTF8() (Count the number of substring occurrences). #17347 (Azat Khuzhin).
  • Add information about used databases, tables and columns in system.query_log. Add query_kind and normalized_query_hash fields. #17726 (Amos Bird).
  • Add a setting optimize_on_insert. When enabled, do the same transformation for INSERTed block of data as if merge was done on this block (e.g. Replacing, Collapsing, Aggregating…). This setting is enabled by default. This can influence Materialized View and MaterializeMySQL behaviour (see detailed description). This closes #10683. #16954 (Kruglov Pavel).
  • Kerberos Authenticaiton for HDFS. #16621 (Ilya Golshtein).
  • Support SHOW SETTINGS statement to show parameters in system.settings. SHOW CHANGED SETTINGS and LIKE/ILIKE clause are also supported. #18056 (Jianmei Zhang).
  • Function position now supports POSITION(needle IN haystack) synax for SQL compatibility. This closes #18701. … #18779 (Jianmei Zhang).
  • Now we have a new storage setting max_partitions_to_read for tables in the MergeTree family. It limits the max number of partitions that can be accessed in one query. A user setting force_max_partition_limit is also added to enforce this constraint. #18712 (Amos Bird).
  • Add query_id column to system.part_log for inserted parts. Closes #10097. #18644 (flynn).
  • Allow create table as select with columns specification. Example CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1;. #18060 (Maksim Kita).
  • Added arrayMin, arrayMax, arrayAvg aggregation functions. #18032 (Maksim Kita).
  • Implemented ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ... query. It creates new table with provided structure and attaches table data from provided directory in user_files. #17903 (tavplubix).
  • Add mutation support for StorageMemory. This closes #9117. #15127 (flynn).
  • Support syntax EXISTS DATABASE name. #18458 (Du Chuan).
  • Support builtin function isIPv4String && isIPv6String like MySQL. #18349 (Du Chuan).
  • Add a new setting insert_distributed_one_random_shard = 1 to allow insertion into multi-sharded distributed table without any distributed key. #18294 (Amos Bird).
  • Add settings min_compress_block_size and max_compress_block_size to MergeTreeSettings, which have higher priority than the global settings and take effect when they are set. close 13890. #17867 (flynn).
  • Add support for 64bit roaring bitmaps. #17858 (Andy Yang).
  • Extended OPTIMIZE ... DEDUPLICATE syntax to allow explicit (or implicit with asterisk/column transformers) list of columns to check for duplicates on. … #17846 (Vasily Nemkov).
  • Added functions toModifiedJulianDay, fromModifiedJulianDay, toModifiedJulianDayOrNull, and fromModifiedJulianDayOrNull. These functions convert between Proleptic Gregorian calendar date and Modified Julian Day number. #17750 (PHO).
  • Add ability to use custom TLD list: added functions firstSignificantSubdomainCustom, cutToFirstSignificantSubdomainCustom. #17748 (Azat Khuzhin).
  • Add support for PROXYv1 protocol to wrap native TCP interface. Allow quotas to be keyed by proxy-forwarded IP address (applied for PROXYv1 address and for X-Forwarded-For from HTTP interface). This is useful when you provide access to ClickHouse only via trusted proxy (e.g. CloudFlare) but want to account user resources by their original IP addresses. This fixes #17268. #17707 (alexey-milovidov).
  • Now clickhouse-client supports opening EDITOR to edit commands. Alt-Shift-E. #17665 (Amos Bird).
  • Add function encodeXMLComponent to escape characters to place string into XML text node or attribute. #17659 (nauta).
  • Introduce DETACH TABLE/VIEW ... PERMANENTLY syntax, so that after restarting the table does not reappear back automatically on restart (only by explicit request). The table can still be attached back using the short syntax ATTACH TABLE. Implements #5555. Fixes #13850. #17642 (filimonov).
  • Add asynchronous metrics on total amount of rows, bytes and parts in MergeTree tables. This fix #11714. #17639 (flynn).
  • Add settings limit and offset for out-of-SQL pagination: #16176 They are useful for building APIs. These two settings will affect SELECT query as if it is added like select * from (your_original_select_query) t limit xxx offset xxx;. #17633 (hexiaoting).
  • Provide a new aggregator combinator : -SimpleState to build SimpleAggregateFunction types via query. It’s useful for defining MaterializedView of AggregatingMergeTree engine, and will benefit projections too. #16853 (Amos Bird).
  • Added queries-file parameter for clickhouse-client and clickhouse-local. #15930 (Maksim Kita).
  • Added query parameter for clickhouse-benchmark. #17832 (Maksim Kita).
  • EXPLAIN AST now support queries other then SELECT. #18136 (taiyang-li).

Experimental Feature

  • Added functions for calculation of minHash and simHash of text n-grams and shingles. They are intended for semi-duplicate search. Also functions bitHammingDistance and tupleHammingDistance are added. #7649 (flynn).
  • Add new data type Map. See #1841. First version for Map only supports String type of key and value. #15806 (hexiaoting).
  • Implement alternative SQL parser based on ANTLR4 runtime and generated from EBNF grammar. #11298 (Ivan).

Performance Improvement

  • New IP Dictionary implementation with lower memory consumption, improved performance for some cases, and fixed bugs. #16804 (vdimir).
  • Parallel formatting for data export. #11617 (Nikita Mikhaylov).
  • LDAP integration: Added verification_cooldown parameter in LDAP server connection configuration to allow caching of successful “bind” attempts for configurable period of time. #15988 (Denis Glazachev).
  • Add --no-system-table option for clickhouse-local to run without system tables. This avoids initialization of DateLUT that may take noticeable amount of time (tens of milliseconds) at startup. #18899 (alexey-milovidov).
  • Replace PODArray with PODArrayWithStackMemory in AggregateFunctionWindowFunnelData to improve windowFunnel function performance. #18817 (flynn).
  • Don’t send empty blocks to shards on synchronous INSERT into Distributed table. This closes #14571. #18775 (alexey-milovidov).
  • Optimized read for StorageMemory. #18052 (Maksim Kita).
  • Using Dragonbox algorithm for float to string conversion instead of ryu. This improves performance of float to string conversion significantly. #17831 (Maksim Kita).
  • Speedup IPv6CIDRToRange implementation. #17569 (vdimir).
  • Add remerge_sort_lowered_memory_bytes_ratio setting (If memory usage after remerge does not reduced by this ratio, remerge will be disabled). #17539 (Azat Khuzhin).
  • Improve performance of AggregatingMergeTree with SimpleAggregateFunction(String) in PK. #17109 (Azat Khuzhin).
  • Now the -If combinator is devirtualized, and count is properly vectorized. It is for this PR. #17043 (Amos Bird).
  • Fix performance of reading from Merge tables over huge number of MergeTree tables. Fixes #7748. #16988 (Anton Popov).
  • Improved performance of function repeat. #16937 (satanson).
  • Slightly improved performance of float parsing. #16809 (Maksim Kita).
  • Add possibility to skip merged partitions for OPTIMIZE TABLE ... FINAL. #15939 (Kruglov Pavel).
  • Integrate with fast_float from Daniel Lemire to parse floating point numbers. #16787 (Maksim Kita). It is not enabled, because performance its performance is still lower than rough float parser in ClickHouse.
  • Fix max_distributed_connections (affects prefer_localhost_replica = 1 and max_threads != max_distributed_connections). #17848 (Azat Khuzhin).
  • Adaptive choice of single/multi part upload when sending data to S3. Single part upload is controlled by a new setting max_single_part_upload_size. #17934 (Pavel Kovalenko).
  • Support for async tasks in PipelineExecutor. Initial support of async sockets for remote queries. #17868 (Nikolai Kochetov).
  • Allow to use optimize_move_to_prewhere optimization with compact parts, when sizes of columns are unknown. #17330 (Anton Popov).

Improvement

  • Avoid deadlock when executing INSERT SELECT into itself from a table with TinyLog or Log table engines. This closes #6802. This closes #18691. This closes #16812. This closes #14570. #15260 (alexey-milovidov).
  • Support SHOW CREATE VIEW name syntax like MySQL. #18095 (Du Chuan).
  • All queries of type Decimal * Float or vice versa are allowed, including aggregate ones (e.g. SELECT sum(decimal_field * 1.1) or SELECT dec_col * float_col), the result type is Float32 or Float64. #18145 (Mike).
  • Improved minimal Web UI: add history; add sharing support; avoid race condition of different requests; add request in-flight and ready indicators; add favicon; detect Ctrl+Enter if textarea is not in focus. #17293 #17770 (alexey-milovidov).
  • clickhouse-server didn’t send close request to ZooKeeper server. #16837 (alesapin).
  • Avoid server abnormal termination in case of too low memory limits (max_memory_usage = 1 / max_untracked_memory = 1). #17453 (Azat Khuzhin).
  • Fix non-deterministic result of windowFunnel function in case of same timestamp for different events. #18884 (Fuwang Hu).
  • Docker: Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server Docker images. #19096 (filimonov).
  • Asynchronous INSERTs to Distributed tables: Two new settings (by analogy with MergeTree family) has been added: - fsync_after_insert - Do fsync for every inserted. Will decreases performance of inserts. - fsync_directories - Do fsync for temporary directory (that is used for async INSERT only) after all operations (writes, renames, etc.). #18864 (Azat Khuzhin).
  • SYSTEM KILL command started to work in Docker. This closes #18847. #18848 (alexey-milovidov).
  • Expand macros in the zk path when executing FETCH PARTITION. #18839 (fastio).
  • Apply ALTER TABLE <replicated_table> ON CLUSTER MODIFY SETTING ... to all replicas. Because we don’t replicate such alter commands. #18789 (Amos Bird).
  • Allow column transformer EXCEPT to accept a string as regular expression matcher. This resolves #18685 . #18699 (Amos Bird).
  • Fix SimpleAggregateFunction in SummingMergeTree. Now it works like AggregateFunction. In previous versions values were summed together regardless to the aggregate function. This fixes #18564 . #8052. #18637 (Amos Bird). Another fix of using SimpleAggregateFunction in SummingMergeTree. This fixes #18676 . #18677 (Amos Bird).
  • Fixed assertion error inside allocator in case when last argument of function bar is NaN. Now simple ClickHouse’s exception is being thrown. This fixes #17876. #18520 (Nikita Mikhaylov).
  • Fix usability issue: no newline after exception message in some tools. #18444 (alexey-milovidov).
  • Add ability to modify primary and partition key column type from LowCardinality(Type) to Type and vice versa. Also add an ability to modify primary key column type from EnumX to IntX type. Fixes #5604. #18362 (alesapin).
  • Implement untuple field access. #18133. #18309 (hexiaoting).
  • Allow to parse Array fields from CSV if it is represented as a string containing array that was serialized as nested CSV. Example: "[""Hello"", ""world"", ""42"""" TV""]" will parse as ['Hello', 'world', '42" TV']. Allow to parse array in CSV in a string without enclosing braces. Example: "'Hello', 'world', '42"" TV'" will parse as ['Hello', 'world', '42" TV']. #18271 (alexey-milovidov).
  • Make better adaptive granularity calculation for merge tree wide parts. #18223 (alesapin).
  • Now clickhouse install could work on Mac. The problem was that there is no procfs on this platform. #18201 (Nikita Mikhaylov).
  • Better hints for SHOW ... query syntax. #18183 (Du Chuan).
  • Array aggregation arrayMin, arrayMax, arraySum, arrayAvg support for Int128, Int256, UInt256. #18147 (Maksim Kita).
  • Add disk to Set and Join storage settings. #18112 (Grigory Pervakov).
  • Access control: Now table function merge() requires current user to have SELECT privilege on each table it receives data from. This PR fixes #16964. #18104 #17983 (Vitaly Baranov).
  • Temporary tables are visible in the system tables system.tables and system.columns now only in those session where they have been created. The internal database _temporary_and_external_tables is now hidden in those system tables; temporary tables are shown as tables with empty database with the is_temporary flag set instead. #18014 (Vitaly Baranov).
  • Fix clickhouse-client rendering issue when the size of terminal window changes. #18009 (Amos Bird).
  • Decrease log verbosity of the events when the client drops the connection from Warning to Information. #18005 (filimonov).
  • Forcibly removing empty or bad metadata files from filesystem for DiskS3. S3 is an experimental feature. #17935 (Pavel Kovalenko).
  • Access control: allow_introspection_functions=0 prohibits usage of introspection functions but doesn’t prohibit giving grants for them anymore (the grantee will need to set allow_introspection_functions=1 for himself to be able to use that grant). Similarly allow_ddl=0 prohibits usage of DDL commands but doesn’t prohibit giving grants for them anymore. #17908 (Vitaly Baranov).
  • Usability improvement: hints for column names. #17112. #17857 (fastio).
  • Add diagnostic information when two merge tables try to read each other’s data. #17854 (徐炘).
  • Let the possibility to override timeout value for running script using the ClickHouse docker image. #17818 (Guillaume Tassery).
  • Check system log tables’ engine definition grammar to prevent some configuration errors. Notes that this grammar check is not semantical, that means such mistakes as non-existent columns / expression functions would be not found out util the table is created. #17739 (Du Chuan).
  • Removed exception throwing at RabbitMQ table initialization if there was no connection (it will be reconnecting in the background). #17709 (Kseniia Sumarokova).
  • Do not ignore server memory limits during Buffer flush. #17646 (Azat Khuzhin).
  • Switch to patched version of RocksDB (from ClickHouse-Extras) to fix use-after-free error. #17643 (Nikita Mikhaylov).
  • Added an offset to exception message for parallel parsing. This fixes #17457. #17641 (Nikita Mikhaylov).
  • Don’t throw “Too many parts” error in the middle of INSERT query. #17566 (alexey-milovidov).
  • Allow query parameters in UPDATE statement of ALTER query. Fixes #10976. #17563 (alexey-milovidov).
  • Query obfuscator: avoid usage of some SQL keywords for identifier names. #17526 (alexey-milovidov).
  • Export current max ddl entry executed by DDLWorker via server metric. It’s useful to check if DDLWorker hangs somewhere. #17464 (Amos Bird).
  • Export asynchronous metrics of all servers current threads. It’s useful to track down issues like this. #17463 (Amos Bird).
  • Include dynamic columns like MATERIALIZED / ALIAS for wildcard query when settings asterisk_include_materialized_columns and asterisk_include_alias_columns are turned on. #17462 (Ken Chen).
  • Allow specifying TTL to remove old entries from system log tables, using the <ttl> attribute in config.xml. #17438 (Du Chuan).
  • Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the interface column of the tablesystem.query_log): 4 for MySQL, and 5 for PostgreSQL, instead of formerly used 1 which is now used for the native protocol only. #17437 (Vitaly Baranov).
  • Fix parsing of SETTINGS clause of the INSERT ... SELECT ... SETTINGS query. #17414 (Azat Khuzhin).
  • Correctly account memory in RadixSort. #17412 (Nikita Mikhaylov).
  • Add eof check in receiveHello in server to prevent getting Attempt to read after eof exception. #17365 (Kruglov Pavel).
  • Avoid possible stack overflow in bigint conversion. Big integers are experimental. #17269 (flynn).
  • Now set indices will work with GLOBAL IN. This fixes #17232 , #5576 . #17253 (Amos Bird).
  • Add limit for http redirects in request to S3 storage (s3_max_redirects). #17220 (ianton-ru).
  • When -OrNull combinator combined -If, -Merge, -MergeState, -State combinators, we should put -OrNull in front. #16935 (flynn).
  • Support HTTP proxy and HTTPS S3 endpoint configuration. #16861 (Pavel Kovalenko).
  • Added proper authentication using environment, ~/.aws and AssumeRole for S3 client. #16856 (Vladimir Chebotarev).
  • Add more OpenTelemetry spans. Add an example of how to export the span data to Zipkin. #16535 (Alexander Kuzmenkov).
  • Cache dictionaries: Completely eliminate callbacks and locks for acquiring them. Keys are not divided into “not found” and “expired”, but stored in the same map during query. #14958 (Nikita Mikhaylov).
  • Fix never worked fsync_part_directory/fsync_after_insert/in_memory_parts_insert_sync (experimental feature). #18845 (Azat Khuzhin).
  • Allow using Atomic engine for nested database of MaterializeMySQL engine. #14849 (tavplubix).

Bug Fix

  • Fix the issue when server can stop accepting connections in very rare cases. #17542 (Amos Bird, alexey-milovidov).
  • Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes #18364. #18373 (Amos Bird).
  • Fix possible wrong index analysis when the types of the index comparison are different. This fixes #17122. #17145 (Amos Bird).
  • Disable write with AIO during merges because it can lead to extremely rare data corruption of primary key columns during merge. #18481 (alesapin).
  • Restrict merges from wide to compact parts. In case of vertical merge it led to broken result part. #18381 (Anton Popov).
  • Fix possible incomplete query result while reading from MergeTree* in case of read backoff (message <Debug> MergeTreeReadPool: Will lower number of threads in logs). Was introduced in #16423. Fixes #18137. #18216 (Nikolai Kochetov).
  • Fix use after free bug in rocksdb library. #18862 (sundyli).
  • Fix infinite reading from file in ORC format (was introduced in #10580). Fixes #19095. #19134 (Nikolai Kochetov).
  • Fix bug in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes #18913. #19123 (alesapin).
  • Fix startup bug when clickhouse was not able to read compression codec from LowCardinality(Nullable(...)) and throws exception Attempt to read after EOF. Fixes #18340. #19101 (alesapin).
  • Restrict MODIFY TTL queries for MergeTree tables created in old syntax. Previously the query succeeded, but actually it had no effect. #19064 (Anton Popov).
  • Make sure groupUniqArray returns correct type for argument of Enum type. This closes #17875. #19019 (alexey-milovidov).
  • Fix possible error Expected single dictionary argument for function if use function ignore with LowCardinality argument. Fixes #14275. #19016 (Nikolai Kochetov).
  • Fix inserting of LowCardinality column to table with TinyLog engine. Fixes #18629. #19010 (Nikolai Kochetov).
  • Join tries to materialize const columns, but our code wants them in other places. #18982 (Nikita Mikhaylov).
  • Disable optimize_move_functions_out_of_any because optimization is not always correct. This closes #18051. This closes #18973. #18981 (alexey-milovidov).
  • Fix possible exception QueryPipeline stream: different number of columns caused by merging of query plan’s Expression steps. Fixes #18190. #18980 (Nikolai Kochetov).
  • Fixed very rare deadlock at shutdown. #18977 (tavplubix).
  • Fix incorrect behavior when ALTER TABLE ... DROP PART 'part_name' query removes all deduplication blocks for the whole partition. Fixes #18874. #18969 (alesapin).
  • Attach partition should reset the mutation. #18804. #18935 (fastio).
  • Fix issue with bitmapOrCardinality that may lead to nullptr dereference. This closes #18911. #18912 (sundyli).
  • Fix possible hang at shutdown in clickhouse-local. This fixes #18891. #18893 (alexey-milovidov).
  • Queries for external databases (MySQL, ODBC, JDBC) were incorrectly rewritten if there was an expression in form of x IN table. This fixes #9756. #18876 (alexey-milovidov).
  • Fix *If combinator with unary function and Nullable types. #18806 (Azat Khuzhin).
  • Fix the issue that asynchronous distributed INSERTs can be rejected by the server if the setting network_compression_method is globally set to non-default value. This fixes #18741. #18776 (alexey-milovidov).
  • Fixed Attempt to read after eof error when trying to CAST NULL from Nullable(String) to Nullable(Decimal(P, S)). Now function CAST returns NULL when it cannot parse decimal from nullable string. Fixes #7690. #18718 (Winter Zhang).
  • Fix minor issue with logging. #18717 (sundyli).
  • Fix removing of empty parts in ReplicatedMergeTree tables, created with old syntax. Fixes #18582. #18614 (Anton Popov).
  • Fix previous bug when date overflow with different values. Strict Date value limit to “2106-02-07”, cast date > “2106-02-07” to value 0. #18565 (hexiaoting).
  • Add FixedString data type support for replication from MySQL. Replication from MySQL is an experimental feature. This patch fixes #18450 Also fixes #6556. #18553 (awesomeleo).
  • Fix possible Pipeline stuck error while using ORDER BY after subquery with RIGHT or FULL join. #18550 (Nikolai Kochetov).
  • Fix bug which may lead to ALTER queries hung after corresponding mutation kill. Found by thread fuzzer. #18518 (alesapin).
  • Proper support for 12AM in parseDateTimeBestEffort function. This fixes #18402. #18449 (vladimir-golovchenko).
  • Fixed value is too short error when executing toType(...) functions (toDate, toUInt32, etc) with argument of type Nullable(String). Now such functions return NULL on parsing errors instead of throwing exception. Fixes #7673. #18445 (tavplubix).
  • Fix the unexpected behaviour of SHOW TABLES. #18431 (fastio).
  • Fix -SimpleState combinator generates incompatible arugment type and return type. #18404 (Amos Bird).
  • Fix possible race condition in concurrent usage of Set or Join tables and selects from system.tables. #18385 (alexey-milovidov).
  • Fix filling table system.settings_profile_elements. This PR fixes #18231. #18379 (Vitaly Baranov).
  • Fix possible crashes in aggregate functions with combinator Distinct, while using two-level aggregation. Fixes #17682. #18365 (Anton Popov).
  • Fixed issue when clickhouse-odbc-bridge process is unreachable by server on machines with dual IPv4/IPv6 stack; Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes of the odbc-bridge process; Possibly closes #14489. #18278 (Denis Glazachev).
  • Access control: SELECT count() FROM table now can be executed if the user has access to at least single column from a table. This PR fixes #10639. #18233 (Vitaly Baranov).
  • Access control: SELECT JOIN now requires the SELECT privilege on each of the joined tables. This PR fixes #17654. #18232 (Vitaly Baranov).
  • Fix key comparison between Enum and Int types. This fixes #17989. #18214 (Amos Bird).
  • Replication from MySQL (experimental feature). Fixes #18186 Fixes #16372 Fix unique key convert issue in MaterializeMySQL database engine. #18211 (Winter Zhang).
  • Fix inconsistency for queries with both WITH FILL and WITH TIES #17466. #18188 (hexiaoting).
  • Fix inserting a row with default value in case of parsing error in the last column. Fixes #17712. #18182 (Jianmei Zhang).
  • Fix Unknown setting profile error on attempt to set settings profile. #18167 (tavplubix).
  • Fix error when query MODIFY COLUMN ... REMOVE TTL doesn’t actually remove column TTL. #18130 (alesapin).
  • Fixed std::out_of_range: basic_string in S3 URL parsing. #18059 (Vladimir Chebotarev).
  • Fix comparison of DateTime64 and Date. Fixes #13804 and #11222. … #18050 (Vasily Nemkov).
  • Replication from MySQL (experimental feature): Fixes #15187 Fixes #17912 support convert MySQL prefix index for MaterializeMySQL. #17944 (Winter Zhang).
  • When server log rotation was configured using logger.size parameter with numeric value larger than 2^32, the logs were not rotated properly. This is fixed. #17905 (Alexander Kuzmenkov).
  • Trivial query optimization was producing wrong result if query contains ARRAY JOIN (so query is actually non trivial). #17887 (sundyli).
  • Fix possible segfault in topK aggregate function. This closes #17404. #17845 (Maksim Kita).
  • WAL (experimental feature): Do not restore parts from WAL if in_memory_parts_enable_wal is disabled. #17802 (detailyang).
  • Exception message about max table size to drop was displayed incorrectly. #17764 (alexey-milovidov).
  • Fixed possible segfault when there is not enough space when inserting into Distributed table. #17737 (tavplubix).
  • Fixed problem when ClickHouse fails to resume connection to MySQL servers. #17681 (Alexander Kazakov).
  • Windows: Fixed Function not implemented error when executing RENAME query in Atomic database with ClickHouse running on Windows Subsystem for Linux. Fixes #17661. #17664 (tavplubix).
  • In might be determined incorrectly if cluster is circular- (cross-) replicated or not when executing ON CLUSTER query due to race condition when pool_size > 1. It’s fixed. #17640 (tavplubix).
  • Fix empty system.stack_trace table when server is running in daemon mode. #17630 (Amos Bird).
  • Exception fmt::v7::format_error can be logged in background for MergeTree tables. This fixes #17613. #17615 (alexey-milovidov).
  • When clickhouse-client is used in interactive mode with multiline queries, single line comment was erronously extended till the end of query. This fixes #13654. #17565 (alexey-milovidov).
  • Fix alter query hang when the corresponding mutation was killed on the different replica. Fixes #16953. #17499 (alesapin).
  • Fix issue with memory accounting when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. #17496 (alesapin).
  • Fix ORDER BY with enabled setting optimize_redundant_functions_in_order_by. #17471 (Anton Popov).
  • Fix duplicates after DISTINCT which were possible because of incorrect optimization. Fixes #17294. #17296 (li chengxiang). #17439 (Nikolai Kochetov).
  • Fixed high CPU usage in background tasks of *MergeTree tables. #17416 (tavplubix).
  • Fix possible crash while reading from JOIN table with LowCardinality types. Fixes #17228. #17397 (Nikolai Kochetov).
  • Replication from MySQL (experimental feature): Fixes #16835 try fix miss match header with MySQL SHOW statement. #17366 (Winter Zhang).
  • Fix nondeterministic functions with predicate optimizer. This fixes #17244. #17273 (Winter Zhang).
  • Fix possible Unexpected packet Data received from client error for Distributed queries with LIMIT. #17254 (Azat Khuzhin).
  • Fix set index invalidation when there are const columns in the subquery. This fixes #17246. #17249 (Amos Bird).
  • clickhouse-copier: Fix for non-partitioned tables #15235. #17248 (Qi Chen).
  • Fixed possible not-working mutations for parts stored on S3 disk (experimental feature). #17227 (Pavel Kovalenko).
  • Bug fix for funciton fuzzBits, related issue: #16980. #17051 (hexiaoting).
  • Fix optimize_distributed_group_by_sharding_key for query with OFFSET only. #16996 (Azat Khuzhin).
  • Fix queries from Merge tables over Distributed tables with JOINs. #16993 (Azat Khuzhin).
  • Fix order by optimization with monotonic functions. Fixes #16107. #16956 (Anton Popov).
  • Fix incorrect comparison of types DateTime64 with different scales. Fixes #16655#16952 (Vasily Nemkov).
  • Fix optimization of group by with enabled setting optimize_aggregators_of_group_by_keys and joins. Fixes #12604. #16951 (Anton Popov).
  • Minor fix in SHOW ACCESS query. #16866 (tavplubix).
  • Fix the behaviour with enabled optimize_trivial_count_query setting with partition predicate. #16767 (Azat Khuzhin).
  • Return number of affected rows for INSERT queries via MySQL wire protocol. Previously ClickHouse used to always return 0, it’s fixed. Fixes #16605. #16715 (Winter Zhang).
  • Fix inconsistent behavior caused by select_sequential_consistency for optimized trivial count query and system tables. #16309 (Hao Chen).
  • Throw error when REPLACE column transformer operates on non existing column. #16183 (hexiaoting).
  • Throw exception in case of not equi-join ON expression in RIGH|FULL JOIN. #15162 (Artem Zuikov).

Build/Testing/Packaging Improvement

  • Add simple integrity check for ClickHouse binary. It allows to detect corruption due to faulty hardware (bit rot on storage media or bit flips in RAM). #18811 (alexey-milovidov).
  • Change OpenSSL to BoringSSL. It allows to avoid issues with sanitizers. This fixes #12490. This fixes #17502. This fixes #12952. #18129 (alexey-milovidov).
  • Simplify Sys/V init script. It was not working on Ubuntu 12.04 or older. #17428 (alexey-milovidov).
  • Multiple improvements in ./clickhouse install script. #17421 (alexey-milovidov).
  • Now ClickHouse can pretend to be a fake ZooKeeper. Currently, storage implementation is just stored in-memory hash-table, and server partially support ZooKeeper protocol. #16877 (alesapin).
  • Fix dead list watches removal for TestKeeperStorage (a mock for ZooKeeper). #18065 (alesapin).
  • Add SYSTEM SUSPEND command for fault injection. It can be used to faciliate failover tests. This closes #15979. #18850 (alexey-milovidov).
  • Generate build id when ClickHouse is linked with lld. It’s appeared that lld does not generate it by default on my machine. Build id is used for crash reports and introspection. #18808 (alexey-milovidov).
  • Fix shellcheck errors in style check. #18566 (Ilya Yatsishin).
  • Update timezones info to 2020e. #18531 (alesapin).
  • Fix codespell warnings. Split style checks into separate parts. Update style checks docker image. #18463 (Ilya Yatsishin).
  • Automated check for leftovers of conflict markers in docs. #18332 (alexey-milovidov).
  • Enable Thread Fuzzer for stateless tests flaky check. #18299 (alesapin).
  • Do not use non thread-safe function strerror. #18204 (alexey-milovidov).
  • Update anchore/[[email protected]](https://clickhouse.com/cdn-cgi/l/email-protection) workflow action (was moved from master to main). #18192 (Stig Bakken).
  • Now clickhouse-test does DROP/CREATE databases with a timeout. #18098 (alesapin).
  • Enable experimental support for Pytest framework for stateless tests. #17902 (Ivan).
  • Now we use the fresh docker daemon version in integration tests. #17671 (alesapin).
  • Send info about official build, memory, cpu and free disk space to Sentry if it is enabled. Sentry is opt-in feature to help ClickHouse developers. This closes #17279. #17543 (alexey-milovidov).
  • There was an uninitialized variable in the code of clickhouse-copier. #17363 (Nikita Mikhaylov).
  • Fix one MSan report/stderr.log) from #17309. #17344 (Nikita Mikhaylov).
  • Fix for the issue with IPv6 in Arrow Flight library. See the comments for details. #16664 (Zhanna).
  • Add a library that replaces some libc functions to traps that will terminate the process. #16366 (alexey-milovidov).
  • Provide diagnostics in server logs in case of stack overflow, send error message to clickhouse-client. This closes #14840. #16346 (alexey-milovidov).
  • Now we can run almost all stateless functional tests in parallel. #15236 (alesapin).
  • Fix corruption in librdkafka snappy decompression (was a problem only for gcc10 builds, but official builds uses clang already, so at least recent official releases are not affected). #18053 (Azat Khuzhin).
  • If server was terminated by OOM killer, print message in log. #13516 (alexey-milovidov).
  • PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes #18525. #18526 (alexey-milovidov).
  • Minor improvement for path concatenation of zookeeper paths inside DDLWorker. #17767 (Bharat Nallan).
  • Allow to reload symbols from debug file. This PR also fixes a build-id issue. #17637 (Amos Bird).

Changelog for 2020