--- sidebar_position: 1 sidebar_label: 2022 --- # 2022 Changelog ### ClickHouse release v21.10.1.8013-prestable FIXME as compared to v21.9.1.7770-prestable #### Backward Incompatible Change * 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](https://github.com/ClickHouse/ClickHouse/issues/25447). This fixes [#26914](https://github.com/ClickHouse/ClickHouse/issues/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](https://github.com/ClickHouse/ClickHouse/pull/26639) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * 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](https://github.com/ClickHouse/ClickHouse/issues/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](https://github.com/ClickHouse/ClickHouse/pull/27680) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Now MergeTreeSettings `replicated_max_parallel_sends`, `replicated_max_parallel_sends_for_table`, `replicated_max_parallel_fetches`, `replicated_max_parallel_fetches_for_table` do nothing. They never worked well and were replaced with `max_replicated_fetches_network_bandwidth`, `max_replicated_sends_network_bandwidth` and `background_fetches_pool_size`. [#28404](https://github.com/ClickHouse/ClickHouse/pull/28404) ([alesapin](https://github.com/alesapin)). #### New Feature * Generate a unique server uuid when server starts. [#20089](https://github.com/ClickHouse/ClickHouse/pull/20089) ([Bharat Nallan](https://github.com/bharatnc)). * Added new commands BACKUP and RESTORE. [#21945](https://github.com/ClickHouse/ClickHouse/pull/21945) ([Vitaly Baranov](https://github.com/vitlibar)). * Partitioned write into s3 table function. [#23051](https://github.com/ClickHouse/ClickHouse/pull/23051) ([Vladimir Chebotarev](https://github.com/excitoon)). * Implementation of short circuit function evaluation, closes [#12587](https://github.com/ClickHouse/ClickHouse/issues/12587). Add settings `short_circuit_function_evaluation` to configure short circuit function evaluation. [#23367](https://github.com/ClickHouse/ClickHouse/pull/23367) ([Kruglov Pavel](https://github.com/Avogar)). * Add feature for creating user-defined functions. [#23978](https://github.com/ClickHouse/ClickHouse/pull/23978) ([Realist007](https://github.com/Realist007)). * Add support for INTERSECT, EXCEPT, ANY, ALL operators. [#24757](https://github.com/ClickHouse/ClickHouse/pull/24757) ([Kirill Ershov](https://github.com/zdikov)). * IDisk interface to store data on web server of static files. Closes [#23982](https://github.com/ClickHouse/ClickHouse/issues/23982). [#25251](https://github.com/ClickHouse/ClickHouse/pull/25251) ([Kseniia Sumarokova](https://github.com/kssenii)). * Introduce lz4 compression for import / export. [#25310](https://github.com/ClickHouse/ClickHouse/pull/25310) ([Bharat Nallan](https://github.com/bharatnc)). * Support the case when the data is enclosed in array in JSONAsString input format. Closes [#25517](https://github.com/ClickHouse/ClickHouse/issues/25517). [#25633](https://github.com/ClickHouse/ClickHouse/pull/25633) ([Kruglov Pavel](https://github.com/Avogar)). * Add new column `last_queue_update_exception` to `system.replicas` table. [#26843](https://github.com/ClickHouse/ClickHouse/pull/26843) ([nvartolomei](https://github.com/nvartolomei)). * ALTER TABLE ... MATERIALIZE COLUMN. [#27038](https://github.com/ClickHouse/ClickHouse/pull/27038) ([Vladimir Chebotarev](https://github.com/excitoon)). * - Add replicated storage of user, roles, row policies, quotas and settings profiles through ZooKeeper (experimental). [#27426](https://github.com/ClickHouse/ClickHouse/pull/27426) ([Kevin Michel](https://github.com/kmichel-aiven)). * Allow positional arguments under setting `enable_positional_arguments`. Closes [#2592](https://github.com/ClickHouse/ClickHouse/issues/2592). [#27530](https://github.com/ClickHouse/ClickHouse/pull/27530) ([Kseniia Sumarokova](https://github.com/kssenii)). * Added ComplexKeyRangeHashed dictionary. Closes [#22029](https://github.com/ClickHouse/ClickHouse/issues/22029). [#27629](https://github.com/ClickHouse/ClickHouse/pull/27629) ([Maksim Kita](https://github.com/kitaisreal)). * add conversion functions between snowflake id and dateTime(dateTime64) Close [#27058](https://github.com/ClickHouse/ClickHouse/issues/27058). [#27704](https://github.com/ClickHouse/ClickHouse/pull/27704) ([jasine](https://github.com/jasine)). * Add feature for creating user-defined functions as lambda expressions. Syntax `CREATE FUNCTION {function_name} as ({parameters}) -> {function core}`. Example `CREATE FUNCTION plus_one as (a) -> a + 1`. Authors @Realist007. [#27796](https://github.com/ClickHouse/ClickHouse/pull/27796) ([Maksim Kita](https://github.com/kitaisreal)). * Add `getServerPort` function to allow getting server port. When the port is not used by the server, throw an exception. [#27900](https://github.com/ClickHouse/ClickHouse/pull/27900) ([Amos Bird](https://github.com/amosbird)). * Accept user settings related to file formats in `SETTINGS` clause in `CREATE` query. This closes [#27580](https://github.com/ClickHouse/ClickHouse/issues/27580). [#28037](https://github.com/ClickHouse/ClickHouse/pull/28037) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Add a system table of table_views, convenient to query the dependency relationship between tables and views. [#28082](https://github.com/ClickHouse/ClickHouse/pull/28082) ([zhongyuankai](https://github.com/zhongyuankai)). * Added `executable` storage engine and table function. Authors @ruct. [#28102](https://github.com/ClickHouse/ClickHouse/pull/28102) ([Maksim Kita](https://github.com/kitaisreal)). * Added `ExecutablePool` storage. [#28518](https://github.com/ClickHouse/ClickHouse/pull/28518) ([Maksim Kita](https://github.com/kitaisreal)). #### Performance Improvement * Introducing two checks in `sequenceMatch` and `sequenceCount` that allow for early exit when some deterministic part of the sequence pattern is missing from the events list. This change unlocks many queries that would previously fail due to reaching operations cap, and generally speeds up the pipeline. [#27729](https://github.com/ClickHouse/ClickHouse/pull/27729) ([Jakub Kuklis](https://github.com/jkuklis)). * Make `hasAll` filter condition leverage bloom filter data-skipping indexes. [#27984](https://github.com/ClickHouse/ClickHouse/pull/27984) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)). * Speed up sumIf and countIf aggregation functions. [#28272](https://github.com/ClickHouse/ClickHouse/pull/28272) ([Raúl Marín](https://github.com/Algunenano)). * Enhance primary key analysis with always monotonic information of binary functions, notably non-zero constant division. [#28302](https://github.com/ClickHouse/ClickHouse/pull/28302) ([Amos Bird](https://github.com/amosbird)). #### Improvement * Create virtual projection for `min_max` indices. Now, when `allow_experimental_projection_optimization ` is enabled, queries will use minmax index instead of reading a part when possible. [#26286](https://github.com/ClickHouse/ClickHouse/pull/26286) ([Amos Bird](https://github.com/amosbird)). * improve Materialize TTL by recalculating ttl.txt only without actual ttl action. [#27019](https://github.com/ClickHouse/ClickHouse/pull/27019) ([lthaooo](https://github.com/lthaooo)). * Improved the existence condition judgment and empty string node judgment when clickhouse-keeper creates znode. [#27125](https://github.com/ClickHouse/ClickHouse/pull/27125) ([小路](https://github.com/nicelulu)). * Don't silently ignore errors and don't count delays in `ReadBufferFromS3`. [#27484](https://github.com/ClickHouse/ClickHouse/pull/27484) ([Vladimir Chebotarev](https://github.com/excitoon)). * Add `log_queries_probability` setting that allows user to write to query_log only a sample of queries. Closes [#16609](https://github.com/ClickHouse/ClickHouse/issues/16609). [#27527](https://github.com/ClickHouse/ClickHouse/pull/27527) ([Nikolay Degterinsky](https://github.com/evillique)). * Disable arrayJoin on partition expressions. [#27648](https://github.com/ClickHouse/ClickHouse/pull/27648) ([Raúl Marín](https://github.com/Algunenano)). * - Add `FROM INFILE` command. [#27655](https://github.com/ClickHouse/ClickHouse/pull/27655) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Enables query parameters to be passed in the body of http requests. [#27706](https://github.com/ClickHouse/ClickHouse/pull/27706) ([Hermano Lustosa](https://github.com/hllustosa)). * Remove duplicate index analysis and avoid possible invalid limit checks during projection analysis. [#27742](https://github.com/ClickHouse/ClickHouse/pull/27742) ([Amos Bird](https://github.com/amosbird)). * Add aggregate function `quantileBFloat16Weighted` similarly to other quantile...Weighted functions. This closes [#27745](https://github.com/ClickHouse/ClickHouse/issues/27745). [#27758](https://github.com/ClickHouse/ClickHouse/pull/27758) ([Ivan Novitskiy](https://github.com/RedClusive)). * Now `ALTER MODIFY COLUM` DataType to `Nullable(DataType)` doesn't require mutation. [#27787](https://github.com/ClickHouse/ClickHouse/pull/27787) ([victorgao](https://github.com/kafka1991)). * Allow symlinks for library dictionaty path. [#27815](https://github.com/ClickHouse/ClickHouse/pull/27815) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add function `SHA512`. [#27830](https://github.com/ClickHouse/ClickHouse/pull/27830) ([zhanglistar](https://github.com/zhanglistar)). * Use Multipart copy upload for large S3 objects. [#27858](https://github.com/ClickHouse/ClickHouse/pull/27858) ([ianton-ru](https://github.com/ianton-ru)). * Improve remote query cancelation (in case of remote server abnormaly terminated). [#27881](https://github.com/ClickHouse/ClickHouse/pull/27881) ([Azat Khuzhin](https://github.com/azat)). * Enable tcp_keep_alive_timeout by default. [#27882](https://github.com/ClickHouse/ClickHouse/pull/27882) ([Azat Khuzhin](https://github.com/azat)). * Fix incorrect assertion during writing to StorageKafka. [#27885](https://github.com/ClickHouse/ClickHouse/pull/27885) ([Azat Khuzhin](https://github.com/azat)). * Support lambda argument for APPLY column transformer which allows applying functions with more than one argument. This is for [#27877](https://github.com/ClickHouse/ClickHouse/issues/27877). [#27901](https://github.com/ClickHouse/ClickHouse/pull/27901) ([Amos Bird](https://github.com/amosbird)). * Add interactive documentation in `clickhouse-client` about how to reset the password. This is useful in scenario when user has installed ClickHouse, set up the password and instantly forget it. See [#27750](https://github.com/ClickHouse/ClickHouse/issues/27750). [#27903](https://github.com/ClickHouse/ClickHouse/pull/27903) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Allow to create dictionaries with empty attributes list. [#27905](https://github.com/ClickHouse/ClickHouse/pull/27905) ([Maksim Kita](https://github.com/kitaisreal)). * Added `replication_wait_for_inactive_replica_timeout` setting. It allows to specify how long to wait for inactive replicas to execute `ALTER`/`OPTIMZE`/`TRUNCATE` query (default is 120 seconds). If `replication_alter_partitions_sync` is 2 and some replicas are not active for more than `replication_wait_for_inactive_replica_timeout` seconds, then `UNFINISHED` will be thrown. [#27931](https://github.com/ClickHouse/ClickHouse/pull/27931) ([Alexander Tokmakov](https://github.com/tavplubix)). * Add a setting `empty_result_for_aggregation_by_constant_keys_on_empty_set` to control the behavior of grouping by constant keys on empty set. This is to bring back the old baviour of [#6842](https://github.com/ClickHouse/ClickHouse/issues/6842). [#27932](https://github.com/ClickHouse/ClickHouse/pull/27932) ([Amos Bird](https://github.com/amosbird)). * Lower restrictions for Enum data type to allow attaching compatible data. Closes [#26672](https://github.com/ClickHouse/ClickHouse/issues/26672). [#28028](https://github.com/ClickHouse/ClickHouse/pull/28028) ([Dmitry Novik](https://github.com/novikd)). * Support ON CONFLICT clause when inserting into PostgreSQL table engine or table function. Closes [#27727](https://github.com/ClickHouse/ClickHouse/issues/27727). [#28081](https://github.com/ClickHouse/ClickHouse/pull/28081) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support implicit conversions between index in operator `[]` and key of type `Map` (e.g. different `Int` types, `String` and `FixedString`). [#28096](https://github.com/ClickHouse/ClickHouse/pull/28096) ([Anton Popov](https://github.com/CurtizJ)). * Enable optimize_distributed_group_by_sharding_key by default. [#28105](https://github.com/ClickHouse/ClickHouse/pull/28105) ([Azat Khuzhin](https://github.com/azat)). * Fix `zookeeper_log.address` (before the first patch in this PR the address was always `::`) and reduce number of calls `getpeername(2)` for this column (since each time entry for `zookeeper_log` is added `getpeername()` is called, cache this address in the zookeeper client to avoid this). [#28212](https://github.com/ClickHouse/ClickHouse/pull/28212) ([Azat Khuzhin](https://github.com/azat)). * Fix removing of parts in a Temporary state which can lead to an unexpected exception (`Part %name% doesn't exist`). Fixes [#23661](https://github.com/ClickHouse/ClickHouse/issues/23661). [#28221](https://github.com/ClickHouse/ClickHouse/pull/28221) ([Azat Khuzhin](https://github.com/azat)). * Added libhdfs3_conf in server config instead of export env LIBHDFS3_CONF in clickhouse-server.service. [#28268](https://github.com/ClickHouse/ClickHouse/pull/28268) ([Zhichang Yu](https://github.com/yuzhichang)). * Use real tmp file instead of predefined "rows_sources" for vertical merges. This avoids generating garbage directories in tmp disks. [#28299](https://github.com/ClickHouse/ClickHouse/pull/28299) ([Amos Bird](https://github.com/amosbird)). * Speed up data parts loading by delaying table startup process. [#28313](https://github.com/ClickHouse/ClickHouse/pull/28313) ([Amos Bird](https://github.com/amosbird)). * Allow ssl connection for RabbitMQ engine. [#28365](https://github.com/ClickHouse/ClickHouse/pull/28365) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix removing of parts in a Temporary state (follow up for [#28221](https://github.com/ClickHouse/ClickHouse/issues/28221)). [#28366](https://github.com/ClickHouse/ClickHouse/pull/28366) ([Azat Khuzhin](https://github.com/azat)). * Do not allow creating StorageMaterializedPostgreSQL with bad arguments. Closes [#28423](https://github.com/ClickHouse/ClickHouse/issues/28423). [#28430](https://github.com/ClickHouse/ClickHouse/pull/28430) ([Kseniia Sumarokova](https://github.com/kssenii)). * Introduce `connection_wait_timeout` (default to 5 seconds, 0 - do not wait) setting for MySQL engine. [#28474](https://github.com/ClickHouse/ClickHouse/pull/28474) ([Azat Khuzhin](https://github.com/azat)). * Fix strange sessions expiration logic in Keeper. Probably it should help in CI: https://clickhouse-test-reports.s3.yandex.net/0/6bd9b82141c98dcd7796fd9d08326831095ba519/stress_test_(debug).html#fail1. [#28519](https://github.com/ClickHouse/ClickHouse/pull/28519) ([alesapin](https://github.com/alesapin)). * To be added. Closes [#28529](https://github.com/ClickHouse/ClickHouse/issues/28529). [#28614](https://github.com/ClickHouse/ClickHouse/pull/28614) ([Kseniia Sumarokova](https://github.com/kssenii)). #### Bug Fix * Bugfix for windowFunnel's "strict" mode. This fixes [#27469](https://github.com/ClickHouse/ClickHouse/issues/27469). [#27563](https://github.com/ClickHouse/ClickHouse/pull/27563) ([achimbab](https://github.com/achimbab)). * - Fix bug with aliased column in `Distributed` table. [#27652](https://github.com/ClickHouse/ClickHouse/pull/27652) ([Vladimir C](https://github.com/vdimir)). * Fixed another case of `Unexpected merged part ... intersecting drop range ...` error. [#27656](https://github.com/ClickHouse/ClickHouse/pull/27656) ([Alexander Tokmakov](https://github.com/tavplubix)). * 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](https://github.com/ClickHouse/ClickHouse/issues/26330). [#27682](https://github.com/ClickHouse/ClickHouse/pull/27682) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix column filtering with union distinct in subquery. Closes [#27578](https://github.com/ClickHouse/ClickHouse/issues/27578). [#27689](https://github.com/ClickHouse/ClickHouse/pull/27689) ([Kseniia Sumarokova](https://github.com/kssenii)). * After https://github.com/ClickHouse/ClickHouse/pull/26384. To execute `GRANT WITH REPLACE OPTION` now the current user should have `GRANT OPTION` for access rights it's going to grant AND for access rights it's going to revoke. [#27701](https://github.com/ClickHouse/ClickHouse/pull/27701) ([Vitaly Baranov](https://github.com/vitlibar)). * After https://github.com/ClickHouse/ClickHouse/pull/25687. Add backquotes for the default database shown in CREATE USER. [#27702](https://github.com/ClickHouse/ClickHouse/pull/27702) ([Vitaly Baranov](https://github.com/vitlibar)). * Remove duplicated source files in CMakeLists.txt in arrow-cmake. [#27736](https://github.com/ClickHouse/ClickHouse/pull/27736) ([李扬](https://github.com/taiyang-li)). * Fix possible crash when asynchronous connection draining is enabled and hedged connection is disabled. [#27774](https://github.com/ClickHouse/ClickHouse/pull/27774) ([Amos Bird](https://github.com/amosbird)). * Prevent crashes for some formats when NULL (tombstone) message was coming from Kafka. Closes [#19255](https://github.com/ClickHouse/ClickHouse/issues/19255). [#27794](https://github.com/ClickHouse/ClickHouse/pull/27794) ([filimonov](https://github.com/filimonov)). * Fix a rare bug in `DROP PART` which can lead to the error `Unexpected merged part intersects drop range`. [#27807](https://github.com/ClickHouse/ClickHouse/pull/27807) ([alesapin](https://github.com/alesapin)). * Fix a couple of bugs that may cause replicas to diverge. [#27808](https://github.com/ClickHouse/ClickHouse/pull/27808) ([Alexander Tokmakov](https://github.com/tavplubix)). * After https://github.com/ClickHouse/ClickHouse/pull/26864. Fix shutdown of `NamedSessionStorage`: session contexts stored in `NamedSessionStorage` are now destroyed before destroying the global context. [#27875](https://github.com/ClickHouse/ClickHouse/pull/27875) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix PostgreSQL-style cast (`::` operator) with negative numbers. [#27876](https://github.com/ClickHouse/ClickHouse/pull/27876) ([Anton Popov](https://github.com/CurtizJ)). * Fix selecting with extremes from a column of the type `LowCardinality(UUID)`. [#27918](https://github.com/ClickHouse/ClickHouse/pull/27918) ([Vitaly Baranov](https://github.com/vitlibar)). * Check cluster name before creating Distributed table, do not allow to create a table with incorrect cluster name. Fixes [#27832](https://github.com/ClickHouse/ClickHouse/issues/27832). [#27927](https://github.com/ClickHouse/ClickHouse/pull/27927) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix checking access grants when executing GRANT WITH REPLACE statement with ON CLUSTER clause. This PR improves fix https://github.com/ClickHouse/ClickHouse/pull/27701. [#27983](https://github.com/ClickHouse/ClickHouse/pull/27983) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix cases, when read buffer fails with 'attempt to read after end of file'. Closes [#26149](https://github.com/ClickHouse/ClickHouse/issues/26149). [#28150](https://github.com/ClickHouse/ClickHouse/pull/28150) ([Filatenkov Artur](https://github.com/FArthur-cmd)). #### Build/Testing/Packaging Improvement * Enable Thread Fuzzer in Stress Test. Thread Fuzzer is ClickHouse feature that allows to test more permutations of thread scheduling and discover more potential issues. This closes [#9813](https://github.com/ClickHouse/ClickHouse/issues/9813). This closes [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814). This closes [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515). This closes [#9516](https://github.com/ClickHouse/ClickHouse/issues/9516). [#27538](https://github.com/ClickHouse/ClickHouse/pull/27538) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add support for build with `clang-13`. This closes [#27705](https://github.com/ClickHouse/ClickHouse/issues/27705). [#27714](https://github.com/ClickHouse/ClickHouse/pull/27714) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Improve support for build with `clang-13`. [#27777](https://github.com/ClickHouse/ClickHouse/pull/27777) ([Sergei Semin](https://github.com/syominsergey)). * Temporarily switched ubuntu apt repository to mirror ru.archive.ubuntu.com as default one(archive.ubuntu.com) is not responding from our CI. [#28016](https://github.com/ClickHouse/ClickHouse/pull/28016) ([Ilya Yatsishin](https://github.com/qoega)). * Print out git status information at CMake configure stage. [#28047](https://github.com/ClickHouse/ClickHouse/pull/28047) ([Braulio Valdivielso Martínez](https://github.com/BraulioVM)). * Add new log level `` for testing environments. [#28559](https://github.com/ClickHouse/ClickHouse/pull/28559) ([alesapin](https://github.com/alesapin)). #### Bug Fix (user-visible misbehaviour in official stable or prestable release) * Fix handling null value with type of Nullable(String) in function JSONExtract. This fixes [#27929](https://github.com/ClickHouse/ClickHouse/issues/27929) and [#27930](https://github.com/ClickHouse/ClickHouse/issues/27930) . This was introduced in https://github.com/ClickHouse/ClickHouse/pull/25452 . [#27939](https://github.com/ClickHouse/ClickHouse/pull/27939) ([Amos Bird](https://github.com/amosbird)). * Fix extremely rare segfaults on shutdown due to incorrect order of context/config reloader shutdown. [#28088](https://github.com/ClickHouse/ClickHouse/pull/28088) ([nvartolomei](https://github.com/nvartolomei)). * Fixed possible excessive number of conditions moved from `WHERE` to `PREWHERE` (optimization controlled by settings `optimize_move_to_prewhere`). [#28139](https://github.com/ClickHouse/ClickHouse/pull/28139) ([lthaooo](https://github.com/lthaooo)). * Fix bug in clickhouse-keeper which can lead to endless logs when `rotate_logs_interval` decreased. [#28152](https://github.com/ClickHouse/ClickHouse/pull/28152) ([alesapin](https://github.com/alesapin)). * Multiple small fixes for projections. See detailed description in pr. [#28178](https://github.com/ClickHouse/ClickHouse/pull/28178) ([Amos Bird](https://github.com/amosbird)). * Fix incorrect behavior in `clickhouse-keeper` when list watches (`getChildren`) triggered with `set` requests for children. [#28190](https://github.com/ClickHouse/ClickHouse/pull/28190) ([alesapin](https://github.com/alesapin)). * Fix a rare bug in `clickhouse-keeper` when the client can receive a watch response before request-response. [#28197](https://github.com/ClickHouse/ClickHouse/pull/28197) ([alesapin](https://github.com/alesapin)). * Fix possible read of uninitialized memory for queries with `Nullable(LowCardinality)` type and extremes. Fixes [#28165](https://github.com/ClickHouse/ClickHouse/issues/28165). [#28205](https://github.com/ClickHouse/ClickHouse/pull/28205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix reading of custom TLD w/o new line at EOF. [#28213](https://github.com/ClickHouse/ClickHouse/pull/28213) ([Azat Khuzhin](https://github.com/azat)). * Fix inconsistent result in queries with `ORDER BY` and `Merge` tables with enabled setting `optimize_read_in_order`. [#28266](https://github.com/ClickHouse/ClickHouse/pull/28266) ([Anton Popov](https://github.com/CurtizJ)). * Fix intersecting parts due to new part had been replaced with an empty part. [#28310](https://github.com/ClickHouse/ClickHouse/pull/28310) ([Azat Khuzhin](https://github.com/azat)). * Fix NOT-IN index optimization when not all key columns are used. This fixes [#28120](https://github.com/ClickHouse/ClickHouse/issues/28120). [#28315](https://github.com/ClickHouse/ClickHouse/pull/28315) ([Amos Bird](https://github.com/amosbird)). * Fix non joined rows from nullable column. Close [#27691](https://github.com/ClickHouse/ClickHouse/issues/27691). [#28349](https://github.com/ClickHouse/ClickHouse/pull/28349) ([Vladimir C](https://github.com/vdimir)). * Fix rare case when changes of `clickhouse-keeper` settings may lead to lost logs and server hung. [#28360](https://github.com/ClickHouse/ClickHouse/pull/28360) ([alesapin](https://github.com/alesapin)). * Fix lack of quotes for table names in MaterializedPostgreSQL engine. Closes [#28316](https://github.com/ClickHouse/ClickHouse/issues/28316). [#28433](https://github.com/ClickHouse/ClickHouse/pull/28433) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fixed possible ZooKeeper watches leak on background processing of distributed DDL queue. Closes [#26036](https://github.com/ClickHouse/ClickHouse/issues/26036). [#28446](https://github.com/ClickHouse/ClickHouse/pull/28446) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix bug which can lead to error `Existing table metadata in ZooKeeper differs in sorting key expression.` after alter of `ReplicatedVersionedCollapsingMergeTree`. Fixes [#28515](https://github.com/ClickHouse/ClickHouse/issues/28515). [#28528](https://github.com/ClickHouse/ClickHouse/pull/28528) ([alesapin](https://github.com/alesapin)). * Fix `There is no subcolumn` error, while select from tables, which have `Nested` columns and scalar columns with dot in name and the same prefix as `Nested` (e.g. `n.id UInt32, n.arr1 Array(UInt64), n.arr2 Array(UInt64)`). [#28531](https://github.com/ClickHouse/ClickHouse/pull/28531) ([Anton Popov](https://github.com/CurtizJ)). * Fix UUID overlap in DROP TABLE for internal DDL from MaterializeMySQL. [#28533](https://github.com/ClickHouse/ClickHouse/pull/28533) ([Azat Khuzhin](https://github.com/azat)). * Fix endless loop for truncated bzip2 archive. [#28543](https://github.com/ClickHouse/ClickHouse/pull/28543) ([Azat Khuzhin](https://github.com/azat)). #### NO CL ENTRY * NO CL ENTRY: 'Add executable table function'. [#23192](https://github.com/ClickHouse/ClickHouse/pull/23192) ([ruct](https://github.com/ruct)). * NO CL ENTRY: 'DOCSUP-12413: macros support in functions cluster and clusterAllReplicas'. [#27759](https://github.com/ClickHouse/ClickHouse/pull/27759) ([olgarev](https://github.com/olgarev)). * NO CL ENTRY: 'Revert "less sys calls #2: make vdso work again"'. [#27829](https://github.com/ClickHouse/ClickHouse/pull/27829) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Revert "Do not miss exceptions from the ThreadPool"'. [#27844](https://github.com/ClickHouse/ClickHouse/pull/27844) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Revert "Improve 01730_distributed_group_by_no_merge_order_by_long"'. [#28128](https://github.com/ClickHouse/ClickHouse/pull/28128) ([alesapin](https://github.com/alesapin)). * NO CL ENTRY: 'Revert "Revert "less sys calls #2: make vdso work again""'. [#28132](https://github.com/ClickHouse/ClickHouse/pull/28132) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Update src/Functions/GatherUtils/Sources.h'. [#28186](https://github.com/ClickHouse/ClickHouse/pull/28186) ([sdk2](https://github.com/sdk2)). * NO CL ENTRY: 'Revert "Add test for [#13398](https://github.com/ClickHouse/ClickHouse/issues/13398)"'. [#28274](https://github.com/ClickHouse/ClickHouse/pull/28274) ([Alexander Tokmakov](https://github.com/tavplubix)). * NO CL ENTRY: 'fix minor typo'. [#28629](https://github.com/ClickHouse/ClickHouse/pull/28629) ([flynn](https://github.com/ucasfl)). #### NOT FOR CHANGELOG / INSIGNIFICANT * Experiment with asynchronous readers [#26791](https://github.com/ClickHouse/ClickHouse/pull/26791) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Introduce sessions [#26864](https://github.com/ClickHouse/ClickHouse/pull/26864) ([Vitaly Baranov](https://github.com/vitlibar)). * FILTER clause for aggregate functions [#27036](https://github.com/ClickHouse/ClickHouse/pull/27036) ([Nikita Taranov](https://github.com/nickitat)). * Add test for parsing maps with integer keys [#27146](https://github.com/ClickHouse/ClickHouse/pull/27146) ([Anton Popov](https://github.com/CurtizJ)). * S3 disk unstable reads test [#27176](https://github.com/ClickHouse/ClickHouse/pull/27176) ([Vladimir Chebotarev](https://github.com/excitoon)). * Refactor NotJoined [#27299](https://github.com/ClickHouse/ClickHouse/pull/27299) ([Vladimir C](https://github.com/vdimir)). * Accept error code by error name in client test hints [#27430](https://github.com/ClickHouse/ClickHouse/pull/27430) ([Azat Khuzhin](https://github.com/azat)). * Break some tests [#27529](https://github.com/ClickHouse/ClickHouse/pull/27529) ([Alexander Tokmakov](https://github.com/tavplubix)). * Remove the remains of ANTLR in the tests [#27637](https://github.com/ClickHouse/ClickHouse/pull/27637) ([Raúl Marín](https://github.com/Algunenano)). * Disable memory tracking for roaring bitmaps on Mac OS [#27681](https://github.com/ClickHouse/ClickHouse/pull/27681) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Use only SSE2 in "unbundled" build [#27683](https://github.com/ClickHouse/ClickHouse/pull/27683) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove trash [#27685](https://github.com/ClickHouse/ClickHouse/pull/27685) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix stress test in `~CompressedWriteBuffer` [#27686](https://github.com/ClickHouse/ClickHouse/pull/27686) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Mark tests for `DatabaseReplicated` as green [#27688](https://github.com/ClickHouse/ClickHouse/pull/27688) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Removed DenseHashMap, DenseHashSet [#27690](https://github.com/ClickHouse/ClickHouse/pull/27690) ([Maksim Kita](https://github.com/kitaisreal)). * Map data type parsing tests [#27692](https://github.com/ClickHouse/ClickHouse/pull/27692) ([Maksim Kita](https://github.com/kitaisreal)). * Refactor arrayJoin check on partition expressions [#27733](https://github.com/ClickHouse/ClickHouse/pull/27733) ([Raúl Marín](https://github.com/Algunenano)). * Fix test 01014_lazy_database_concurrent_recreate_reattach_and_show_tables [#27734](https://github.com/ClickHouse/ClickHouse/pull/27734) ([Alexander Tokmakov](https://github.com/tavplubix)). * Better code around decompression [2] [#27743](https://github.com/ClickHouse/ClickHouse/pull/27743) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Disable jemalloc under OSX [#27751](https://github.com/ClickHouse/ClickHouse/pull/27751) ([Raúl Marín](https://github.com/Algunenano)). * try to collect some core dumps in perf tests [#27752](https://github.com/ClickHouse/ClickHouse/pull/27752) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Fix jemalloc under osx (zone_register() had been optimized out again) [#27753](https://github.com/ClickHouse/ClickHouse/pull/27753) ([Azat Khuzhin](https://github.com/azat)). * Merging [#20089](https://github.com/ClickHouse/ClickHouse/issues/20089) [#27755](https://github.com/ClickHouse/ClickHouse/pull/27755) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix intersect/except with limit [#27757](https://github.com/ClickHouse/ClickHouse/pull/27757) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add HTTP string parsing test [#27762](https://github.com/ClickHouse/ClickHouse/pull/27762) ([Nikolay Degterinsky](https://github.com/evillique)). * Fix some tests [#27785](https://github.com/ClickHouse/ClickHouse/pull/27785) ([Alexander Tokmakov](https://github.com/tavplubix)). * Set function divide as suitable for short-circuit in case of Nullable(Decimal) [#27788](https://github.com/ClickHouse/ClickHouse/pull/27788) ([Kruglov Pavel](https://github.com/Avogar)). * Remove unnecessary files [#27789](https://github.com/ClickHouse/ClickHouse/pull/27789) ([Kruglov Pavel](https://github.com/Avogar)). * Revert "Mark tests for `DatabaseReplicated` as green" [#27791](https://github.com/ClickHouse/ClickHouse/pull/27791) ([Alexander Tokmakov](https://github.com/tavplubix)). * Remove hardening for watches in DDLWorker [#27792](https://github.com/ClickHouse/ClickHouse/pull/27792) ([Alexander Tokmakov](https://github.com/tavplubix)). * Stateless test: Cleanup leftovers [#27793](https://github.com/ClickHouse/ClickHouse/pull/27793) ([Raúl Marín](https://github.com/Algunenano)). * Dictionaries key types refactoring [#27795](https://github.com/ClickHouse/ClickHouse/pull/27795) ([Maksim Kita](https://github.com/kitaisreal)). * Update 01822_short_circuit.reference (after merging [#27680](https://github.com/ClickHouse/ClickHouse/issues/27680)) [#27802](https://github.com/ClickHouse/ClickHouse/pull/27802) ([Azat Khuzhin](https://github.com/azat)). * Proper shutdown global context [#27804](https://github.com/ClickHouse/ClickHouse/pull/27804) ([Amos Bird](https://github.com/amosbird)). * 01766_todatetime64_no_timezone_arg: Use a date without timezone changes [#27810](https://github.com/ClickHouse/ClickHouse/pull/27810) ([Raúl Marín](https://github.com/Algunenano)). * Use sessions more [#27817](https://github.com/ClickHouse/ClickHouse/pull/27817) ([Vitaly Baranov](https://github.com/vitlibar)). * Add test for clickhouse-keeper start after conversion [#27818](https://github.com/ClickHouse/ClickHouse/pull/27818) ([alesapin](https://github.com/alesapin)). * Fix setting name "allow_experimental_database_materialized_postgresql" in the error message [#27824](https://github.com/ClickHouse/ClickHouse/pull/27824) ([Denny Crane](https://github.com/den-crane)). * Fix bug in short-circuit found by fuzzer [#27826](https://github.com/ClickHouse/ClickHouse/pull/27826) ([Kruglov Pavel](https://github.com/Avogar)). * Add more checks for LC in native protocol. [#27827](https://github.com/ClickHouse/ClickHouse/pull/27827) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix test 00443_preferred_block_size_bytes.sh [#27846](https://github.com/ClickHouse/ClickHouse/pull/27846) ([Alexander Tokmakov](https://github.com/tavplubix)). * Mute some integration tests until failures are fixed [#27862](https://github.com/ClickHouse/ClickHouse/pull/27862) ([Ilya Yatsishin](https://github.com/qoega)). * Fix bad cast in insertPostgreSQLValue [#27869](https://github.com/ClickHouse/ClickHouse/pull/27869) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add fuzzers for codecs [#27872](https://github.com/ClickHouse/ClickHouse/pull/27872) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Remove tmp folders from tests [#27878](https://github.com/ClickHouse/ClickHouse/pull/27878) ([Kruglov Pavel](https://github.com/Avogar)). * blog article about perf tests [#27879](https://github.com/ClickHouse/ClickHouse/pull/27879) ([Alexander Kuzmenkov](https://github.com/akuzm)). * make the sql-standard window functions case insensitive [#27880](https://github.com/ClickHouse/ClickHouse/pull/27880) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Bump librdkafka (fixes use of an invalid/destroyed mutex) [#27883](https://github.com/ClickHouse/ClickHouse/pull/27883) ([Azat Khuzhin](https://github.com/azat)). * fix decimal formatting settings in perf test [#27884](https://github.com/ClickHouse/ClickHouse/pull/27884) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add separate constants for interfaces LOCAL and TCP_INTERSERVER. [#27886](https://github.com/ClickHouse/ClickHouse/pull/27886) ([Vitaly Baranov](https://github.com/vitlibar)). * Build fuzzers with clang-tidy [#27895](https://github.com/ClickHouse/ClickHouse/pull/27895) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Allow implicit cast bool to Field [#27921](https://github.com/ClickHouse/ClickHouse/pull/27921) ([Vitaly Baranov](https://github.com/vitlibar)). * Improve server logs checking in integration tests [#27934](https://github.com/ClickHouse/ClickHouse/pull/27934) ([Azat Khuzhin](https://github.com/azat)). * Get rid of mutable value in FunctionGetSetting. [#27982](https://github.com/ClickHouse/ClickHouse/pull/27982) ([Vitaly Baranov](https://github.com/vitlibar)). * Disable fuzzers build with clang-tidy [#27985](https://github.com/ClickHouse/ClickHouse/pull/27985) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Build fuzzers in CI [#27990](https://github.com/ClickHouse/ClickHouse/pull/27990) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix file progress for local [#27991](https://github.com/ClickHouse/ClickHouse/pull/27991) ([Kseniia Sumarokova](https://github.com/kssenii)). * Update libunwind [#27993](https://github.com/ClickHouse/ClickHouse/pull/27993) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix style typos [#28020](https://github.com/ClickHouse/ClickHouse/pull/28020) ([Ilya Yatsishin](https://github.com/qoega)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). * Bump replxx [#28039](https://github.com/ClickHouse/ClickHouse/pull/28039) ([Azat Khuzhin](https://github.com/azat)). * Add test for [#13398](https://github.com/ClickHouse/ClickHouse/issues/13398) [#28054](https://github.com/ClickHouse/ClickHouse/pull/28054) ([Kseniia Sumarokova](https://github.com/kssenii)). * Set version of tzlocal to 2.1 [#28063](https://github.com/ClickHouse/ClickHouse/pull/28063) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix zookeeper secure client test [#28066](https://github.com/ClickHouse/ClickHouse/pull/28066) ([alesapin](https://github.com/alesapin)). * Fix typo in docs [#28077](https://github.com/ClickHouse/ClickHouse/pull/28077) ([Kruglov Pavel](https://github.com/Avogar)). * Fixed a typo in comments to `SinkToStorage` [#28078](https://github.com/ClickHouse/ClickHouse/pull/28078) ([Vladimir Chebotarev](https://github.com/excitoon)). * Use jinja template tests in fuzzer [#28079](https://github.com/ClickHouse/ClickHouse/pull/28079) ([Vladimir C](https://github.com/vdimir)). * Clickhouse-keeper: renames and comments [#28080](https://github.com/ClickHouse/ClickHouse/pull/28080) ([alesapin](https://github.com/alesapin)). * Update nanodbc [#28084](https://github.com/ClickHouse/ClickHouse/pull/28084) ([Kseniia Sumarokova](https://github.com/kssenii)). * Improve 01730_distributed_group_by_no_merge_order_by_long [#28123](https://github.com/ClickHouse/ClickHouse/pull/28123) ([Azat Khuzhin](https://github.com/azat)). * Get rid of useless projection columns during merge [#28135](https://github.com/ClickHouse/ClickHouse/pull/28135) ([Amos Bird](https://github.com/amosbird)). * Fix style [#28140](https://github.com/ClickHouse/ClickHouse/pull/28140) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Fix clickhouse keeper jepsen tests [#28143](https://github.com/ClickHouse/ClickHouse/pull/28143) ([alesapin](https://github.com/alesapin)). * Updated ya.make files [#28157](https://github.com/ClickHouse/ClickHouse/pull/28157) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Better detection of the default interface in replicated fetches tests [#28184](https://github.com/ClickHouse/ClickHouse/pull/28184) ([alesapin](https://github.com/alesapin)). * Reserve protocol number for ALTER PRIMARY KEY. [#28193](https://github.com/ClickHouse/ClickHouse/pull/28193) ([Amos Bird](https://github.com/amosbird)). * Maybe fix livelock in ZooKeeper client [#28195](https://github.com/ClickHouse/ClickHouse/pull/28195) ([Alexander Tokmakov](https://github.com/tavplubix)). * Guard UDF container with a lock [#28211](https://github.com/ClickHouse/ClickHouse/pull/28211) ([Azat Khuzhin](https://github.com/azat)). * Fix error codes [#28234](https://github.com/ClickHouse/ClickHouse/pull/28234) ([Kseniia Sumarokova](https://github.com/kssenii)). * CHJIT custom memory manager [#28236](https://github.com/ClickHouse/ClickHouse/pull/28236) ([Maksim Kita](https://github.com/kitaisreal)). * Dictionaries small fixes [#28249](https://github.com/ClickHouse/ClickHouse/pull/28249) ([Maksim Kita](https://github.com/kitaisreal)). * Better nullable primary key implementation [#28269](https://github.com/ClickHouse/ClickHouse/pull/28269) ([Amos Bird](https://github.com/amosbird)). * ODBC connection holder fix dangling reference [#28298](https://github.com/ClickHouse/ClickHouse/pull/28298) ([Maksim Kita](https://github.com/kitaisreal)). * test/stress: fix patterns for filtering out Raft messages [#28303](https://github.com/ClickHouse/ClickHouse/pull/28303) ([Azat Khuzhin](https://github.com/azat)). * Rename system.views columns [#28319](https://github.com/ClickHouse/ClickHouse/pull/28319) ([Kseniia Sumarokova](https://github.com/kssenii)). * Try to fix arcadia build (generate ya.make) [#28326](https://github.com/ClickHouse/ClickHouse/pull/28326) ([DimasKovas](https://github.com/DimasKovas)). * Try to fix arcadia build [#28333](https://github.com/ClickHouse/ClickHouse/pull/28333) ([DimasKovas](https://github.com/DimasKovas)). * Fix test_storage_s3/test_put_get_with_globs (cleanup after test) [#28336](https://github.com/ClickHouse/ClickHouse/pull/28336) ([ianton-ru](https://github.com/ianton-ru)). * Fix sed argument in test/fuzzer/run-fuzzer.sh [#28350](https://github.com/ClickHouse/ClickHouse/pull/28350) ([Vladimir C](https://github.com/vdimir)). * Another try to fix BackgroundPoolTask decrement. [#28353](https://github.com/ClickHouse/ClickHouse/pull/28353) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Rework kafka topic creation. [#28354](https://github.com/ClickHouse/ClickHouse/pull/28354) ([Ilya Yatsishin](https://github.com/qoega)). * Add compat between SinkToStorage and BlockOutputStream [#28361](https://github.com/ClickHouse/ClickHouse/pull/28361) ([DimasKovas](https://github.com/DimasKovas)). * Try to fix arcadia build (generate ya.make) [#28382](https://github.com/ClickHouse/ClickHouse/pull/28382) ([DimasKovas](https://github.com/DimasKovas)). * Add a test for a friend [#28396](https://github.com/ClickHouse/ClickHouse/pull/28396) ([alesapin](https://github.com/alesapin)). * Update ya.make [#28403](https://github.com/ClickHouse/ClickHouse/pull/28403) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix disk with static files a little [#28411](https://github.com/ClickHouse/ClickHouse/pull/28411) ([Kseniia Sumarokova](https://github.com/kssenii)). * More accurate check that zk root exists. [#28412](https://github.com/ClickHouse/ClickHouse/pull/28412) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Merging [#27980](https://github.com/ClickHouse/ClickHouse/issues/27980) [#28413](https://github.com/ClickHouse/ClickHouse/pull/28413) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix 01457_create_as_table_function_structure [#28428](https://github.com/ClickHouse/ClickHouse/pull/28428) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove some rename tests [#28437](https://github.com/ClickHouse/ClickHouse/pull/28437) ([alesapin](https://github.com/alesapin)). * Fix race in UDF (follow up) [#28438](https://github.com/ClickHouse/ClickHouse/pull/28438) ([Azat Khuzhin](https://github.com/azat)). * Executable multiple pipes added test [#28503](https://github.com/ClickHouse/ClickHouse/pull/28503) ([Maksim Kita](https://github.com/kitaisreal)). * UserDefinedFunctionFactory added comments [#28516](https://github.com/ClickHouse/ClickHouse/pull/28516) ([Maksim Kita](https://github.com/kitaisreal)). * BorrowedObjectPool fix style [#28523](https://github.com/ClickHouse/ClickHouse/pull/28523) ([Maksim Kita](https://github.com/kitaisreal)). * Add test for keeper 2 node configuration [#28526](https://github.com/ClickHouse/ClickHouse/pull/28526) ([alesapin](https://github.com/alesapin)). * Function dictGet default implementation for nulls [#28530](https://github.com/ClickHouse/ClickHouse/pull/28530) ([Maksim Kita](https://github.com/kitaisreal)). * Fix race in zlib [#28534](https://github.com/ClickHouse/ClickHouse/pull/28534) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Bump poco to remove getpid() calls [#28537](https://github.com/ClickHouse/ClickHouse/pull/28537) ([Azat Khuzhin](https://github.com/azat)). * Fix broken kafka test [#28542](https://github.com/ClickHouse/ClickHouse/pull/28542) ([alesapin](https://github.com/alesapin)). * Fix format names in docs [#28557](https://github.com/ClickHouse/ClickHouse/pull/28557) ([Kruglov Pavel](https://github.com/Avogar)). * Fix wrong header of minmax_count projection [#28560](https://github.com/ClickHouse/ClickHouse/pull/28560) ([Amos Bird](https://github.com/amosbird)). * remove recursion in ZstdInflatingReadBuffer [#28561](https://github.com/ClickHouse/ClickHouse/pull/28561) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Improve [C|T]SV errors [#28579](https://github.com/ClickHouse/ClickHouse/pull/28579) ([Raúl Marín](https://github.com/Algunenano)). * Function dictGet small fix [#28615](https://github.com/ClickHouse/ClickHouse/pull/28615) ([Maksim Kita](https://github.com/kitaisreal)). * Fix arcadia build [#28640](https://github.com/ClickHouse/ClickHouse/pull/28640) ([DimasKovas](https://github.com/DimasKovas)). * Add missed log level into TextLog [#28648](https://github.com/ClickHouse/ClickHouse/pull/28648) ([alesapin](https://github.com/alesapin)). * Revert [#28082](https://github.com/ClickHouse/ClickHouse/issues/28082) [#28665](https://github.com/ClickHouse/ClickHouse/pull/28665) ([Kseniia Sumarokova](https://github.com/kssenii)). * Revert [#28397](https://github.com/ClickHouse/ClickHouse/issues/28397) [#28667](https://github.com/ClickHouse/ClickHouse/pull/28667) ([Kseniia Sumarokova](https://github.com/kssenii)).