Changelog for 23.10

This commit is contained in:
Alexey Milovidov 2023-11-02 12:29:10 +01:00
parent 1267246785
commit cdbf7c41f5

View File

@ -12,6 +12,182 @@
# 2023 Changelog
### ClickHouse release master (823b62a55c1) FIXME as compared to v23.9.1.1854-stable (8f9a227de1f)
#### Backward Incompatible Change
* Rewrited the experimental `S3Queue` table engine completely: changed the way we keep information in zookeeper which allows to make less zookeeper requests, added caching of zookeeper state in cases when we know the state will not change, improved the polling from s3 process to make it less aggressive, changed the way ttl and max set for trached files is maintained, now it is a background process. Added `system.s3queue` and `system.s3queue_log` tables. Closes [#54998](https://github.com/ClickHouse/ClickHouse/issues/54998). [#54422](https://github.com/ClickHouse/ClickHouse/pull/54422) ([Kseniia Sumarokova](https://github.com/kssenii)).
* There is no longer an option to automatically remove broken data parts. This closes [#55174](https://github.com/ClickHouse/ClickHouse/issues/55174). [#55184](https://github.com/ClickHouse/ClickHouse/pull/55184) ([Alexey Milovidov](https://github.com/alexey-milovidov)). [#55557](https://github.com/ClickHouse/ClickHouse/pull/55557) ([Jihyuk Bok](https://github.com/tomahawk28)).
* The obsolete in-memory data parts can no longer be read from the write-ahead log. If you have configured in-memory parts before, they have to be removed before the upgrade. [#55186](https://github.com/ClickHouse/ClickHouse/pull/55186) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Remove the integration with Meilisearch. Reason: it was compatible only with the old version 0.18. The recent version of Meilisearch changed the protocol and does not work anymore. Note: we would appreciate it if you help to return it back. [#55189](https://github.com/ClickHouse/ClickHouse/pull/55189) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Rename directory monitor concept into background INSERT. All the settings `*directory_monitor*` had been renamed to `distributed_background_insert*`. *Backward compatibility should be preserved* (since old settings had been added as an alias). [#55978](https://github.com/ClickHouse/ClickHouse/pull/55978) ([Azat Khuzhin](https://github.com/azat)).
* Do not interpret the `send_timeout` set on the client side as the `receive_timeout` on the server side and vise-versa. [#56035](https://github.com/ClickHouse/ClickHouse/pull/56035) ([Azat Khuzhin](https://github.com/azat)).
* Comparison of time intervals with different units will throw an exception. This closes [#55942](https://github.com/ClickHouse/ClickHouse/issues/55942). You might have occasionally rely on the previous behavior when the underlying numeric values were compared regardless of the units. [#56090](https://github.com/ClickHouse/ClickHouse/pull/56090) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
#### New Feature
* Add function `arrayFold(accumulator, x1, ..., xn -> expression, initial, array1, ..., arrayn)` which applies a lambda function to multiple arrays of the same cardinality and collects the result in an accumulator. [#49794](https://github.com/ClickHouse/ClickHouse/pull/49794) ([Lirikl](https://github.com/Lirikl)).
* Support for `Npy` format. `SELECT * FROM file('example_array.npy', Npy)`. [#55982](https://github.com/ClickHouse/ClickHouse/pull/55982) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* If a table has a space-filling curve in its key, e.g., `ORDER BY mortonEncode(x, y)`, the conditions on its arguments, e.g., `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30` can be used for indexing. A setting `analyze_index_with_space_filling_curves` is added to enable or disable this analysis. This closes [#41195](https://github.com/ClickHouse/ClickHouse/issue/41195). Continuation of [#4538](https://github.com/ClickHouse/ClickHouse/pull/4538). Continuation of [#6286](https://github.com/ClickHouse/ClickHouse/pull/6286). Continuation of [#28130](https://github.com/ClickHouse/ClickHouse/pull/28130). Continuation of [#41753](https://github.com/ClickHouse/ClickHouse/pull/#41753). [#55642](https://github.com/ClickHouse/ClickHouse/pull/55642) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* A new setting called `force_optimize_projection_name`, it takes a name of projection as an argument. If it's value set to a non-empty string, ClickHouse checks that this projection is used in the query at least once. Closes [#55331](https://github.com/ClickHouse/ClickHouse/issues/55331). [#56134](https://github.com/ClickHouse/ClickHouse/pull/56134) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* Support asynchronous inserts with external data via native protocol. Previously it worked only if data is inlined into query. [#54730](https://github.com/ClickHouse/ClickHouse/pull/54730) ([Anton Popov](https://github.com/CurtizJ)).
* Added aggregation function `lttb` which uses the [Largest-Triangle-Three-Buckets](https://skemman.is/bitstream/1946/15343/3/SS_MSthesis.pdf) algorithm for downsampling data for visualization. [#53145](https://github.com/ClickHouse/ClickHouse/pull/53145) ([Sinan](https://github.com/sinsinan)).
* Query`CHECK TABLE` has better performance and usability (sends progress updates, cancellable). Support checking particular part with `CHECK TABLE ... PART 'part_name'`. [#53404](https://github.com/ClickHouse/ClickHouse/pull/53404) ([vdimir](https://github.com/vdimir)).
* Added function `jsonMergePatch`. When working with JSON data as strings, it provides a way to merge these strings (of JSON objects) together to form a single string containing a single JSON object. [#54364](https://github.com/ClickHouse/ClickHouse/pull/54364) ([Memo](https://github.com/Joeywzr)).
* The second part of Kusto Query Language dialect support. [Phase 1 implementation ](https://github.com/ClickHouse/ClickHouse/pull/37961) has been merged. [#42510](https://github.com/ClickHouse/ClickHouse/pull/42510) ([larryluogit](https://github.com/larryluogit)).
* Added a new SQL function, `arrayRandomSample(arr, k)` which returns a sample of k elements from the input array. Similar functionality could previously be achieved only with less convenient syntax, e.g. "SELECT arrayReduce('groupArraySample(3)', range(10))". [#54391](https://github.com/ClickHouse/ClickHouse/pull/54391) ([itayisraelov](https://github.com/itayisraelov)).
* Introduce `-ArgMin`/`-ArgMax` aggregate combinators which allow to aggregate by min/max values only. One use case can be found in [#54818](https://github.com/ClickHouse/ClickHouse/issues/54818). This PR also reorganize combinators into dedicated folder. [#54947](https://github.com/ClickHouse/ClickHouse/pull/54947) ([Amos Bird](https://github.com/amosbird)).
* Allow to drop cache for Protobuf format with `SYSTEM DROP SCHEMA FORMAT CACHE [FOR Protobuf]`. [#55064](https://github.com/ClickHouse/ClickHouse/pull/55064) ([Aleksandr Musorin](https://github.com/AVMusorin)).
* Add external HTTP Basic authenticator. [#55199](https://github.com/ClickHouse/ClickHouse/pull/55199) ([Aleksei Filatov](https://github.com/aalexfvk)).
* Added function `byteSwap` which reverses the bytes of unsigned integers. This is particularly useful for reversing values of types which are represented as unsigned integers internally such as IPv4. [#55211](https://github.com/ClickHouse/ClickHouse/pull/55211) ([Priyansh Agrawal](https://github.com/Priyansh121096)).
* Added function `formatQuery()` which returns a formatted version (possibly spanning multiple lines) of a SQL query string. Also added function `formatQuerySingleLine()` which does the same but the returned string will not contain linebreaks. [#55239](https://github.com/ClickHouse/ClickHouse/pull/55239) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Added `DWARF` input format that reads debug symbols from an ELF executable/library/object file. [#55450](https://github.com/ClickHouse/ClickHouse/pull/55450) ([Michael Kolupaev](https://github.com/al13n321)).
* Allow to save unparsed records and errors in RabbitMQ, NATS and FileLog engines. Add virtual columns `_error` and `_raw_message`(for NATS and RabbitMQ), `_raw_record` (for FileLog) that are filled when ClickHouse fails to parse new record. The behaviour is controlled under storage settings `nats_handle_error_mode` for NATS, `rabbitmq_handle_error_mode` for RabbitMQ, `handle_error_mode` for FileLog similar to `kafka_handle_error_mode`. If it's set to `default`, en exception will be thrown when ClickHouse fails to parse a record, if it's set to `stream`, erorr and raw record will be saved into virtual columns. Closes [#36035](https://github.com/ClickHouse/ClickHouse/issues/36035). [#55477](https://github.com/ClickHouse/ClickHouse/pull/55477) ([Kruglov Pavel](https://github.com/Avogar)).
* Keeper client improvement: add `get_all_children_number command` that returns number of all children nodes under a specific path. [#55485](https://github.com/ClickHouse/ClickHouse/pull/55485) ([guoxiaolong](https://github.com/guoxiaolongzte)).
* Keeper client improvement: add `get_direct_children_number` command that returns number of direct children nodes under a path. [#55898](https://github.com/ClickHouse/ClickHouse/pull/55898) ([xuzifu666](https://github.com/xuzifu666)).
* Add setting `optimize_trivial_approximate_count_query` to use `count()` approximation for storage EmbeddedRocksDB. Enable trivial count for StorageJoin. [#55806](https://github.com/ClickHouse/ClickHouse/pull/55806) ([Duc Canh Le](https://github.com/canhld94)).
* Add statement `SHOW SETTING setting_name` which is a simpler version of existing statement `SHOW SETTINGS`. [#55979](https://github.com/ClickHouse/ClickHouse/pull/55979) ([Maksim Kita](https://github.com/kitaisreal)).
* Added fields `substreams` and `filenames` to the `system.parts_columns` table. [#55108](https://github.com/ClickHouse/ClickHouse/pull/55108) ([Anton Popov](https://github.com/CurtizJ)).
* Add support for `SHOW MERGES` query. [#55815](https://github.com/ClickHouse/ClickHouse/pull/55815) ([megao](https://github.com/jetgm)).
* Introduce a setting `create_table_empty_primary_key_by_default` for default `ORDER BY ()`. [#55899](https://github.com/ClickHouse/ClickHouse/pull/55899) ([Srikanth Chekuri](https://github.com/srikanthccv)).
* On a Keeper with lots of watches AsyncMetrics threads can consume 100% of CPU for noticable time in `DB::KeeperStorage::getSessionsWithWatchesCount()`. The fix is to avoid traversing heavy `watches` and `list_watches` sets. [#56054](https://github.com/ClickHouse/ClickHouse/pull/56054) ([Alexander Gololobov](https://github.com/davenger)).
#### Performance Improvement
* Add option `query_plan_preserve_num_streams_after_window_functions` to preserve the number of streams after evaluating window functions to allow parallel stream processing. [#50771](https://github.com/ClickHouse/ClickHouse/pull/50771) ([frinkr](https://github.com/frinkr)).
* Release more num_streams if data is small. [#53867](https://github.com/ClickHouse/ClickHouse/pull/53867) ([Jiebin Sun](https://github.com/jiebinn)).
* Posting lists in inverted indexes are now optimized to use the smallest possible representation for internal bitmaps. Depending on the repetitiveness of the data, this may significantly reduce the space consumption of inverted indexes. [#55069](https://github.com/ClickHouse/ClickHouse/pull/55069) ([Harry Lee](https://github.com/HarryLeeIBM)).
* Fix contention on Context lock, this significantly improves performance for a lot of short-running concurrent queries. [#55121](https://github.com/ClickHouse/ClickHouse/pull/55121) ([Maksim Kita](https://github.com/kitaisreal)).
* Improved the performance of inverted index creation by 30%. This was achieved by replacing `std::unordered_map` with `absl::flat_hash_map`. [#55210](https://github.com/ClickHouse/ClickHouse/pull/55210) ([Harry Lee](https://github.com/HarryLeeIBM)).
* Support ORC filter push down (rowgroup level). [#55330](https://github.com/ClickHouse/ClickHouse/pull/55330) ([李扬](https://github.com/taiyang-li)).
* Improve performance of external aggregation with a lot of temporary files. [#55489](https://github.com/ClickHouse/ClickHouse/pull/55489) ([Maksim Kita](https://github.com/kitaisreal)).
* Set a reasonable size for the marks cache for secondary indices by default to avoid loading the marks over and over again. [#55654](https://github.com/ClickHouse/ClickHouse/pull/55654) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Avoid unnecessary reconstruction of index granules when reading skip indexes. This addresses [#55653](https://github.com/ClickHouse/ClickHouse/issues/55653#issuecomment-1763766009). [#55683](https://github.com/ClickHouse/ClickHouse/pull/55683) ([Amos Bird](https://github.com/amosbird)).
* Cache CAST function in set during execution to improve the performance of function `IN` when set element type doesn't exactly match column type. [#55712](https://github.com/ClickHouse/ClickHouse/pull/55712) ([Duc Canh Le](https://github.com/canhld94)).
* Performance improvement for `ColumnVector::insertMany` and `ColumnVector::insertManyFrom`. [#55714](https://github.com/ClickHouse/ClickHouse/pull/55714) ([frinkr](https://github.com/frinkr)).
* Optimized Map subscript operations by predicting the next row's key position and reduce the comparisons. [#55929](https://github.com/ClickHouse/ClickHouse/pull/55929) ([lgbo](https://github.com/lgbo-ustc)).
* Support struct fields pruning in Parquet (in previous versions it didn't work in some cases). [#56117](https://github.com/ClickHouse/ClickHouse/pull/56117) ([lgbo](https://github.com/lgbo-ustc)).
* Add the ability to tune the number of parallel replicas used in a query execution based on the estimation of rows to read. [#51692](https://github.com/ClickHouse/ClickHouse/pull/51692) ([Raúl Marín](https://github.com/Algunenano)).
* Optimized external aggregation memory consumption in case many temporary files were generated. [#54798](https://github.com/ClickHouse/ClickHouse/pull/54798) ([Nikita Taranov](https://github.com/nickitat)).
* Distributed queries executed in `async_socket_for_remote` mode (default) now respect `max_threads` limit. Previously, some queries could create excessive threads (up to `max_distributed_connections`), causing server performance issues. [#53504](https://github.com/ClickHouse/ClickHouse/pull/53504) ([filimonov](https://github.com/filimonov)).
* Caching skip-able entries while executing DDL from Zookeeper distributed DDL queue. [#54828](https://github.com/ClickHouse/ClickHouse/pull/54828) ([Duc Canh Le](https://github.com/canhld94)).
* Experimental inverted indexes do not store tokens with too many matches (i.e. row ids in the posting list). This saves space and avoids ineffective index lookups when sequential scans would be equally fast or faster. The previous heuristics (`density` parameter passed to the index definition) that controlled when tokens would not be stored was too confusing for users. A much simpler heuristics based on parameter `max_rows_per_postings_list` (default: 64k) is introduced which directly controls the maximum allowed number of row ids in a postings list. [#55616](https://github.com/ClickHouse/ClickHouse/pull/55616) ([Harry Lee](https://github.com/HarryLeeIBM)).
* Improve write performance to `EmbeddedRocksDB` tables. [#55732](https://github.com/ClickHouse/ClickHouse/pull/55732) ([Duc Canh Le](https://github.com/canhld94)).
* Improved overall resilience for ClickHouse in case of many parts within partition (more than 1000). It might reduce the number of `TOO_MANY_PARTS` errors. [#55526](https://github.com/ClickHouse/ClickHouse/pull/55526) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Reduced memory consumption during loading of hierarchical dictionaries. [#55838](https://github.com/ClickHouse/ClickHouse/pull/55838) ([Nikita Taranov](https://github.com/nickitat)).
* All dictionaries support setting `dictionary_use_async_executor`. [#55839](https://github.com/ClickHouse/ClickHouse/pull/55839) ([vdimir](https://github.com/vdimir)).
* Prevent excesive memory usage when deserializing AggregateFunctionTopKGenericData. [#55947](https://github.com/ClickHouse/ClickHouse/pull/55947) ([Raúl Marín](https://github.com/Algunenano)).
#### Improvement
* Functions `toDayOfWeek()` (MySQL alias: `DAYOFWEEK()`), `toYearWeek()` (`YEARWEEK()`) and `toWeek()` (`WEEK()`) now supports `String` arguments. This makes its behavior consistent with MySQL's behavior. [#55589](https://github.com/ClickHouse/ClickHouse/pull/55589) ([Robert Schulze](https://github.com/rschu1ze)).
* Introduced setting `date_time_overflow_behavior` with possible values `ignore`, `throw`, `saturate` that controls the overflow behavior when converting from Date, Date32, DateTime64, Integer or Float to Date, Date32, DateTime or DateTime64. [#55696](https://github.com/ClickHouse/ClickHouse/pull/55696) ([Andrey Zvonov](https://github.com/zvonand)).
* Implement query parameters support for `ALTER TABLE ... ACTION PARTITION [ID] {parameter_name:ParameterType}`. Merges [#49516](https://github.com/ClickHouse/ClickHouse/issues/49516). Closes [#49449](https://github.com/ClickHouse/ClickHouse/issues/49449). [#55604](https://github.com/ClickHouse/ClickHouse/pull/55604) ([alesapin](https://github.com/alesapin)).
* Print processor ids in a prettier manner in EXPLAIN. [#48852](https://github.com/ClickHouse/ClickHouse/pull/48852) ([Vlad Seliverstov](https://github.com/behebot)).
* Creating a direct dictionary with a lifetime field will be rejected at create time (as the lifetime does not make sense for direct dictionaries). Fixes: [#27861](https://github.com/ClickHouse/ClickHouse/issues/27861). [#49043](https://github.com/ClickHouse/ClickHouse/pull/49043) ([Rory Crispin](https://github.com/RoryCrispin)).
* Allow parameters in queries with partitions like `ALTER TABLE t DROP PARTITION`. Closes [#49449](https://github.com/ClickHouse/ClickHouse/issues/49449). [#49516](https://github.com/ClickHouse/ClickHouse/pull/49516) ([Nikolay Degterinsky](https://github.com/evillique)).
* Add a new column `xid` for `system.zookeeper_connection`. [#50702](https://github.com/ClickHouse/ClickHouse/pull/50702) ([helifu](https://github.com/helifu)).
* Display the correct server settings in `system.server_settings` after configuration reload. [#53774](https://github.com/ClickHouse/ClickHouse/pull/53774) ([helifu](https://github.com/helifu)).
* Add support for mathematical minus `` character in queries, similar to `-`. [#54100](https://github.com/ClickHouse/ClickHouse/pull/54100) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add replica groups to the experimental `Replicated` database engine. Closes [#53620](https://github.com/ClickHouse/ClickHouse/issues/53620). [#54421](https://github.com/ClickHouse/ClickHouse/pull/54421) ([Nikolay Degterinsky](https://github.com/evillique)).
* It is better to retry retriable s3 errors than totally fail the query. Set bigger value to the s3_retry_attempts by default. [#54770](https://github.com/ClickHouse/ClickHouse/pull/54770) ([Sema Checherinda](https://github.com/CheSema)).
* Add load balancing mode `hostname_levenshtein_distance`. [#54826](https://github.com/ClickHouse/ClickHouse/pull/54826) ([JackyWoo](https://github.com/JackyWoo)).
* Improve hiding secrets in logs. [#55089](https://github.com/ClickHouse/ClickHouse/pull/55089) ([Vitaly Baranov](https://github.com/vitlibar)).
* For now the projection analysis will be performed only on top of query plan. The setting `query_plan_optimize_projection` became obsolete (it was enabled by default long time ago). [#55112](https://github.com/ClickHouse/ClickHouse/pull/55112) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* When function `untuple` is now called on a tuple with named elements and itself has an alias (e.g. `select untuple(tuple(1)::Tuple(element_alias Int)) AS untuple_alias`), then the result column name is now generated from the untuple alias and the tuple element alias (in the example: "untuple_alias.element_alias"). [#55123](https://github.com/ClickHouse/ClickHouse/pull/55123) ([garcher22](https://github.com/garcher22)).
* Added setting `describe_include_virtual_columns`, which allows to include virtual columns of table into result of `DESCRIBE` query. Added setting `describe_compact_output`. If it is set to `true`, `DESCRIBE` query returns only names and types of columns without extra information. [#55129](https://github.com/ClickHouse/ClickHouse/pull/55129) ([Anton Popov](https://github.com/CurtizJ)).
* Sometimes `OPTIMIZE` with `optimize_throw_if_noop=1` may fail with an error `unknown reason` while the real cause of it - different projections in different parts. This behavior is fixed. [#55130](https://github.com/ClickHouse/ClickHouse/pull/55130) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)).
* Allow to have several `MaterializedPostgreSQL` tables following the same Postgres table. By default this behaviour is not enabled (for compatibility, because it is a backward-incompatible change), but can be turned on with setting `materialized_postgresql_use_unique_replication_consumer_identifier`. Closes [#54918](https://github.com/ClickHouse/ClickHouse/issues/54918). [#55145](https://github.com/ClickHouse/ClickHouse/pull/55145) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Allow to parse negative `DateTime64` and `DateTime` with fractional part from short strings. [#55146](https://github.com/ClickHouse/ClickHouse/pull/55146) ([Andrey Zvonov](https://github.com/zvonand)).
* To improve compatibility with MySQL, 1. `information_schema.tables` now includes the new field `table_rows`, and 2. `information_schema.columns` now includes the new field `extra`. [#55215](https://github.com/ClickHouse/ClickHouse/pull/55215) ([Robert Schulze](https://github.com/rschu1ze)).
* Clickhouse-client won't show "0 rows in set" if it is zero and if exception was thrown. [#55240](https://github.com/ClickHouse/ClickHouse/pull/55240) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Support rename table without keyword `TABLE` like `RENAME db.t1 to db.t2`. [#55373](https://github.com/ClickHouse/ClickHouse/pull/55373) ([凌涛](https://github.com/lingtaolf)).
* Add `internal_replication` to `system.clusters`. [#55377](https://github.com/ClickHouse/ClickHouse/pull/55377) ([Konstantin Morozov](https://github.com/k-morozov)).
* Select remote proxy resolver based on request protocol, add proxy feature docs and remove `DB::ProxyConfiguration::Protocol::ANY`. [#55430](https://github.com/ClickHouse/ClickHouse/pull/55430) ([Arthur Passos](https://github.com/arthurpassos)).
* Avoid retrying keeper operations on INSERT after table shutdown. [#55519](https://github.com/ClickHouse/ClickHouse/pull/55519) ([Azat Khuzhin](https://github.com/azat)).
* `SHOW COLUMNS` now correctly reports type `FixedString` as `BLOB` if setting `use_mysql_types_in_show_columns` is on. Also added two new settings, `mysql_map_string_to_text_in_show_columns` and `mysql_map_fixed_string_to_text_in_show_columns` to switch the output for types `String` and `FixedString` as `TEXT` or `BLOB`. [#55617](https://github.com/ClickHouse/ClickHouse/pull/55617) ([Serge Klochkov](https://github.com/slvrtrn)).
* During ReplicatedMergeTree tables startup clickhouse server checks set of parts for unexpected parts (exists locally, but not in zookeeper). All unexpected parts move to detached directory and instead of them server tries to restore some ancestor (covered) parts. Now server tries to restore closest ancestors instead of random covered parts. [#55645](https://github.com/ClickHouse/ClickHouse/pull/55645) ([alesapin](https://github.com/alesapin)).
* The advanced dashboard now supports draggable charts on touch devices. This closes [#54206](https://github.com/ClickHouse/ClickHouse/issues/54206). [#55649](https://github.com/ClickHouse/ClickHouse/pull/55649) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Use the default query format if declared when outputting exception with `http_write_exception_in_output_format`. [#55739](https://github.com/ClickHouse/ClickHouse/pull/55739) ([Raúl Marín](https://github.com/Algunenano)).
* Provide a better message for common MATERIALIZED VIEW pitfalls. [#55826](https://github.com/ClickHouse/ClickHouse/pull/55826) ([Raúl Marín](https://github.com/Algunenano)).
* If you dropped the current database, you will still be able to run some queries in `clickhouse-local` and switch to another database. This makes the behavior consistent with `clickhouse-client`. This closes [#55834](https://github.com/ClickHouse/ClickHouse/issues/55834). [#55853](https://github.com/ClickHouse/ClickHouse/pull/55853) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Functions `(add|subtract)(Year|Quarter|Month|Week|Day|Hour|Minute|Second|Millisecond|Microsecond|Nanosecond)` now support string-encoded date arguments, e.g. `SELECT addDays('2023-10-22', 1)`. This increases compatibility with MySQL and is needed by Tableau Online. [#55869](https://github.com/ClickHouse/ClickHouse/pull/55869) ([Robert Schulze](https://github.com/rschu1ze)).
* The setting `apply_deleted_mask` when disabled allows to read rows that where marked as deleted by lightweight DELETE queries. This is useful for debugging. [#55952](https://github.com/ClickHouse/ClickHouse/pull/55952) ([Alexander Gololobov](https://github.com/davenger)).
* Allow skipping `null` values when serailizing Tuple to json objects, which makes it possible to keep compatiability with Spark's `to_json` function, which is also useful for gluten. [#55956](https://github.com/ClickHouse/ClickHouse/pull/55956) ([李扬](https://github.com/taiyang-li)).
* Functions `(add|sub)Date()` now support string-encoded date arguments, e.g. `SELECT addDate('2023-10-22 11:12:13', INTERVAL 5 MINUTE)`. The same support for string-encoded date arguments is added to the plus and minus operators, e.g. `SELECT '2023-10-23' + INTERVAL 1 DAY`. This increases compatibility with MySQL and is needed by Tableau Online. [#55960](https://github.com/ClickHouse/ClickHouse/pull/55960) ([Robert Schulze](https://github.com/rschu1ze)).
* Allow unquoted strings with CR (`\r`) in CSV format. Closes [#39930](https://github.com/ClickHouse/ClickHouse/issues/39930). [#56046](https://github.com/ClickHouse/ClickHouse/pull/56046) ([Kruglov Pavel](https://github.com/Avogar)).
* Allow to run `clickhouse-keeper` using embedded config. [#56086](https://github.com/ClickHouse/ClickHouse/pull/56086) ([Maksim Kita](https://github.com/kitaisreal)).
* Set limit of the maximum configuration value for `queued.min.messages` to avoid problem with start fetching data with Kafka. [#56121](https://github.com/ClickHouse/ClickHouse/pull/56121) ([Stas Morozov](https://github.com/r3b-fish)).
* Fixed a typo in SQL function `minSampleSizeContinous` (renamed `minSampleSizeContinuous`). Old name is preserved for backward compatibility. This closes: [#56139](https://github.com/ClickHouse/ClickHouse/issues/56139). [#56143](https://github.com/ClickHouse/ClickHouse/pull/56143) ([Dorota Szeremeta](https://github.com/orotaday)).
* Print path for broken parts on disk before shutting down the server. Before this change if a part is corrupted on disk and server cannot start, it was almost impossible to understand which part is broken. This is fixed. [#56181](https://github.com/ClickHouse/ClickHouse/pull/56181) ([Duc Canh Le](https://github.com/canhld94)).
#### Build/Testing/Packaging Improvement
* If the database in Docker is already initialized, it doesn't need to be initialized again upon subsequent launches. This can potentially fix the issue of infinite container restarts when the database fails to load within 1000 attempts (relevant for very large databases and multi-node setups). [#50724](https://github.com/ClickHouse/ClickHouse/pull/50724) ([Alexander Nikolaev](https://github.com/AlexNik)).
* Resource with source code including submodules is built in Darwin special build task. It may be used to build ClickHouse without checking out the submodules. [#51435](https://github.com/ClickHouse/ClickHouse/pull/51435) ([Ilya Yatsishin](https://github.com/qoega)).
* An error was occuring when building ClickHouse with the AVX series of instructions enabled globally (which isn't recommended). The reason is that snappy does not enable `SNAPPY_HAVE_X86_CRC32`. [#55049](https://github.com/ClickHouse/ClickHouse/pull/55049) ([monchickey](https://github.com/monchickey)).
* Solve issue with launching standalone `clickhouse-keeper` from `clickhouse-server` package. [#55226](https://github.com/ClickHouse/ClickHouse/pull/55226) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* In the tests, RabbitMQ version is updated to 3.12.6. Improved logs collection for RabbitMQ tests. [#55424](https://github.com/ClickHouse/ClickHouse/pull/55424) ([Ilya Yatsishin](https://github.com/qoega)).
* Modified the error message difference between openssl and boringssl to fix the functional test. [#55975](https://github.com/ClickHouse/ClickHouse/pull/55975) ([MeenaRenganathan22](https://github.com/MeenaRenganathan22)).
* Use upstream repo for apache datasketches. [#55787](https://github.com/ClickHouse/ClickHouse/pull/55787) ([Nikita Taranov](https://github.com/nickitat)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Skip hardlinking inverted index files in mutation [#47663](https://github.com/ClickHouse/ClickHouse/pull/47663) ([cangyin](https://github.com/cangyin)).
* Fix 'Cannot find column' in read-in-order optimization with ARRAY JOIN [#51746](https://github.com/ClickHouse/ClickHouse/pull/51746) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
* Support missed experimental `Object(Nullable(json))` subcolumns in query. [#54052](https://github.com/ClickHouse/ClickHouse/pull/54052) ([zps](https://github.com/VanDarkholme7)).
* Re-add fix for `accurateCastOrNull()` [#54629](https://github.com/ClickHouse/ClickHouse/pull/54629) ([Salvatore Mesoraca](https://github.com/aiven-sal)).
* Fix detecting `DEFAULT` for columns of a Distributed table created without AS [#55060](https://github.com/ClickHouse/ClickHouse/pull/55060) ([Vitaly Baranov](https://github.com/vitlibar)).
* Proper cleanup in case of exception in ctor of ShellCommandSource [#55103](https://github.com/ClickHouse/ClickHouse/pull/55103) ([Alexander Gololobov](https://github.com/davenger)).
* Fix deadlock in LDAP assigned role update [#55119](https://github.com/ClickHouse/ClickHouse/pull/55119) ([Julian Maicher](https://github.com/jmaicher)).
* Suppress error statistics update for internal exceptions [#55128](https://github.com/ClickHouse/ClickHouse/pull/55128) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix deadlock in backups [#55132](https://github.com/ClickHouse/ClickHouse/pull/55132) ([alesapin](https://github.com/alesapin)).
* Fix storage Iceberg files retrieval [#55144](https://github.com/ClickHouse/ClickHouse/pull/55144) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Fix partition pruning of extra columns in set. [#55172](https://github.com/ClickHouse/ClickHouse/pull/55172) ([Amos Bird](https://github.com/amosbird)).
* Fix recalculation of skip indexes in ALTER UPDATE queries when table has adaptive granularity [#55202](https://github.com/ClickHouse/ClickHouse/pull/55202) ([Duc Canh Le](https://github.com/canhld94)).
* Fix for background download in fs cache [#55252](https://github.com/ClickHouse/ClickHouse/pull/55252) ([Kseniia Sumarokova](https://github.com/kssenii)).
* Avoid possible memory leaks in compressors in case of missing buffer finalization [#55262](https://github.com/ClickHouse/ClickHouse/pull/55262) ([Azat Khuzhin](https://github.com/azat)).
* Fix functions execution over sparse columns [#55275](https://github.com/ClickHouse/ClickHouse/pull/55275) ([Azat Khuzhin](https://github.com/azat)).
* Fix incorrect merging of Nested for SELECT FINAL FROM SummingMergeTree [#55276](https://github.com/ClickHouse/ClickHouse/pull/55276) ([Azat Khuzhin](https://github.com/azat)).
* Fix bug with inability to drop detached partition in replicated merge tree on top of S3 without zero copy [#55309](https://github.com/ClickHouse/ClickHouse/pull/55309) ([alesapin](https://github.com/alesapin)).
* Fix a crash in MergeSortingPartialResultTransform (due to zero chunks after `remerge`) [#55335](https://github.com/ClickHouse/ClickHouse/pull/55335) ([Azat Khuzhin](https://github.com/azat)).
* Fix data-race in CreatingSetsTransform (on errors) due to throwing shared exception [#55338](https://github.com/ClickHouse/ClickHouse/pull/55338) ([Azat Khuzhin](https://github.com/azat)).
* Fix trash optimization (up to a certain extent) [#55353](https://github.com/ClickHouse/ClickHouse/pull/55353) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Fix leak in StorageHDFS [#55370](https://github.com/ClickHouse/ClickHouse/pull/55370) ([Azat Khuzhin](https://github.com/azat)).
* Fix parsing of arrays in cast operator [#55417](https://github.com/ClickHouse/ClickHouse/pull/55417) ([Anton Popov](https://github.com/CurtizJ)).
* Fix filtering by virtual columns with OR filter in query [#55418](https://github.com/ClickHouse/ClickHouse/pull/55418) ([Azat Khuzhin](https://github.com/azat)).
* Fix MongoDB connection issues [#55419](https://github.com/ClickHouse/ClickHouse/pull/55419) ([Nikolay Degterinsky](https://github.com/evillique)).
* Fix MySQL interface boolean representation [#55427](https://github.com/ClickHouse/ClickHouse/pull/55427) ([Serge Klochkov](https://github.com/slvrtrn)).
* Fix MySQL text protocol DateTime formatting and LowCardinality(Nullable(T)) types reporting [#55479](https://github.com/ClickHouse/ClickHouse/pull/55479) ([Serge Klochkov](https://github.com/slvrtrn)).
* Make `use_mysql_types_in_show_columns` affect only `SHOW COLUMNS` [#55481](https://github.com/ClickHouse/ClickHouse/pull/55481) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix stack symbolizer parsing `DW_FORM_ref_addr` incorrectly and sometimes crashing [#55483](https://github.com/ClickHouse/ClickHouse/pull/55483) ([Michael Kolupaev](https://github.com/al13n321)).
* Destroy fiber in case of exception in cancelBefore in AsyncTaskExecutor [#55516](https://github.com/ClickHouse/ClickHouse/pull/55516) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix Query Parameters not working with custom HTTP handlers [#55521](https://github.com/ClickHouse/ClickHouse/pull/55521) ([Konstantin Bogdanov](https://github.com/thevar1able)).
* Fix checking of non handled data for Values format [#55527](https://github.com/ClickHouse/ClickHouse/pull/55527) ([Azat Khuzhin](https://github.com/azat)).
* Fix 'Invalid cursor state' in odbc interacting with MS SQL Server [#55558](https://github.com/ClickHouse/ClickHouse/pull/55558) ([vdimir](https://github.com/vdimir)).
* Fix max execution time and 'break' overflow mode [#55577](https://github.com/ClickHouse/ClickHouse/pull/55577) ([Alexander Gololobov](https://github.com/davenger)).
* Fix crash in QueryNormalizer with cyclic aliases [#55602](https://github.com/ClickHouse/ClickHouse/pull/55602) ([vdimir](https://github.com/vdimir)).
* Disable wrong optimization and add a test [#55609](https://github.com/ClickHouse/ClickHouse/pull/55609) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Merging [#52352](https://github.com/ClickHouse/ClickHouse/issues/52352) [#55621](https://github.com/ClickHouse/ClickHouse/pull/55621) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
* Add a test to avoid incorrect decimal sorting [#55662](https://github.com/ClickHouse/ClickHouse/pull/55662) ([Amos Bird](https://github.com/amosbird)).
* Fix progress bar for s3 and azure Cluster functions with url without globs [#55666](https://github.com/ClickHouse/ClickHouse/pull/55666) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix filtering by virtual columns with OR filter in query (resubmit) [#55678](https://github.com/ClickHouse/ClickHouse/pull/55678) ([Azat Khuzhin](https://github.com/azat)).
* Fixes and improvements for Iceberg storage [#55695](https://github.com/ClickHouse/ClickHouse/pull/55695) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix data race in CreatingSetsTransform (v2) [#55786](https://github.com/ClickHouse/ClickHouse/pull/55786) ([Azat Khuzhin](https://github.com/azat)).
* Throw exception when parsing illegal string as float if precise_float_parsing is true [#55861](https://github.com/ClickHouse/ClickHouse/pull/55861) ([李扬](https://github.com/taiyang-li)).
* Disable predicate pushdown if the CTE contains stateful functions [#55871](https://github.com/ClickHouse/ClickHouse/pull/55871) ([Raúl Marín](https://github.com/Algunenano)).
* Fix normalize ASTSelectWithUnionQuery, as it was stripping `FORMAT` from the query [#55887](https://github.com/ClickHouse/ClickHouse/pull/55887) ([flynn](https://github.com/ucasfl)).
* Try to fix possible segfault in Native ORC input format [#55891](https://github.com/ClickHouse/ClickHouse/pull/55891) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix window functions in case of sparse columns. [#55895](https://github.com/ClickHouse/ClickHouse/pull/55895) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)).
* fix: StorageNull supports subcolumns [#55912](https://github.com/ClickHouse/ClickHouse/pull/55912) ([FFish](https://github.com/wxybear)).
* Do not write retriable errors for Replicated mutate/merge into error log [#55944](https://github.com/ClickHouse/ClickHouse/pull/55944) ([Azat Khuzhin](https://github.com/azat)).
* Fix `SHOW DATABASES LIMIT <N>` [#55962](https://github.com/ClickHouse/ClickHouse/pull/55962) ([Raúl Marín](https://github.com/Algunenano)).
* Fix autogenerated Protobuf schema with fields with underscore [#55974](https://github.com/ClickHouse/ClickHouse/pull/55974) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix dateTime64ToSnowflake64() with non-default scale [#55983](https://github.com/ClickHouse/ClickHouse/pull/55983) ([Robert Schulze](https://github.com/rschu1ze)).
* Fix output/input of Arrow dictionary column [#55989](https://github.com/ClickHouse/ClickHouse/pull/55989) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix fetching schema from schema registry in AvroConfluent [#55991](https://github.com/ClickHouse/ClickHouse/pull/55991) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table [#55995](https://github.com/ClickHouse/ClickHouse/pull/55995) ([Michael Kolupaev](https://github.com/al13n321)).
* Fix incorrect free space accounting for least_used JBOD policy [#56030](https://github.com/ClickHouse/ClickHouse/pull/56030) ([Azat Khuzhin](https://github.com/azat)).
* Fix missing scalar issue when evaluating subqueries inside table functions [#56057](https://github.com/ClickHouse/ClickHouse/pull/56057) ([Amos Bird](https://github.com/amosbird)).
* Fix wrong query result when http_write_exception_in_output_format=1 [#56135](https://github.com/ClickHouse/ClickHouse/pull/56135) ([Kruglov Pavel](https://github.com/Avogar)).
* Fix schema cache for fallback JSON->JSONEachRow with changed settings [#56172](https://github.com/ClickHouse/ClickHouse/pull/56172) ([Kruglov Pavel](https://github.com/Avogar)).
* Add error handler to odbc-bridge [#56185](https://github.com/ClickHouse/ClickHouse/pull/56185) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)).
### ClickHouse release 23.9, 2023-09-28
#### Backward Incompatible Change