mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
94 KiB
94 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.3.1.2672-lts (2c5c589a88
) FIXME as compared to v24.2.1.2248-stable (891689a415
)
Backward Incompatible Change
- Don't allow to set max_parallel_replicas to 0 as it doesn't make sense. Setting it to 0 could lead to unexpected logical errors. Closes #60140. #60430 (Kruglov Pavel).
- Change the column name from
duration_ms
toduration_microseconds
in thesystem.zookeeper
table to reflect the reality that the duration is in the microsecond resolution. #60774 (Duc Canh Le). - Reject incoming INSERT queries in case when query-level settings
async_insert
anddeduplicate_blocks_in_dependent_materialized_views
are enabled together. This behaviour is controlled by a settingthrow_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert
and enabled by default. This is a continuation of https://github.com/ClickHouse/ClickHouse/pull/59699 needed to unblock https://github.com/ClickHouse/ClickHouse/pull/59915. #60888 (Nikita Mikhaylov). - Utility
clickhouse-copier
is moved to a separate repository on GitHub: https://github.com/ClickHouse/copier. It is no longer included in the bundle but is still available as a separate download. This closes: #60734 This closes: #60540 This closes: #60250 This closes: #52917 This closes: #51140 This closes: #47517 This closes: #47189 This closes: #46598 This closes: #40257 This closes: #36504 This closes: #35485 This closes: #33702 This closes: #26702 ### Documentation entry for user-facing changes. #61058 (Nikita Mikhaylov). - To increase compatibility with MySQL, function
locate
now accepts arguments(needle, haystack[, start_pos])
by default. The previous behavior(haystack, needle, [, start_pos])
can be restored by settingfunction_locate_has_mysql_compatible_argument_order = 0
. #61092 (Robert Schulze). - The obsolete in-memory data parts have been deprecated since version 23.5 and have not been supported since version 23.10. Now the remaining code is removed. Continuation of #55186 and #45409. It is unlikely that you have used in-memory data parts because they were available only before version 23.5 and only when you enabled them manually by specifying the corresponding SETTINGS for a MergeTree table. To check if you have in-memory data parts, run the following query:
SELECT part_type, count() FROM system.parts GROUP BY part_type ORDER BY part_type
. To disable the usage of in-memory data parts, doALTER TABLE ... MODIFY SETTING min_bytes_for_compact_part = DEFAULT, min_rows_for_compact_part = DEFAULT
. Before upgrading from old ClickHouse releases, first check that you don't have in-memory data parts. If there are in-memory data parts, disable them first, then wait while there are no in-memory data parts and continue the upgrade. #61127 (Alexey Milovidov). - Forbid
SimpleAggregateFunction
inORDER BY
ofMergeTree
tables (likeAggregateFunction
is forbidden, but they are forbidden because they are not comparable) by default (useallow_suspicious_primary_key
to allow them). #61399 (Azat Khuzhin). - ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. This is controlled by the settings,
output_format_parquet_string_as_string
,output_format_orc_string_as_string
,output_format_arrow_string_as_string
. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases. Parquet/ORC/Arrow supports many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools lack support for the fasterlz4
compression method, that's why we setzstd
by default. This is controlled by the settingsoutput_format_parquet_compression_method
,output_format_orc_compression_method
, andoutput_format_arrow_compression_method
. We changed the default tozstd
for Parquet and ORC, but not Arrow (it is emphasized for low-level usages). #61817 (Alexey Milovidov). - In the new ClickHouse version, the functions
geoDistance
,greatCircleDistance
, andgreatCircleAngle
will use 64-bit double precision floating point data type for internal calculations and return type if all the arguments are Float64. This closes #58476. In previous versions, the function always used Float32. You can switch to the old behavior by settinggeo_distance_returns_float64_on_float64_arguments
tofalse
or settingcompatibility
to24.2
or earlier. #61848 (Alexey Milovidov).
New Feature
- Topk/topkweighed support mode, which return count of values and it's error. #54508 (UnamedRus).
- Add generate_series as a table function. This function generates table with an arithmetic progression with natural numbers. #59390 (divanik).
- Support reading and writing backups as tar archives. #59535 (josh-hildred).
- Implemented support for S3Express buckets. #59965 (Nikita Taranov).
- Allow to attach parts from a different disk * attach partition from the table on other disks using copy instead of hard link (such as instant table) * attach partition using copy when the hard link fails even on the same disk. #60112 (Unalian).
- Added function
toMillisecond
which returns the millisecond component for values of typeDateTime
orDateTime64
. #60281 (Shaun Struwig). - Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. #60420 (豪肥肥).
- Add four properties to the
StorageMemory
(memory-engine)min_bytes_to_keep, max_bytes_to_keep, min_rows_to_keep
andmax_rows_to_keep
- Add tests to reflect new changes - Updatememory.md
documentation - Add tablecontext
property toMemorySink
to enable access to table parameter bounds. #60612 (Jake Bamrah). - Added function
toMillisecond
which returns the millisecond component for values of typeDateTime
orDateTime64
. #60649 (Robert Schulze). - Separate limits on number of waiting and executing queries. Added new server setting
max_waiting_queries
that limits the number of queries waiting due toasync_load_databases
. Existing limits on number of executing queries no longer count waiting queries. #61053 (Sergei Trifonov). - Add support for
ATTACH PARTITION ALL
. #61107 (Kirill Nikiforov). - Add a new function,
getClientHTTPHeader
. This closes #54665. Co-authored with @lingtaolf. #61820 (Alexey Milovidov).
Performance Improvement
- Improve the performance of serialized aggregation method when involving multiple [nullable] columns. This is a general version of #51399 that doesn't compromise on abstraction integrity. #55809 (Amos Bird).
- Lazy build join output to improve performance of ALL join. #58278 (LiuNeng).
- Improvements to aggregate functions ArgMin / ArgMax / any / anyLast / anyHeavy, as well as
ORDER BY {u8/u16/u32/u64/i8/i16/u32/i64) LIMIT 1
queries. #58640 (Raúl Marín). - Trivial optimize on column filter. Avoid those filter columns whoes underlying data type is not number being filtered with
result_size_hint = -1
. Peak memory can be reduced to 44% of the original in some cases. #59698 (李扬). - If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting
primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns
with the value0.9
by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. #60255 (Alexey Milovidov). - Execute multiIf function columnarly when result_type's underlying type is number. #60384 (李扬).
- Faster (almost 2x) mutexes (was slower due to ThreadFuzzer). #60823 (Azat Khuzhin).
- Move connection drain from prepare to work, and drain multiple connections in parallel. #60845 (lizhuoyu5).
- Optimize insertManyFrom of nullable number or nullable string. #60846 (李扬).
- Optimized function
dotProduct
to omit unnecessary and expensive memory copies. #60928 (Robert Schulze). - Operations with the filesystem cache will suffer less from the lock contention. #61066 (Alexey Milovidov).
- Optimize ColumnString::replicate and prevent memcpySmallAllowReadWriteOverflow15Impl from being optimized to built-in memcpy. Close #61074. ColumnString::replicate speeds up by 2.46x on x86-64. #61075 (李扬).
- 30x faster printing for 256-bit integers. #61100 (Raúl Marín).
- If a query with a syntax error contained COLUMNS matcher with a regular expression, the regular expression was compiled each time during the parser's backtracking, instead of being compiled once. This was a fundamental error. The compiled regexp was put to AST. But the letter A in AST means "abstract" which means it should not contain heavyweight objects. Parts of AST can be created and discarded during parsing, including a large number of backtracking. This leads to slowness on the parsing side and consequently allows DoS by a readonly user. But the main problem is that it prevents progress in fuzzers. #61543 (Alexey Milovidov).
- Add a new analyzer pass to optimize in single value. #61564 (LiuNeng).
Improvement
- While running the MODIFY COLUMN query for materialized views, check the inner table's structure to ensure every column exists. #47427 (sunny).
- Added table
system.keywords
which contains all the keywords from parser. Mostly needed and will be used for better fuzzing and syntax highlighting. #51808 (Nikita Mikhaylov). - Ordinary database engine is deprecated. You will receive a warning in clickhouse-client if your server is using it. This closes #52229. #56942 (shabroo).
- All zero copy locks related to a table have to be dropped when the table is dropped. The directory which contains these locks has to be removed also. #57575 (Sema Checherinda).
- Allow declaring enum in external table structure. #57857 (Duc Canh Le).
- Consider lightweight deleted rows when selecting parts to merge. #58223 (Zhuo Qiu).
- This PR makes http/https connections reusable for all uses cases. Even when response is 3xx or 4xx. #58845 (Sema Checherinda).
- Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. #59016 (Nikita Mikhaylov).
- Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like
_part_offset
. #59033 (Amos Bird). - Add ability to skip read-only replicas for INSERT into Distributed engine (Controlled with
distributed_insert_skip_read_only_replicas
setting, by default OFF - backward compatible). #59176 (Azat Khuzhin). - Instead using a constant key, now object storage generates key for determining remove objects capability. #59495 (Sema Checherinda).
- Add positional pread in libhdfs3. If you want to call positional read in libhdfs3, use the hdfsPread function in hdfs.h as follows.
tSize hdfsPread(hdfsFS fs, hdfsFile file, void * buffer, tSize length, tOffset position);
. #59624 (M1eyu). - Add asynchronous WriteBuffer for AzureBlobStorage similar to S3. #59929 (SmitaRKulkarni).
- Allow "local" as object storage type instead of "local_blob_storage". #60165 (Kseniia Sumarokova).
- Improved overall usability of virtual columns. Now it is allowed to use virtual columns in
PREWHERE
(it's worthwhile for non-const virtual columns like_part_offset
). Now a builtin documentation is available for virtual columns as a comment of column inDESCRIBE
query with enabled settingdescribe_include_virtual_columns
. #60205 (Anton Popov). - Parallel flush of pending INSERT blocks of Distributed engine on
DETACH
/server shutdown andSYSTEM FLUSH DISTRIBUTED
(Parallelism will work only if you have multi disk policy for table (like everything in Distributed engine right now)). #60225 (Azat Khuzhin). - Filter setting is improper in
joinRightColumnsSwitchNullability
, resolve #59625. #60259 (lgbo). - Add a setting to force read-through cache for merges. #60308 (Kseniia Sumarokova).
- Issue #57598 mentions a variant behaviour regarding transaction handling. An issued COMMIT/ROLLBACK when no transaction is active is reported as an error contrary to MySQL behaviour. #60338 (PapaToemmsn).
- Added
none_only_active
mode fordistributed_ddl_output_mode
setting. #60340 (Alexander Tokmakov). - Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make
/
redirect to the Play UI. #60390 (Alexey Milovidov). - The advanced dashboard has slightly better colors for multi-line graphs. #60391 (Alexey Milovidov).
- Function
substring
now has a new aliasbyteSlice
. #60494 (Robert Schulze). - Renamed server setting
dns_cache_max_size
todns_cache_max_entries
to reduce ambiguity. #60500 (Kirill Nikiforov). SHOW INDEX | INDEXES | INDICES | KEYS
no longer sorts by the primary key columns (which was unintuitive). #60514 (Robert Schulze).- Keeper improvement: abort during startup if an invalid snapshot is detected to avoid data loss. #60537 (Antonio Andelic).
- Added MergeTree read split ranges into intersecting and non intersecting fault injection using
merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_fault_probability
setting. #60548 (Maksim Kita). - The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. #60692 (Alexey Milovidov).
- String types and Enums can be used in the same context, such as: arrays, UNION queries, conditional expressions. This closes #60726. #60727 (Alexey Milovidov).
- Update tzdata to 2024a. #60768 (Raúl Marín).
- Support files without format extension in Filesystem database. #60795 (Kruglov Pavel).
- Keeper improvement: support
leadership_expiry_ms
in Keeper's settings. #60806 (Brokenice0415). - Always infer exponential numbers in JSON formats regardless of the setting
input_format_try_infer_exponent_floats
. Add settinginput_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects
that allows to use String type for ambiguous paths instead of an exception during named Tuples inference from JSON objects. #60808 (Kruglov Pavel). - Add support for
START TRANSACTION
syntax typically used in MySQL syntax, resolving https://github.com/ClickHouse/ClickHouse/discussions/60865. #60886 (Zach Naimon). - Add a flag for SMJ to treat null as biggest/smallest. So the behavior can be compitable with other SQL systems, like Apache Spark. #60896 (loudongfeng).
- Clickhouse version has been added to docker labels. Closes #54224. #60949 (Nikolay Monkov).
- Add a setting
parallel_replicas_allow_in_with_subquery = 1
which allows subqueries for IN work with parallel replicas. #60950 (Nikolai Kochetov). - DNSResolver shuffles set of resolved IPs. #60965 (Sema Checherinda).
- Support detect output format by file exctension in
clickhouse-client
andclickhouse-local
. #61036 (豪肥肥). - Check memory limit update periodically. #61049 (Han Fei).
- Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. #61096 (Azat Khuzhin).
- Add the function
toUInt128OrZero
, which was missed by mistake (the mistake is related to https://github.com/ClickHouse/ClickHouse/pull/945). The compatibility aliasesFROM_UNIXTIME
andDATE_FORMAT
(they are not ClickHouse-native and only exist for MySQL compatibility) have been made case insensitive, as expected for SQL-compatibility aliases. #61114 (Alexey Milovidov). - Improvements for the access checks, allowing to revoke of unpossessed rights in case the target user doesn't have the revoking grants either. Example: ```sql GRANT SELECT ON . TO user1; REVOKE SELECT ON system.* FROM user1;. #61115 (pufit).
- Fix an error in previeous opt: https://github.com/ClickHouse/ClickHouse/pull/59698: remove break to make sure the first filtered column has minimum size cc @jsc0218. #61145 (李扬).
- Fix
has()
function withNullable
column (fixes #60214). #61249 (Mikhail Koviazin). - Now it's possible to specify attribute
merge="true"
in config substitutions for subtrees<include from_zk="/path" merge="true">
. In case this attribute specified, clickhouse will merge subtree with existing configuration, otherwise default behavior is append new content to configuration. #61299 (alesapin). - Add async metrics for virtual memory mappings: VMMaxMapCount & VMNumMaps. Closes #60662. #61354 (Tuan Pham Anh).
- Use
temporary_files_codec
setting in all places where we create temporary data, for example external memory sorting and external memory GROUP BY. Before it worked only inpartial_merge
JOIN algorithm. #61456 (Maksim Kita). - Remove duplicated check
containing_part.empty()
, It's already being checked here:1296dac3c7/src/Storages/MergeTree/MergeTreeData.cpp (L6141)
. #61467 (William Schoeffel). - Add a new setting
max_parser_backtracks
which allows to limit the complexity of query parsing. #61502 (Alexey Milovidov). - Support parallel reading for azure blob storage. #61503 (SmitaRKulkarni).
- Less contention during dynamic resize of filesystem cache. #61524 (Kseniia Sumarokova).
- Disallow sharded mode of StorageS3 queue, because it will be rewritten. #61537 (Kseniia Sumarokova).
- Fixed typo: from
use_leagcy_max_level
touse_legacy_max_level
. #61545 (William Schoeffel). - Remove some duplicate entries in blob_storage_log. #61622 (YenchangChan).
- Enable
allow_experimental_analyzer
setting by default. #61652 (Dmitry Novik). - Added
current_user
function as a compatibility alias for MySQL. #61770 (Yarik Briukhovetskyi). - Use managed identity for backups IO when using Azure Blob Storage. Add a setting to prevent ClickHouse from attempting to create a non-existent container, which requires permissions at the storage account level. #61785 (Daniel Pozo Escalona).
- Enable
output_format_pretty_row_numbers
by default. It is better for usability. #61791 (Alexey Milovidov). - In the previous version, some numbers in Pretty formats were not pretty enough. #61794 (Alexey Milovidov).
- A long value in Pretty formats won't be cut if it is the single value in the resultset, such as in the result of the
SHOW CREATE TABLE
query. #61795 (Alexey Milovidov). - Similarly to
clickhouse-local
,clickhouse-client
will accept the--output-format
option as a synonym to the--format
option. This closes #59848. #61797 (Alexey Milovidov). - If stdout is a terminal and the output format is not specified,
clickhouse-client
and similar tools will usePrettyCompact
by default, similarly to the interactive mode.clickhouse-client
andclickhouse-local
will handle command line arguments for input and output formats in a unified fashion. This closes #61272. #61800 (Alexey Milovidov). - Underscore digit groups in Pretty formats for better readability. This is controlled by a new setting,
output_format_pretty_highlight_digit_groups
. #61802 (Alexey Milovidov). - Add ability to override initial INSERT SETTINGS via SYSTEM FLUSH DISTRIBUTED. #61832 (Azat Khuzhin).
- Fixed grammar from "a" to "the" in the warning message. There is only one Atomic engine, so it should be "to the new Atomic engine" instead of "to a new Atomic engine". #61952 (shabroo).
Build/Testing/Packaging Improvement
- Update sccache to the latest version; significantly reduce images size by reshaking the dependency trees; use the latest working odbc driver. #59953 (Mikhail f. Shiryaev).
- Update python related style checkers. Continue the #50174. #60408 (Mikhail f. Shiryaev).
- Upgrade
prqlc
to 0.11.3. #60616 (Maximilian Roos). - Attach gdb to running fuzzer process. #60654 (Nikita Mikhaylov).
- Use explicit template instantiation more aggressively. Get rid of templates in favor of overloaded functions in some places. #60730 (Nikita Mikhaylov).
- The real-time query profiler now works on AArch64. In previous versions, it worked only when a program didn't spend time inside a syscall. #60807 (Alexey Milovidov).
- ... Too big translation unit in
Aggregator
. #61211 (lgbo). - Fixed flakiness of 01603_insert_select_too_many_parts test. Closes #61158. #61259 (Ilya Yatsishin).
- Now it possible to use
chassert(expression, comment)
in the codebase. #61263 (Nikita Mikhaylov). - Teach the fuzzer to use other numeric types. #61317 (Raúl Marín).
- Increase memory limit for coverage builds. #61405 (Raúl Marín).
- Add generic query text fuzzer in
clickhouse-local
. #61508 (Alexey Milovidov).
Bug Fix (user-visible misbehavior in an official stable release)
- Fix function execution over const and LowCardinality with GROUP BY const for analyzer #59986 (Azat Khuzhin).
- Fix finished_mutations_to_keep=0 for MergeTree (as docs says 0 is to keep everything) #60031 (Azat Khuzhin).
- PartsSplitter invalid ranges for the same part #60041 (Maksim Kita).
- Azure Blob Storage : Fix issues endpoint and prefix #60251 (SmitaRKulkarni).
- fix LRUResource Cache bug (Hive cache) #60262 (shanfengp).
- Force reanalysis if parallel replicas changed #60362 (Raúl Marín).
- Fix usage of plain metadata type with new disks configuration option #60396 (Kseniia Sumarokova).
- Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike #60451 (Kruglov Pavel).
- Try to avoid calculation of scalar subqueries for CREATE TABLE. #60464 (Nikolai Kochetov).
- Fix deadlock in parallel parsing when lots of rows are skipped due to errors #60516 (Kruglov Pavel).
- Fix_max_query_size_for_kql_compound_operator: #60534 (Yong Wang).
- Keeper fix: add timeouts when waiting for commit logs #60544 (Antonio Andelic).
- Reduce the number of read rows from
system.numbers
#60546 (JackyWoo). - Don't output number tips for date types #60577 (Raúl Marín).
- Fix reading from MergeTree with non-deterministic functions in filter #60586 (Kruglov Pavel).
- Fix logical error on bad compatibility setting value type #60596 (Kruglov Pavel).
- Fix inconsistent aggregate function states in mixed x86-64 / ARM clusters #60610 (Harry Lee).
- fix(prql): Robust panic handler #60615 (Maximilian Roos).
- Fix
intDiv
for decimal and date arguments #60672 (Yarik Briukhovetskyi). - Fix: expand CTE in alter modify query #60682 (Yakov Olkhovskiy).
- Fix system.parts for non-Atomic/Ordinary database engine (i.e. Memory) #60689 (Azat Khuzhin).
- Fix "Invalid storage definition in metadata file" for parameterized views #60708 (Azat Khuzhin).
- Fix buffer overflow in CompressionCodecMultiple #60731 (Alexey Milovidov).
- Remove nonsense from SQL/JSON #60738 (Alexey Milovidov).
- Remove wrong sanitize checking in aggregate function quantileGK #60740 (李扬).
- Fix insert-select + insert_deduplication_token bug by setting streams to 1 #60745 (Jordi Villar).
- Prevent setting custom metadata headers on unsupported multipart upload operations #60748 (Francisco J. Jurado Moreno).
- Fix toStartOfInterval #60763 (Andrey Zvonov).
- Fix crash in arrayEnumerateRanked #60764 (Raúl Marín).
- Fix crash when using input() in INSERT SELECT JOIN #60765 (Kruglov Pavel).
- Fix crash with different allow_experimental_analyzer value in subqueries #60770 (Dmitry Novik).
- Remove recursion when reading from S3 #60849 (Antonio Andelic).
- Fix possible stuck on error in HashedDictionaryParallelLoader #60926 (vdimir).
- Fix async RESTORE with Replicated database #60934 (Antonio Andelic).
- fix csv format not support tuple #60994 (shuai.xu).
- Fix deadlock in async inserts to
Log
tables via native protocol #61055 (Anton Popov). - Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary #61196 (Kruglov Pavel).
- Fix multiple bugs in groupArraySorted #61203 (Raúl Marín).
- Fix Keeper reconfig for standalone binary #61233 (Antonio Andelic).
- Fix usage of session_token in S3 engine #61234 (Kruglov Pavel).
- Fix possible incorrect result of aggregate function
uniqExact
#61257 (Anton Popov). - Fix bugs in show database #61269 (Raúl Marín).
- Fix logical error in RabbitMQ storage with MATERIALIZED columns #61320 (vdimir).
- Fix CREATE OR REPLACE DICTIONARY #61356 (Vitaly Baranov).
- Fix crash in ObjectJson parsing array with nulls #61364 (vdimir).
- Fix ATTACH query with external ON CLUSTER #61365 (Nikolay Degterinsky).
- Fix consecutive keys optimization for nullable keys #61393 (Anton Popov).
- fix issue of actions dag split #61458 (Raúl Marín).
- Fix finishing a failed RESTORE #61466 (Vitaly Baranov).
- Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings #61468 (Raúl Marín).
- Allow queuing in restore pool #61475 (Nikita Taranov).
- Fix bug when reading system.parts using UUID (issue 61220). #61479 (Dan Wu).
- Fix ALTER QUERY MODIFY SQL SECURITY #61480 (pufit).
- Fix crash in window view #61526 (Alexey Milovidov).
- Fix
repeat
with non native integers #61527 (Antonio Andelic). - Fix client
-s
argument #61530 (Mikhail f. Shiryaev). - Reset part level upon attach from disk on MergeTree #61536 (Arthur Passos).
- Fix crash in arrayPartialReverseSort #61539 (Raúl Marín).
- Fix string search with const position #61547 (Antonio Andelic).
- Fix addDays cause an error when used datetime64 #61561 (Shuai li).
- disallow LowCardinality input type for JSONExtract #61617 (Julia Kartseva).
- Fix
system.part_log
for async insert with deduplication #61620 (Antonio Andelic). - Fix Non-ready set for system.parts. #61666 (Nikolai Kochetov).
- Don't allow the same expression in ORDER BY with and without WITH FILL #61667 (Kruglov Pavel).
- Fix actual_part_name for REPLACE_RANGE (
Entry actual part isn't empty yet
) #61675 (Alexander Tokmakov). - Fix columns after executing MODIFY QUERY for a materialized view with internal table #61734 (Vitaly Baranov).
- Fix crash in
multiSearchAllPositionsCaseInsensitiveUTF8
for incorrect UTF-8 #61749 (pufit). - Fix RANGE frame is not supported for Nullable columns. #61766 (YuanLiu).
- Revert "Revert "Fix bug when reading system.parts using UUID (issue 61220)."" #61779 (János Benjamin Antal).
CI Fix or Improvement (changelog entry is not required)
- Decoupled changes from #60408. #60553 (Mikhail f. Shiryaev).
- Eliminates the need to provide input args to docker server jobs to clean yml files. #60602 (Max K.).
- Debug and fix markreleaseready. #60611 (Mikhail f. Shiryaev).
- Fix build_report job so that it's defined by ci_config only (not yml file). #60613 (Max K.).
- Do not await ci pending jobs on release branches decrease wait timeout to fit into gh job timeout. #60652 (Max K.).
- Set limited number of builds for "special build check" report in backports. #60850 (Max K.).
- ... #60935 (Max K.).
- ... #60947 (Max K.).
- ... #60952 (Max K.).
- ... #60958 (Max K.).
- ... #61022 (Max K.).
- Just a preparation for the merge queue support. #61099 (Max K.).
- ... #61133 (Max K.).
- In PRs: - run typos, aspell check - always - run pylint, mypy - only if py file(s) changed in PRs - run basic source files style check - only if not all changes in py files. #61148 (Max K.).
- ... #61172 (Max K.).
- ... #61183 (Han Fei).
- ... #61185 (Max K.).
- TBD. #61197 (Mikhail f. Shiryaev).
- ... #61214 (Max K.).
- ... #61441 (Max K.).
- . #61801 (Alexey Milovidov).
- ... #61877 (Max K.).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Revert "Use
MergeTree
as a default table engine""'. #60524 (Alexey Milovidov). - NO CL ENTRY: 'Revert "Revert "Support resource request canceling""'. #60558 (Sergei Trifonov).
- NO CL ENTRY: 'Revert "Add
toMillisecond
function"'. #60644 (Alexander Tokmakov). - NO CL ENTRY: 'Revert "Synchronize parsers"'. #60759 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "Fix wacky primary key sorting in
SHOW INDEX
"'. #60898 (Antonio Andelic). - NO CL ENTRY: 'Revert "CI: make style check faster"'. #61142 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Don't allow to set max_parallel_replicas to 0 as it doesn't make sense"'. #61200 (Kruglov Pavel).
- NO CL ENTRY: 'Revert "Fix usage of session_token in S3 engine"'. #61359 (Antonio Andelic).
- NO CL ENTRY: 'Revert "Revert "Fix usage of session_token in S3 engine""'. #61362 (Kruglov Pavel).
- NO CL ENTRY: 'Reorder hidden and shown checks in comment, change url of Mergeable check'. #61373 (Mikhail f. Shiryaev).
- NO CL ENTRY: 'Remove unnecessary layers from clickhouse/cctools'. #61374 (Mikhail f. Shiryaev).
- NO CL ENTRY: 'Revert "Updated format settings references in the docs (datetime.md)"'. #61435 (Kruglov Pavel).
- NO CL ENTRY: 'Revert "CI: ARM integration tests: disable tests with HDFS "'. #61449 (Max K.).
- NO CL ENTRY: 'Revert "Analyzer: Fix virtual columns in StorageMerge"'. #61518 (Antonio Andelic).
- NO CL ENTRY: 'Revert "Revert "Analyzer: Fix virtual columns in StorageMerge""'. #61528 (Dmitry Novik).
- NO CL ENTRY: 'Improve build_download_helper'. #61592 (Mikhail f. Shiryaev).
- NO CL ENTRY: 'Revert "Un-flake
test_undrop_query
"'. #61668 (Robert Schulze). - NO CL ENTRY: 'Fix flaky tests (stateless, integration)'. #61816 (Nikita Fomichev).
- NO CL ENTRY: 'Better usability of "expect" tests: less trouble with running directly'. #61818 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Fix flaky
02122_parallel_formatting_Template
"'. #61868 (Alexander Tokmakov). - NO CL ENTRY: 'Revert "Add --now option to enable and start the service" #job_Install_packages_amd64'. #61878 (Max K.).
- NO CL ENTRY: 'Revert "disallow LowCardinality input type for JSONExtract"'. #61960 (Nikita Mikhaylov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Improve query performance in case of very small blocks #58879 (Azat Khuzhin).
- Analyzer: fixes for JOIN columns resolution #59007 (vdimir).
- Fix race on
Context::async_insert_queue
#59082 (Alexander Tokmakov). - CI: support batch specification in commit message #59738 (Max K.).
- Update storing-data.md #60024 (Kseniia Sumarokova).
- Make max_insert_delayed_streams_for_parallel_write actually work #60079 (alesapin).
- Analyzer: support join using column from select list #60182 (vdimir).
- test for #60223 #60258 (Denny Crane).
- Analyzer: Refactor execution name for ConstantNode #60313 (Dmitry Novik).
- Fix database iterator waiting code #60314 (Sergei Trifonov).
- QueryCache: Don't acquire the query count mutex if not necessary #60348 (zhongyuankai).
- Fix bugfix check (due to unknown commit_logs_cache_size_threshold) #60375 (Azat Khuzhin).
- Enable testing with
io_uring
back #60383 (Nikita Taranov). - Analyzer - improve hiding secret arguments. #60386 (Yakov Olkhovskiy).
- CI: make workflow yml abstract #60421 (Max K.).
- Improve test test_reload_clusters_config #60426 (Kruglov Pavel).
- Revert "Revert "Merge pull request #56864 from ClickHouse/broken-projections-better-handling"" #60452 (Kseniia Sumarokova).
- Do not check to and from files existence in metadata_storage because it does not see uncommitted changes #60462 (Alexander Gololobov).
- Fix option ambiguous in
clickhouse-local
#60475 (豪肥肥). - Fix: test_parallel_replicas_custom_key_load_balancing #60485 (Igor Nikonov).
- Fix: progress bar for *Cluster table functions #60491 (Igor Nikonov).
- Analyzer: Support different ObjectJSON on shards #60497 (Dmitry Novik).
- Cancel PipelineExecutor properly in case of exception in spawnThreads #60499 (Kruglov Pavel).
- Refactor StorageSystemOneBlock #60510 (Nikolai Kochetov).
- Simple cleanup while fixing progress bar #60513 (Igor Nikonov).
- PullingAsyncPipelineExecutor cleanup #60515 (Igor Nikonov).
- Fix bad error message #60518 (Alexey Milovidov).
- Synchronize Access #60519 (Alexey Milovidov).
- Synchronize metrics and Keeper #60520 (Alexey Milovidov).
- Enforce clang-tidy in
programs/
andutils/
headers #60521 (Robert Schulze). - Synchronize parsers #60522 (Alexey Milovidov).
- Fix a bunch of clang-tidy warnings in headers #60523 (Robert Schulze).
- General sanity in function
seriesOutliersDetectTukey
#60535 (Alexey Milovidov). - Update Chinese document for max_query_size, max_parser_depth and optimize_functions_to_subcolumns #60541 (Alex Cheng).
- Userspace page cache again #60552 (Michael Kolupaev).
- Traverse shadow directory for system.remote_data_paths #60585 (Aleksei Filatov).
- Add test for #58906 #60597 (Raúl Marín).
- Use python zipfile to have x-platform idempotent lambda packages #60603 (Mikhail f. Shiryaev).
- tests: suppress data-race in librdkafka statistics code #60604 (Azat Khuzhin).
- Update version after release #60605 (Alexey Milovidov).
- Update version_date.tsv and changelogs after v24.2.1.2248-stable #60607 (robot-clickhouse).
- Addition to changelog #60609 (Anton Popov).
- internal: Refine rust prql code #60617 (Maximilian Roos).
- fix(rust): Fix skim's panic handler #60621 (Maximilian Roos).
- Resubmit "Analyzer: compute ALIAS columns right after reading" #60641 (vdimir).
- Analyzer: Fix bug with join_use_nulls and PREWHERE #60655 (vdimir).
- Add test for #59891 #60657 (Raúl Marín).
- Fix missed entries in system.part_log in case of fetch preferred over merges/mutations #60659 (Azat Khuzhin).
- Always apply first minmax index among available skip indices #60675 (Igor Nikonov).
- Remove bad test
02152_http_external_tables_memory_tracking
#60690 (Alexey Milovidov). - Fix questionable behavior in the
parseDateTimeBestEffort
function. #60691 (Alexey Milovidov). - Fix flaky checks #60694 (Azat Khuzhin).
- Resubmit http_external_tables_memory_tracking test #60695 (Azat Khuzhin).
- Fix bugfix and upgrade checks (due to "Unknown handler type 'redirect'" error) #60696 (Azat Khuzhin).
- Fix test_grant_and_revoke/test.py::test_grant_all_on_table (after syncing with cloud) #60699 (Azat Khuzhin).
- Remove unit test for ColumnObject #60709 (Alexey Milovidov).
- Improve unit tests #60710 (Alexey Milovidov).
- Fix scheduler fairness test #60712 (Sergei Trifonov).
- Do not retry queries if container is down in integration tests (resubmit) #60714 (Azat Khuzhin).
- Mark one setting as obsolete #60715 (Alexey Milovidov).
- Fix a test with Analyzer #60723 (Alexey Milovidov).
- Two tests are fixed with Analyzer #60724 (Alexey Milovidov).
- Remove old code #60728 (Alexey Milovidov).
- Remove more code from LIVE VIEW #60729 (Alexey Milovidov).
- Fix
test_keeper_back_to_back/test.py::test_concurrent_watches
#60749 (Antonio Andelic). - Catch exceptions on finalize in
InterserverIOHTTPHandler
#60769 (Antonio Andelic). - Reduce flakiness of 02932_refreshable_materialized_views #60771 (Michael Kolupaev).
- Use 64-bit capabilities if available #60775 (Azat Khuzhin).
- Include multiline logs in fuzzer fatal.log report #60796 (Raúl Marín).
- Add missing clone calls related to compression #60810 (Raúl Marín).
- New private runners #60811 (Mikhail f. Shiryaev).
- Move userspace page cache settings to the correct section of SettingsChangeHistory.h #60812 (Michael Kolupaev).
- Update version_date.tsv and changelogs after v23.8.10.43-lts #60851 (robot-clickhouse).
- Fix fuzzer report #60853 (Raúl Marín).
- Update version_date.tsv and changelogs after v23.3.20.27-lts #60857 (robot-clickhouse).
- Refactor OptimizeDateOrDateTimeConverterWithPreimageVisitor #60875 (Zhiguo Zhou).
- Fix race in PageCache #60878 (Michael Kolupaev).
- Small changes in async inserts code #60885 (Nikita Taranov).
- Remove useless verbose logging from AWS library #60921 (alesapin).
- Throw on query timeout in ZooKeeperRetries #60922 (Antonio Andelic).
- Bring clickhouse-test changes from private #60924 (Raúl Marín).
- Add debug info to exceptions in
IMergeTreeDataPart::checkConsistency()
#60981 (Nikita Taranov). - Fix a typo #60987 (Alexey Milovidov).
- Replace some header includes with forward declarations #61003 (Amos Bird).
- Speed up cctools building #61011 (Mikhail f. Shiryaev).
- Fix ASTRenameQuery::clone #61013 (vdimir).
- Update README.md #61021 (Tyler Hannan).
- Fix TableFunctionExecutable::skipAnalysisForArguments #61037 (Dmitry Novik).
- Fix: parallel replicas with PREWHERE (ubsan) #61052 (Igor Nikonov).
- Fast fix tests. #61056 (Nikolai Kochetov).
- Fix
test_placement_info
#61057 (Konstantin Bogdanov). - Fix: parallel replicas with CTEs, crash in EXPLAIN SYNTAX with analyzer #61059 (Igor Nikonov).
- Debug fuzzer failures #61062 (Nikita Mikhaylov).
- Add regression tests for fixed issues #61076 (Antonio Andelic).
- Analyzer: Fix 01244_optimize_distributed_group_by_sharding_key #61089 (Dmitry Novik).
- Use global scalars cache with analyzer #61104 (Nikolai Kochetov).
- Fix removing is_active node after re-creation #61105 (Alexander Tokmakov).
- Update 02962_system_sync_replica_lightweight_from_modifier.sh #61110 (Alexander Tokmakov).
- Simplify bridges #61118 (Alexey Milovidov).
- update cppkafka to v0.4.1 #61119 (Ilya Golshtein).
- CI: add wf class in ci_config #61122 (Max K.).
- QueryFuzzer: replace element randomly when AST part buffer is full #61124 (Tomer Shafir).
- CI: make style check fast #61125 (Max K.).
- Better gitignore #61128 (Alexey Milovidov).
- Fix something strange #61129 (Alexey Milovidov).
- Update check-large-objects.sh to be language neutral #61130 (Dan Wu).
- Throw memory limit exceptions to avoid OOM in some places #61132 (alesapin).
- Fix test_distributed_directory_monitor_split_batch_on_failure flakienss #61136 (Azat Khuzhin).
- Fix llvm symbolizer on CI #61147 (Azat Khuzhin).
- Some clang-tidy fixes #61150 (Robert Schulze).
- Revive "Less contention in the cache, part 2" #61152 (Kseniia Sumarokova).
- Enable black back #61159 (Mikhail f. Shiryaev).
- CI: fix nightly job issue #61160 (Max K.).
- Split
RangeHashedDictionary
#61162 (János Benjamin Antal). - Remove a few templates from Aggregator.cpp #61171 (Raúl Marín).
- Avoid some logical errors in experimantal Object type #61173 (Kruglov Pavel).
- Update ReadSettings.h #61174 (Kseniia Sumarokova).
- CI: ARM integration tests: disable tests with HDFS #61182 (Max K.).
- Disable sanitizers with 02784_parallel_replicas_automatic_decision_join #61184 (Raúl Marín).
- Fix
02887_mutations_subcolumns
test flakiness #61198 (Nikita Taranov). - Make variant tests a bit faster #61199 (Kruglov Pavel).
- Fix strange log message #61206 (Alexey Milovidov).
- Fix 01603_insert_select_too_many_parts flakiness #61218 (Azat Khuzhin).
- Make every style-checker runner types scaling-out very quickly #61231 (Mikhail f. Shiryaev).
- Improve
test_failed_mutations
#61235 (János Benjamin Antal). - Fix
test_merge_tree_load_parts/test.py::test_merge_tree_load_parts_corrupted
#61236 (János Benjamin Antal). - fix
forget_partition
test #61237 (Sergei Trifonov). - Print more info in
02572_system_logs_materialized_views_ignore_errors
to debug #61246 (János Benjamin Antal). - Fix runtime error in AST Fuzzer #61248 (Yarik Briukhovetskyi).
- Add retries to
02908_many_requests_to_system_replicas
#61253 (Nikita Taranov). - Followup fix ASTRenameQuery::clone #61254 (vdimir).
- Disable test 02998_primary_key_skip_columns.sql in sanitizer builds as it can be slow #61256 (Kruglov Pavel).
- Update curl to curl with data race fix #61264 (SmitaRKulkarni).
- Fix
01417_freeze_partition_verbose
#61266 (Kseniia Sumarokova). - Free memory earlier in inserts #61267 (Anton Popov).
- Fixing test_build_sets_from_multiple_threads/test.py::test_set #61286 (Nikolai Kochetov).
- Analyzer: Fix virtual columns in StorageMerge #61298 (Dmitry Novik).
- Fix 01952_optimize_distributed_group_by_sharding_key with analyzer. #61301 (Nikolai Kochetov).
- fix data race in poco tcp server #61309 (Sema Checherinda).
- Don't use default cluster in test test_distibuted_settings #61314 (Kruglov Pavel).
- Fix false positive assertion in cache #61319 (Kseniia Sumarokova).
- Fix test test_input_format_parallel_parsing_memory_tracking #61322 (Kruglov Pavel).
- Fix 01761_cast_to_enum_nullable with analyzer. #61323 (Nikolai Kochetov).
- Add zookeeper retries for exists check in forcefullyRemoveBrokenOutdatedPartFromZooKeeper #61324 (Kseniia Sumarokova).
- Minor changes in stress and fuzzer reports #61333 (Raúl Marín).
- Un-flake
test_undrop_query
#61348 (Robert Schulze). - Tiny improvement for replication.lib #61361 (alesapin).
- Fix bugfix check (due to "unknown object storage type: azure") #61363 (Azat Khuzhin).
- Fix
01599_multiline_input_and_singleline_comments
3 minute wait #61371 (Sergei Trifonov). - Terminate EC2 on spot event if runner isn't running #61377 (Mikhail f. Shiryaev).
- Try fix docs check #61378 (Kseniia Sumarokova).
- Fix
heap-use-after-free
for Merge table with alias #61380 (Antonio Andelic). - Disable
optimize_rewrite_sum_if_to_count_if
if return type is nullable (new analyzer) #61389 (Antonio Andelic). - Analyzer: Fix planner context for subquery in StorageMerge #61392 (Dmitry Novik).
- Fix
test_failed_async_inserts
#61394 (Nikolay Degterinsky). - Fix test test_system_clusters_actual_information flakiness #61395 (Kruglov Pavel).
- Remove default cluster from default config from test config #61396 (Raúl Marín).
- Enable clang-tidy in headers #61406 (Robert Schulze).
- Add sanity check for poll_max_batch_size FileLog setting #61408 (Kruglov Pavel).
- ThreadFuzzer: randomize sleep time #61410 (Tomer Shafir).
- Update version_date.tsv and changelogs after v23.8.11.28-lts #61416 (robot-clickhouse).
- Update version_date.tsv and changelogs after v23.3.21.26-lts #61418 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.1.7.18-stable #61419 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.2.2.71-stable #61420 (robot-clickhouse).
- Update version_date.tsv and changelogs after v23.12.5.81-stable #61421 (robot-clickhouse).
- Restore automerge for approved PRs #61433 (Mikhail f. Shiryaev).
- Disable broken SonarCloud #61434 (Mikhail f. Shiryaev).
- Fix
01599_multiline_input_and_singleline_comments
properly #61440 (Sergei Trifonov). - Convert test 02998_system_dns_cache_table to smoke and mirrors #61443 (vdimir).
- Check boundaries for some settings in parallel replicas #61455 (Nikita Mikhaylov).
- Use SHARD_LOAD_QUEUE_BACKLOG for dictionaries in tests #61462 (vdimir).
- Split
02125_lz4_compression_bug
#61465 (Antonio Andelic). - Correctly process last stacktrace in
postprocess-traces.pl
#61470 (Antonio Andelic). - Fix test
test_polymorphic_parts
#61477 (Anton Popov). - A definitive guide to CAST #61491 (Alexey Milovidov).
- Minor rename in FileCache #61494 (Kseniia Sumarokova).
- Remove useless code #61498 (Alexey Milovidov).
- Fix fuzzers #61499 (Alexey Milovidov).
- Update jdbc.md #61506 (San).
- Fix error in clickhouse-client #61507 (Alexey Milovidov).
- Fix clang-tidy build #61519 (Antonio Andelic).
- Fix infinite loop in function
hop
#61523 (Alexey Milovidov). - Improve tests 00159_parallel_formatting_* to to avoid timeouts #61532 (Kruglov Pavel).
- Refactoring of reading from compact parts #61535 (Anton Popov).
- Don't run 01459_manual_write_to_replicas in debug build as it's too slow #61538 (Kruglov Pavel).
- CI: ARM integration test - skip hdfs, kerberos, kafka #61542 (Max K.).
- More logging for loading of tables #61546 (Sergei Trifonov).
- Fixing 01584_distributed_buffer_cannot_find_column with analyzer. #61550 (Nikolai Kochetov).
- Wait for done mutation with more logs and asserts #61554 (alesapin).
- Fix read_rows count with external group by #61555 (Alexander Tokmakov).
- queries-file should be used to specify file #61557 (danila-ermakov).
- Fix
02481_async_insert_dedup_token
#61568 (Antonio Andelic). - Add a comment after #61458 #61580 (Nikolai Kochetov).
- Fix clickhouse-test client option and CLICKHOUSE_URL_PARAMS interference #61596 (vdimir).
- CI: remove compose files from integration test docker #61597 (Max K.).
- Fix 01244_optimize_distributed_group_by_sharding_key by ordering output #61602 (Mikhail f. Shiryaev).
- Remove some tests from analyzer_tech_debt #61603 (Nikolai Kochetov).
- Reduce header dependencies #61604 (Raúl Marín).
- Remove some magic_enum from headers #61606 (Raúl Marín).
- Fix configs for upgrade and bugfix #61607 (Mikhail f. Shiryaev).
- Add tests for multiple fuzzer issues #61614 (Raúl Marín).
- Try to fix
02908_many_requests_to_system_replicas
again #61616 (Nikita Taranov). - Verbose error message about analyzer_compatibility_join_using_top_level_identifier #61631 (vdimir).
- Fix 00223_shard_distributed_aggregation_memory_efficient with analyzer #61649 (Nikolai Kochetov).
- Better fuzzer logs #61650 (Raúl Marín).
- Fix flaky
02122_parallel_formatting_Template
#61651 (Antonio Andelic). - Fix Aggregator when data is empty #61654 (Antonio Andelic).
- Restore poco SUN files #61655 (Andy Fiddaman).
- Another fix for
SumIfToCountIfPass
#61656 (Antonio Andelic). - Keeper: fix data race during snapshot destructor call #61657 (Antonio Andelic).
- CI: integration tests: use runner as py module #61658 (Max K.).
- Fix logging of autoscaling lambda, add test for effective_capacity #61662 (Mikhail f. Shiryaev).
- Small change in
DatabaseOnDisk::iterateMetadataFiles()
#61664 (Nikita Taranov). - Build improvements by removing magic enum from header and apply some explicit template instantiation #61665 (János Benjamin Antal).
- Update the dictionary for OSSFuzz #61672 (Nikita Mikhaylov).
- Inhibit randomization in some tests and exclude some long tests from debug runs #61676 (Alexey Milovidov).
- Add a test for #61669 #61678 (Alexey Milovidov).
- Fix use-of-uninitialized-value in HedgedConnections #61679 (Nikolay Degterinsky).
- Remove clickhouse-diagnostics from the package #61681 (Alexey Milovidov).
- Fix use-of-uninitialized-value in parseDateTimeBestEffort #61694 (Nikolay Degterinsky).
- poco foundation: add illumos support #61701 (Andy Fiddaman).
- contrib/c-ares: add illumos as a platform #61702 (Andy Fiddaman).
- contrib/curl: Add illumos support #61704 (Andy Fiddaman).
- Fuzzer: Try a different way to wait for the server #61706 (Raúl Marín).
- Disable some tests for SMT #61708 (Raúl Marín).
- Fix signal handler for sanitizer signals #61709 (Antonio Andelic).
- Avoid
IsADirectoryError: Is a directory contrib/azure
#61710 (Mikhail f. Shiryaev). - Analyzer: fix group_by_use_nulls #61717 (Dmitry Novik).
- Analyzer: Clear list of broken integration tests #61718 (Dmitry Novik).
- CI: modify CI from PR body #61725 (Max K.).
- Add test for #57820 #61726 (Dmitry Novik).
- Revert "Revert "Un-flake test_undrop_query"" #61727 (Yarik Briukhovetskyi).
- FunctionsConversion: Start simplifying templates #61733 (Raúl Marín).
- CI: modify it #61735 (Max K.).
- Fix segfault in SquashingTransform #61736 (Michael Kolupaev).
- Fix DWARF format failing to skip DW_FORM_strx3 attributes #61737 (Michael Kolupaev).
- There is no such thing as broken tests #61739 (Alexey Milovidov).
- Process removed files, decouple _check_mime #61751 (Mikhail f. Shiryaev).
- Keeper fix: destroy
KeeperDispatcher
first #61752 (Antonio Andelic). - Fix flaky
03014_async_with_dedup_part_log_rmt
#61757 (Antonio Andelic). - FunctionsConversion: Remove another batch of bad templates #61773 (Raúl Marín).
- Revert "Fix bug when reading system.parts using UUID (issue 61220)." #61774 (János Benjamin Antal).
- CI: disable grpc tests on ARM #61778 (Max K.).
- Fix more tests with virtual columns in StorageMerge. #61787 (Nikolai Kochetov).
- Remove already not flaky tests with analyzer. #61788 (Nikolai Kochetov).
- Analyzer: Fix assert in JOIN with Distributed table #61789 (vdimir).
- A test can be slow in debug build #61796 (Alexey Milovidov).
- Updated clang-19 to master. #61798 (Alexey Milovidov).
- Fix test "00002_log_and_exception_messages_formatting" #61821 (Alexey Milovidov).
- A test is too slow for debug #61822 (Alexey Milovidov).
- Remove DataStreams #61824 (Alexey Milovidov).
- Better message for logging errors #61827 (Azat Khuzhin).
- Fix sanitizers suppressions #61828 (Azat Khuzhin).
- Remove unused code #61830 (Alexey Milovidov).
- Remove DataStreams (2) #61831 (Alexey Milovidov).
- Update xxhash to v0.8.2 #61838 (Shubham Ranjan).
- Fix: DISTINCT in subquery with analyzer #61847 (Igor Nikonov).
- Analyzer: fix limit/offset on shards #61849 (Yakov Olkhovskiy).
- Remove PoolBase::AllocateNewBypassingPool #61866 (Azat Khuzhin).
- Try to fix 02901_parallel_replicas_rollup with analyzer. #61875 (Nikolai Kochetov).
- Add test for #57808 #61879 (Nikita Mikhaylov).
- CI: merge queue support #61881 (Max K.).
- Update create.sql #61885 (Kseniia Sumarokova).
- no smaller unit in date_trunc #61888 (jsc0218).
- Move KQL trash where it is supposed to be #61903 (Alexey Milovidov).
- Changelog for 24.3 #61909 (Alexey Milovidov).
- Update version_date.tsv and changelogs after v23.3.22.3-lts #61914 (robot-clickhouse).
- Update version_date.tsv and changelogs after v23.8.12.13-lts #61915 (robot-clickhouse).
- No "please" #61916 (Alexey Milovidov).
- Update version_date.tsv and changelogs after v23.12.6.19-stable #61917 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.1.8.22-stable #61918 (robot-clickhouse).
- Fix flaky test_broken_projestions/test.py::test_broken_ignored_replic... #61932 (Kseniia Sumarokova).
- Check is Rust avaiable for build, if not, suggest a way to disable Rust support #61938 (Azat Khuzhin).
- CI: new ci menu in PR body #61948 (Max K.).
- Remove flaky test
01193_metadata_loading
#61961 (Nikita Taranov).
Packaging Improvement
- Adding the
--now
option to enable and start service automatically when installing the database server completely. #60656 (Chun-Sheng, Li).