ClickHouse/docs/changelogs/v23.1.1.3077-stable.md
Alexey Milovidov e0252db8d4 No prestable
2023-03-27 12:19:32 +02:00

108 KiB

sidebar_position sidebar_label
1 2023

2023 Changelog

ClickHouse release v23.1.1.3077-stable (dcaac47702) FIXME as compared to v22.12.1.1752-stable (688e488e93)

Backward Incompatible Change

  • Remove query SYSTEM RESTART DISK. #44647 (alesapin).
  • Disallow Gorilla compression on columns of non-Float32 or non-Float64 type. #45252 (Robert Schulze).
  • Remove PREALLOCATE for HASHED/SPARSE_HASHED dictionaries. #45388 (Azat Khuzhin).
  • Parallel quorum inserts might work incorrectly with *MergeTree tables created with deprecated syntax. Therefore, parallel quorum inserts support is completely disabled for such tables. It does not affect tables created with a new syntax. #45430 (Alexander Tokmakov).

New Feature

  • Add quantileInterpolatedWeighted/quantilesInterpolatedWeighted functions. #38252 (Bharat Nallan).
  • Add an experimental inverted index as a new secondary index type for efficient text search. #38667 (larryluogit).
  • Add column ptr to system.trace_log for trace_type = 'MemorySample'. This column contains an address of allocation. Added function flameGraph which can build flamegraph containing allocated and not released memory. Reworking of #38391. #38953 (Nikolai Kochetov).
  • Dictionary source for extracting keys by traversing regular expressions tree. #40878 (Vage Ogannisian).
  • Added parametrized view functionality, now it's possible to specify query parameters for View table engine. resolves #40907. #41687 (SmitaRKulkarni).
  • added extendable and configurable scheduling subsystem for IO requests (not yet integrated with IO code itself). #41840 (Sergei Trifonov).
  • Added SYSTEM DROP DATABASE REPLICA that removes metadata of dead replica of Replicated database. Resolves #41794. #42807 (Alexander Tokmakov).
  • Array join support map type, like function explode in spark. #43239 (李扬).
  • Support SQL standard binary and hex string literals. #43785 (Mo Xuan).
  • Add experimental query result cache. #43797 (Robert Schulze).
  • format datetime in joda datetime style. Refer to https://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html. #43818 (李扬).
  • to merge #40878 , supporting regexp dictionary. #43858 (Han Fei).
  • Implemented a fractional second formatter (%f) for formatDateTime. #44060 (ltrk2).
  • Added age function to calculate difference between two dates or dates with time values expressed as number of full units. Closes #41115. #44421 (Robert Schulze).
  • Implemented a fractional second formatter (%f) for formatDateTime. This is slightly modified PR #44060 by @ltrk2. #44497 (Alexander Gololobov).
  • Add null source for dictionaries. Closes #44240. #44502 (mayamika).
  • We can use s3_storage_class to set different tier. Such as <disks> <s3> <type>s3</type> <endpoint>xxx</endpoint> <access_key_id>xxx</access_key_id> <secret_access_key>xxx</secret_access_key> <s3_storage_class>STANDARD/INTELLIGENT_TIERING</s3_storage_class> </s3> </disks> Closes #44443. #44707 (chen).
  • Try to detect header with column names (and maybe types) for CSV/TSV/CustomSeparated input formats. Add settings input_format_tsv/csv/custom_detect_header that enables this behaviour (enabled by default). Closes #44640. #44953 (Kruglov Pavel).
  • Insert default values in case of missing elements in JSON object while parsing named tuple. Add setting input_format_json_defaults_for_missing_elements_in_named_tuple that controls this behaviour. Closes #45142#issuecomment-1380153217. #45231 (Kruglov Pavel).
  • Introduce non-throwing variants of hasToken and hasTokenCaseInsensitive. #45341 (ltrk2).

Performance Improvement

  • Added sharding support in HashedDictionary to allow parallel load (almost linear scaling based on number of shards). #40003 (Azat Khuzhin).
  • Do not load inactive parts at startup of MergeTree tables. #42181 (Anton Popov).
  • Always replace OR chain expr = x1 OR ... OR expr = xN to expr IN (x1, ..., xN) in case if expr is a LowCardinality column. Setting optimize_min_equality_disjunction_chain_length is ignored in this case. #42889 (Guo Wangyang).
  • Original changelog In the original implementation, the memory of ThreadGroupStatus:: finished_threads_counters_memory is released by moving it to a temporary std::vector, which soon expired and gets destructed. This method is viable, however not straightforward enough. To enhance the code readability, this commit releases the memory in the vector by firstly resizing it to 0 and then shrinking the capacity accordingly. #43586 (Zhiguo Zhou).

  • As a follow-up of #42214, this PR tries to optimize the column-wise ternary logic evaluation by achieving auto-vectorization. In the performance test of this microbenchmark, we've observed a peak performance gain of 21x on the ICX device (Intel Xeon Platinum 8380 CPU). #43669 (Zhiguo Zhou).
  • Improved latency of reading from storage S3 and table function s3 with large number of small files. Now settings remote_filesystem_read_method and remote_filesystem_read_prefetch take effect while reading from storage S3. #43726 (Anton Popov).
  • The performance experiments of SSB (Star Schema Benchmark) on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) shows that this change could effectively decrease the lock contention for ThreadPoolImpl::mutex by 75%, increasing the CPU utilization and improving the overall performance by 2.4%. #44308 (Zhiguo Zhou).
  • Now optimisation is applied only if the cached HT size is sufficiently large (thresholds were determined empirically and hardcoded). #44455 (Nikita Taranov).
  • ... The whole struct field will be loaded at current, even though we just want to read one field of the struct. #44484 (lgbo).
  • Small performance improvement for asynchronous reading from remote fs. #44868 (Kseniia Sumarokova).
  • Switched to faster shared (RW) mutex implementation. Performance may be improved in queries with a lot of thread synchronization or for data structures experiencing heavy contention. #45007 (Sergei Trifonov).
  • Add fast path for: - col like '%%' - col like '%' - col not like '%' - col not like '%' - match(col, '.*'). #45244 (李扬).
  • todo. #45289 (Nikita Taranov).

Improvement

  • Refactor and Improve streaming engines Kafka/RabbitMQ/NATS and add support for all formats, also refactor formats a bit: - Fix producing messages in row-based formats with suffixes/prefixes. Now every message is formatted complitely with all delimiters and can be parsed back using input format. - Support block-based formats like Native, Parquet, ORC, etc. Every block is formatted as a separated message. The number of rows in one message depends on block size, so you can control it via setting max_block_size. - Add new engine settings kafka_max_rows_per_message/rabbitmq_max_rows_per_message/nats_max_rows_per_message. They control the number of rows formatted in one message in row-based formats. Default value: 1. - Fix high memory consumption in NATS table engine. - Support arbitrary binary data in NATS producer (previously it worked only with strings contained \0 at the end) - Add missing Kafka/RabbitMQ/NATS engine settings in documentation. - Refactor producing and consuming in Kafka/RabbitMQ/NATS, separate it from WriteBuffers/ReadBuffers semantic. - Refactor output formats: remove callbacks on each row used in Kafka/RabbitMQ/NATS (now we don't use callbacks there), allow to use IRowOutputFormat directly, clarify row end and row between delimiters, make it possible to reset output format to start formatting again - Add proper implementation in formatRow function (bonus after formats refactoring). #42777 (Kruglov Pavel).
  • Support optimize_or_like_chain in the new infrastructure. Part of #42648. #42797 (Dmitry Novik).
  • Improve the Asterisk and ColumnMatcher parsers. Part of #42648. #42884 (Nikolay Degterinsky).
  • Implement optimize_redundant_functions_in_order_by on top of QueryTree. Part of #42648. #42970 (Dmitry Novik).
  • Support optimize_group_by_function_keys in the new analyzer architecture. Also, add support for optimizing GROUPING SETS keys. Part of #42648. #43261 (Dmitry Novik).
  • Improve reading CSV field in CustomSeparated/Template format. Closes #42352 Closes #39620. #43332 (Kruglov Pavel).
  • Support reading/writing Nested tables as List of Struct in CapnProto format. Read/write Decimal32/64 as Int32/64. Closes #43319. #43379 (Kruglov Pavel).
  • Support scalar subqueries cache Implementation: * Added a map with hash of the node (without alias) and the evaluated value to Context. Testing: * Added a test-case with new analyser in 02174_cte_scalar_cache.sql. #43640 (SmitaRKulkarni).
  • Improve automatic usage of structure from insertion table in table functions file/hdfs/s3 when virtual columns present in select query, it fixes possible error Block structure mismatch or number of columns mismatch. #43695 (Kruglov Pavel).
  • Add support for signed arguments in range(). Fixes #43333. #43733 (sanyu).
  • Remove redundant sorting, for example, sorting related ORDER BY clauses in subqueries. Implemented on top of query plan. It does similar optimization as optimize_duplicate_order_by_and_distinct regarding ORDER BY clauses, but more generic, since it's applied to any redundant sorting steps (not only caused by ORDER BY clause) and applied to subqueries of any depth. Related to #42648. #43905 (Igor Nikonov).
  • Added mmap support for StorageFile, which should improve the performance of clickhouse-local. #43927 (pufit).
  • Add ability to disable deduplication for BACKUP (for backups wiithout deduplication ATTACH can be used instead of full RESTORE), example BACKUP foo TO S3(...) SETTINGS deduplicate_files=0 (default deduplicate_files=1). #43947 (Azat Khuzhin).
  • Make system.replicas table do parallel fetches of replicas statuses. Closes #43918. #43998 (Nikolay Degterinsky).
  • Refactor and improve schema inference for text formats. Add new setting schema_inference_make_columns_nullable that controls making result types Nullable (enabled by default);. #44019 (Kruglov Pavel).
  • Better support for PROXYv1. #44135 (Yakov Olkhovskiy).
  • Add information about the latest part check by cleanup thread into system.parts table. #44244 (Dmitry Novik).
  • Disable functions in readonly for inserts. #44290 (SmitaRKulkarni).
  • Add a setting simultaneous_parts_removal_limit to allow to limit the number of parts being processed by one iteration of CleanupThread. #44461 (Dmitry Novik).
  • If user only need virtual columns, we don't need to initialize ReadBufferFromS3. May be helpful to #44246. #44493 (chen).
  • Prevent duplicate column names hints. Closes #44130. #44519 (Joanna Hulboj).
  • Allow macro substitution in endpoint of disks resolve #40951. #44533 (SmitaRKulkarni).
  • Added a message_format_string column to system.text_log. The column contains a pattern that was used to format the message. #44543 (Alexander Tokmakov).
  • Improve schema inference when input_format_json_read_object_as_string is enabled. #44546 (Kruglov Pavel).
  • Add user-level setting database_replicated_allow_replicated_engine_arguments which allow to ban creation of ReplicatedMergeTree tables with arguments in DatabaseReplicated. #44566 (alesapin).
  • Prevent users from mistakenly specifying zero (invalid) value for index_granularity. This closes #44536. #44578 (Alexey Milovidov).
  • Added possibility to set path to service keytab file in keytab parameter in kerberos section of config.xml. #44594 (Roman Vasin).
  • Use already written part of the query for fuzzy search (pass to skim). #44600 (Azat Khuzhin).
  • Enable input_format_json_read_objects_as_strings by default to be able to read nested JSON objects while JSON Object type is experimental. #44657 (Kruglov Pavel).
  • When users do duplicate async inserts, we should dedup inside the memory before we query keeper. #44682 (Han Fei).
  • Input/ouptut Avro bool type as ClickHouse bool type. #44684 (Kruglov Pavel).
  • Infer UInt64 in case of Int64 overflow and fix some transforms in schema inference. #44696 (Kruglov Pavel).
  • Previously dependency resolving inside DatabaseReplicated was done in a hacky way and now it done right using an explicit graph. #44697 (Nikita Mikhaylov).
  • Support Bool type in Arrow/Parquet/ORC. Closes #43970. #44698 (Kruglov Pavel).
  • Fix output_format_pretty_row_numbers does not preserve the counter across the blocks. Closes #44815. #44832 (flynn).
  • Extend function "toDayOfWeek" with a mode argument describing if a) the week starts on Monday or Sunday and b) if counting starts at 0 or 1. #44860 (李扬).
    • Don't report errors in system.errors due to parts being merged concurrently with the background cleanup process. #44874 (Raúl Marín).
  • Optimize and fix metrics for Distributed async INSERT. #44922 (Azat Khuzhin).
  • Added settings to disallow concurrent backups and restores resolves #43891 Implementation: * Added server level settings to disallow concurrent backups and restores, which are read and set when BackupWorker is created in Context. * Settings are set to true by default. * Before starting backup or restores, added a check to see if any other backups/restores are running. For internal request it checks if its from the self node using backup_uuid. #45072 (SmitaRKulkarni).
  • add a cache for async block ids. This will reduce the requests of zookeeper when we enable async inserts deduplication. #45106 (Han Fei).
  • CRC32 changes to address the WeakHash collision issue in PowerPC. #45144 (MeenaRenganathan22).
  • Optimize memory consumption during backup to S3: files to S3 now will be copied directly without using WriteBufferFromS3 (which could use a lot of memory). #45188 (Vitaly Baranov).
  • Use structure from insertion table in generateRandom without arguments. #45239 (Kruglov Pavel).
  • Use GetObjectAttributes request instead of HeadObject request to get the size of an object in AWS S3. This change fixes handling endpoints without explicit region, for example. #45288 (Vitaly Baranov).
  • Add <storage_policy> config parameter for system logs. #45320 (Stig Bakken).
  • Remove redundant sorting, for example, sorting related ORDER BY clauses in subqueries. Implemented on top of query plan. It does similar optimization as optimize_duplicate_order_by_and_distinct regarding ORDER BY clauses, but more generic, since it's applied to any redundant sorting steps (not only caused by ORDER BY clause) and applied to subqueries of any depth. Related to #42648. #45420 (Igor Nikonov).
  • Allow to implicitly convert floats stored in string fields of JSON to integers in JSONExtract functions. E.g. JSONExtract('{"a": "1000.111"}', 'a', 'UInt64') -> 1000, previously it returned 0. #45432 (Anton Popov).
  • Added fields supports_parallel_parsing and supports_parallel_formatting to table system.formats for better introspection. #45499 (Anton Popov).
  • Attempt to improve fsync latency (by syncing all files at once during fetches and small files after mutations) and one tiny fix for fsync_part_directory. #45537 (Azat Khuzhin).

Bug Fix

  • Fix HTTP requests without path for AWS. After updating AWS SDK the sdk no longer adds a slash to requesting paths so we need to do it in our PocoHTTPClient to keep HTTP requests correct. #45238 (Vitaly Baranov).
  • Fix backup if mutations get killed during the backup process. #45351 (Vitaly Baranov).

Build/Testing/Packaging Improvement

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

  • #40651 #41404. #42126 (Alexander Gololobov).
  • Fix possible use-of-unitialized value after executing expressions after sorting. Closes #43386 CC: @nickitat. #43635 (Kruglov Pavel).
  • Better handling of NULL in aggregate combinators, fix possible segfault/logical error while using optimization optimize_rewrite_sum_if_to_count_if. Closes #43758. #43813 (Kruglov Pavel).
  • Fix CREATE USER/ROLE query settings constraints. #43993 (Nikolay Degterinsky).
    • Fix wrong behavior of JOIN ON t1.x = t2.x AND 1 = 1, forbid such queries. #44016 (Vladimir C).
  • Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. #44026 (Yakov Olkhovskiy).
  • Fix parsing of bad version from compatibility setting. #44224 (Kruglov Pavel).
  • Bring interval subtraction from datetime in line with addition. #44241 (ltrk2).
  • Fix double-free in HashTable::clearAndShrink() with zero elements in it. #44256 (Azat Khuzhin).
  • Remove limits on maximum size of the result for view. #44261 (lizhuoyu5).
  • Fix possible logical error in cache if do_not_evict_index_and_mrk_files=1. Closes #42142. #44268 (Kseniia Sumarokova).
  • Fix possible too early cache write interruption in write-through cache (caching could be stopped due to false assumption when it shouldn't have). #44289 (Kseniia Sumarokova).
  • Fix possible crash in case function IN with constant arguments was used as a constant argument together with LowCardinality. Fixes #44221. #44346 (Nikolai Kochetov).
  • Fix support for complex parameters (like arrays) of parametric aggregate functions. This closes #30975. The aggregate function sumMapFiltered was unusable in distributed queries before this change. #44358 (Alexey Milovidov).
    • Fix possible nullptr deference in JoinSwitcher with allow_experimental_analyzer. #44371 (Vladimir C).
  • Fix reading ObjectId in BSON schema inference. #44382 (Kruglov Pavel).
  • Fix race which can lead to premature temp parts removal before merge finished in ReplicatedMergeTree. This issue could lead to errors like No such file or directory: xxx. Fixes #43983. #44383 (alesapin).
  • Some invalid SYSTEM ... ON CLUSTER queries worked in an unexpected way if a cluster name was not specified. It's fixed, now invalid queries throw SYNTAX_ERROR as they should. Fixes #44264. #44387 (Alexander Tokmakov).
  • Fix reading Map type in ORC format. #44400 (Kruglov Pavel).
  • Fix reading columns that are not presented in input data in Parquet/ORC formats. Previously it could lead to error INCORRECT_NUMBER_OF_COLUMNS. Closes #44333. #44405 (Kruglov Pavel).
  • Previously bar() function used the same '▋' (U+258B "Left five eighths block") character to display both 5/8 and 6/8 bars. This change corrects this behavior by using '▊' (U+258A "Left three quarters block") for displaying 6/8 bar. #44410 (Alexander Gololobov).
  • Placing profile settings after profile settings constraints in the configuration file made constraints ineffective. #44411 (Konstantin Bogdanov).
  • Fix SYNTAX_ERROR while running EXPLAIN AST INSERT queries with data. Closes #44207. #44413 (save-my-heart).
  • Fix reading bool value with CRLF in CSV format. Closes #44401. #44442 (Kruglov Pavel).
  • Don't execute and/or/if/multiIf on LowCardinality dictionary, so the result type cannot be LowCardinality. It could lead to error Illegal column ColumnLowCardinality in some cases. Fixes #43603. #44469 (Kruglov Pavel).
  • Fix mutations with setting max_streams_for_merge_tree_reading. #44472 (Anton Popov).
  • Fix potential null pointer dereference with GROUPING SETS in ASTSelectQuery::formatImpl (#43049). #44479 (Robert Schulze).
  • Validate types in table function arguments, CAST function arguments, JSONAsObject schema inference according to settings. #44501 (Kruglov Pavel).
  • Fixed a bug in normalization of a DEFAULT expression in CREATE TABLE statement. The second argument of function in (or the right argument of operator IN) might be replaced with the result of its evaluation during CREATE query execution. Fixes #44496. #44547 (Alexander Tokmakov).
  • Projections do not work in presence of WITH ROLLUP, WITH CUBE and WITH TOTALS. In previous versions, a query produced an exception instead of skipping the usage of projections. This closes #44614. This closes #42772. #44615 (Alexey Milovidov).
    • Fix bug in experimental analyzer and aggregate_functions_null_for_empty = 1. Close #44644. #44648 (Vladimir C).
  • async blocks are not cleaned because the function get all blocks sorted by time didn't get async blocks. #44651 (Han Fei).
  • Fix LOGICAL_ERROR The top step of the right pipeline should be ExpressionStep for JOIN with subquery, UNION, and TOTALS. Fixes #43687. #44673 (Nikolai Kochetov).
  • Avoid std::out_of_range exception in StorageExecutable. #44681 (Kruglov Pavel).
  • Do not apply optimize_syntax_fuse_functions to quantiles on AST, close #44712. #44713 (Vladimir C).
  • Fix bug with wrong type in Merge table and PREWHERE, close #43324. #44716 (Vladimir C).
  • Fix possible crash during shutdown (while destroying TraceCollector). Fixes #44757. #44758 (Nikolai Kochetov).
  • Fix a possible crash in distributed query processing. The crash could happen if a query with totals or extremes returned an empty result and there are mismatched types in the Distrubuted and the local tables. Fixes #44738. #44760 (Nikolai Kochetov).
  • Fix fsync for fetches (min_compressed_bytes_to_fsync_after_fetch)/small files (ttl.txt, columns.txt) in mutations (min_rows_to_fsync_after_merge/min_compressed_bytes_to_fsync_after_merge). #44781 (Azat Khuzhin).
  • A rare race condition was possible when querying the system.parts or system.parts_columns tables in the presence of parts being moved between disks. Introduced in #41145. #44809 (Alexey Milovidov).
  • Fix the error Context has expired which could appear with enabled projections optimization. Can be reproduced for queries with specific functions, like dictHas/dictGet which use context in runtime. Fixes #44844. #44850 (Nikolai Kochetov).
  • Another fix for Cannot read all data error which could happen while reading LowCardinality dictionary from remote fs. Fixes #44709. #44875 (Nikolai Kochetov).
  • Use max_delay_to_insert value in case calculated time to delay INSERT exceeds the setting value. Related to #44902. #44916 (Igor Nikonov).
  • Fix error Different order of columns in UNION subquery for queries with UNION. Fixes #44866. #44920 (Nikolai Kochetov).
  • Delay for INSERT can be calculated incorrectly, which can lead to always using max_delay_to_insert setting as delay instead of a correct value. Using simple formula max_delay_to_insert * (parts_over_threshold/max_allowed_parts_over_threshold) i.e. delay grows proportionally to parts over threshold. Closes #44902. #44954 (Igor Nikonov).
  • fix alter table ttl error when wide part has light weight delete mask. #44959 (Mingliang Pan).
  • Follow-up fix for Replace domain IP types (IPv4, IPv6) with native #43221. #45024 (Yakov Olkhovskiy).
  • Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. #45043 (Yakov Olkhovskiy).
  • A buffer overflow was possible in the parser. Found by fuzzer. #45047 (Alexey Milovidov).
  • Fix possible cannot-read-all-data error in storage FileLog. Closes #45051, #38257. #45057 (Kseniia Sumarokova).
  • Memory efficient aggregation (setting distributed_aggregation_memory_efficient) is disabled when grouping sets are present in the query. #45058 (Nikita Taranov).
  • Fix RANGE_HASHED dictionary to count range columns as part of primary key during updates when update_field is specified. Closes #44588. #45061 (Maksim Kita).
  • Fix error Cannot capture column for LowCardinality captured argument of nested labmda. Fixes #45028. #45065 (Nikolai Kochetov).
  • Fix the wrong query result of additional_table_filters (additional filter was not applied) in case if minmax/count projection is used. #45133 (Nikolai Kochetov).
    • Fixed bug in histogram function accepting negative values. #45147 (simpleton).
  • Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. #45150 (Yakov Olkhovskiy).
  • Fix wrong column nullability in StoreageJoin, close #44940. #45184 (Vladimir C).
  • Fix background_fetches_pool_size settings reload (increase at runtime). #45189 (Raúl Marín).
  • Correctly process SELECT queries on KV engines (e.g. KeeperMap, EmbeddedRocksDB) using IN on the key with subquery producing different type. #45215 (Antonio Andelic).
  • Fix logical error in SEMI JOIN & join_use_nulls in some cases, close #45163, close #45209. #45230 (Vladimir C).
  • Fix heap-use-after-free in reading from s3. #45253 (Kruglov Pavel).
  • Fix bug when the Avro Union type is ['null', Nested type], closes #45275. Fix bug that incorrectly infer bytes type to Float. #45276 (flynn).
  • Throw a correct exception when explicit PREWHERE cannot be used with table using storage engine Merge. #45319 (Antonio Andelic).
  • Under WSL1 Ubuntu self-extracting clickhouse fails to decompress due to inconsistency - /proc/self/maps reporting 32bit file's inode, while stat reporting 64bit inode. #45339 (Yakov Olkhovskiy).
  • Fix race in Distributed table startup (that could lead to processing file of async INSERT multiple times). #45360 (Azat Khuzhin).
  • Fix possible crash while reading from storage S3 and table function s3 in case when ListObject request has failed. #45371 (Anton Popov).
    • Fixed some bugs in JOINS with WHERE by disabling "move to prewhere" optimization for it, close #44062. #45391 (Vladimir C).
  • Fix SELECT ... FROM system.dictionaries exception when there is a dictionary with a bad structure (e.g. incorrect type in xml config). #45399 (Aleksei Filatov).
  • Fix s3Cluster schema inference when structure from insertion table is used in INSERT INTO ... SELECT * FROM s3Cluster queries. #45422 (Kruglov Pavel).
  • Fix bug in JSON/BSONEachRow parsing with HTTP that could lead to using default values for some columns instead of values from data. #45424 (Kruglov Pavel).
  • Fixed bug (Code: 632. DB::Exception: Unexpected data ... after parsed IPv6 value ...) with typed parsing of IP types from text source. #45425 (Yakov Olkhovskiy).
  • close #45297 Add check for empty regular expressions. #45428 (Han Fei).
  • Fix possible (likely distributed) query hung. #45448 (Azat Khuzhin).
  • Fix disabled two-level aggregation from HTTP. #45450 (Nikolai Kochetov).
  • Fix possible deadlock with allow_asynchronous_read_from_io_pool_for_merge_tree enabled in case of exception from ThreadPool::schedule. #45481 (Nikolai Kochetov).
  • Fix possible in-use table after DETACH. #45493 (Azat Khuzhin).
  • Fix rare abort in case when query is canceled and parallel parsing was used during its execution. #45498 (Anton Popov).
  • Fix a race between Distributed table creation and INSERT into it (could lead to CANNOT_LINK during INSERT into the table). #45502 (Azat Khuzhin).
  • Add proper default (SLRU) to cache policy getter. Closes #45514. #45524 (Kseniia Sumarokova).
  • Remove AST-based optimization optimize_fuse_sum_count_avg, close #45439. #45558 (Vladimir C).

Bug-fix

  • Disallow arrayjoin in mutations closes #42637 Implementation: * Added a new parameter to ActionsVisitor::Data disallow_arrayjoin, which is set by MutationsIterator when it appends expression. * ActionsVisitor uses disallow_arrayjoin and throws error when its used with mutations. Testing: * Added test for the same 02504_disallow_arrayjoin_in_mutations.sql. #44447 (SmitaRKulkarni).
  • Fix for qualified asterisks with alias table name and column transformer resolves #44736. #44755 (SmitaRKulkarni).
  • Updated backup/restore status when concurrent backups & restores are not allowed resolves #45486 Implementation: * Moved concurrent backup/restore check inside try-catch block which sets the status so that other nodes in cluster are aware of failures. * Renamed backup_uuid to restore_uuid in RestoreSettings. #45497 (SmitaRKulkarni).

Build Improvement

Feature

  • Record server startup time in ProfileEvents resolves #43188 Implementation: * Added ProfileEvents::ServerStartupMilliseconds. * Recorded time from start of main till listening to sockets. Testing: * Added a test 02532_profileevents_server_startup_time.sql. #45250 (SmitaRKulkarni).

NO CL ENTRY

  • NO CL ENTRY: 'Revert "If user only need virtual columns, we don't need to initialize ReadBufferFromS3"'. #44939 (Anton Popov).
  • NO CL ENTRY: 'Revert "Custom reading for mutation"'. #45121 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Revert "Custom reading for mutation""'. #45122 (Nikolai Kochetov).
  • NO CL ENTRY: 'Revert "update function DAYOFWEEK and add new function WEEKDAY for mysql/spark compatiability"'. #45221 (Alexander Tokmakov).
  • NO CL ENTRY: 'Revert "Validate function arguments in query tree"'. #45299 (Maksim Kita).
  • NO CL ENTRY: 'Revert "Revert "Validate function arguments in query tree""'. #45300 (Maksim Kita).
  • NO CL ENTRY: 'Revert "Support optimize_or_like_chain in QueryTreePassManager"'. #45406 (Anton Popov).
  • NO CL ENTRY: 'Resubmit Support optimize_or_like_chain in QueryTreePassManager'. #45410 (Dmitry Novik).
  • NO CL ENTRY: 'Revert "Remove redundant sorting"'. #45414 (Igor Nikonov).

NOT FOR CHANGELOG / INSIGNIFICANT