mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
143 KiB
143 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.6.1.4423-stable (dcced7c847
) FIXME as compared to v24.4.1.2088-stable (6d4b31322d
)
Backward Incompatible Change
- Enable asynchronous load of databases and tables by default. See the
async_load_databases
in config.xml. While this change is fully compatible, it can introduce a difference in behavior. Whenasync_load_databases
is false, as in the previous versions, the server will not accept connections until all tables are loaded. Whenasync_load_databases
is true, as in the new version, the server can accept connections before all the tables are loaded. If a query is made to a table that is not yet loaded, it will wait for the table's loading, which can take considerable time. It can change the behavior of the server if it is part of a large distributed system under a load balancer. In the first case, the load balancer can get a connection refusal and quickly failover to another server. In the second case, the load balancer can connect to a server that is still loading the tables, and the query will have a higher latency. Moreover, if many queries accumulate in the waiting state, it can lead to a "thundering herd" problem when they start processing simultaneously. This can make a difference only for highly loaded distributed backends. You can set the value ofasync_load_databases
to false to avoid this problem. #57695 (Alexey Milovidov). - Some invalid queries will fail earlier during parsing. Note: disabled the support for inline KQL expressions (the experimental Kusto language) when they are put into a
kql
table function without a string literal, e.g.kql(garbage | trash)
instead ofkql('garbage | trash')
orkql($$garbage | trash$$)
. This feature was introduced unintentionally and should not exist. #61500 (Alexey Milovidov). - Renamed "inverted indexes" to "full-text indexes" which is a less technical / more user-friendly name. This also changes internal table metadata and breaks tables with existing (experimental) inverted indexes. Please make to drop such indexes before upgrade and re-create them after upgrade. #62884 (Robert Schulze).
- Usage of functions
neighbor
,runningAccumulate
,runningDifferenceStartingWithFirstValue
,runningDifference
deprecated (because it is error-prone). Proper window functions should be used instead. To enable them back, setallow_deprecated_functions=1
. #63132 (Nikita Taranov). - Queries from
system.columns
will work faster if there is a large number of columns, but many databases or tables are not granted forSHOW TABLES
. Note that in previous versions, if you grantSHOW COLUMNS
to individual columns without grantingSHOW TABLES
to the corresponding tables, thesystem.columns
table will show these columns, but in a new version, it will skip the table entirely. Remove trace log messages "Access granted" and "Access denied" that slowed down queries. #63439 (Alexey Milovidov). - Rework parallel processing in
Ordered
mode of storageS3Queue
. This PR is backward incompatible for Ordered mode if you used settingss3queue_processing_threads_num
ors3queue_total_shards_num
. Settings3queue_total_shards_num
is deleted, previously it was allowed to use only unders3queue_allow_experimental_sharded_mode
, which is now deprecated. A new setting is added -s3queue_buckets
. #64349 (Kseniia Sumarokova). - New functions
snowflakeIDToDateTime
,snowflakeIDToDateTime64
,dateTimeToSnowflakeID
, anddateTime64ToSnowflakeID
were added. Unlike the existing functionssnowflakeToDateTime
,snowflakeToDateTime64
,dateTimeToSnowflake
, anddateTime64ToSnowflake
, the new functions are compatible with functiongenerateSnowflakeID
, i.e. they accept the snowflake IDs generated bygenerateSnowflakeID
and produce snowflake IDs of the same type asgenerateSnowflakeID
(i.e.UInt64
). Furthermore, the new functions default to the UNIX epoch (aka. 1970-01-01), just likegenerateSnowflakeID
. If necessary, a different epoch, e.g. Twitter's/X's epoch 2010-11-04 aka. 1288834974657 msec since UNIX epoch, can be passed. The old conversion functions are deprecated and will be removed after a transition period: to use them regardless, enable settingallow_deprecated_snowflake_conversion_functions
. #64948 (Robert Schulze).
New Feature
- Provide support for AzureBlobStorage function in ClickHouse server to use Azure Workload identity to authenticate against Azure blob storage. If
use_workload_identity
parameter is set in config, workload identity is used for authentication. #57881 (Vinay Suryadevara). - Introduce bulk loading to StorageEmbeddedRocksDB by creating and ingesting SST file instead of relying on rocksdb build-in memtable. This help to increase importing speed, especially for long-running insert query to StorageEmbeddedRocksDB tables. Also, introduce
StorageEmbeddedRocksDB
table settings. #59163 (Duc Canh Le). - Introduce statistics of type "number of distinct values". #59357 (Han Fei).
- User can now parse CRLF with TSV format using a setting
input_format_tsv_crlf_end_of_line
. Closes #56257. #59747 (Shaun Struwig). - Add Hilbert Curve encode and decode functions. #60156 (Artem Mustafin).
- Adds the Form Format to read/write a single record in the application/x-www-form-urlencoded format. #60199 (Shaun Struwig).
- Added possibility to compress in CROSS JOIN. #60459 (p1rattttt).
- New setting
input_format_force_null_for_omitted_fields
that forces NULL values for omitted fields. #60887 (Constantine Peresypkin). - Support join with inequal conditions which involve columns from both left and right table. e.g.
t1.y < t2.y
. To enable,SET allow_experimental_join_condition = 1
. #60920 (lgbo). - Earlier our s3 storage and s3 table function didn't support selecting from archive files. I created a solution that allows to iterate over files inside archives in S3. #62259 (Daniil Ivanik).
- Support for conditional function
clamp
. #62377 (skyoct). - Add npy output format. #62430 (豪肥肥).
- Added support for reading LINESTRING geometry in WKT format using function
readWKTLineString
. #62519 (Nikita Mikhaylov). - Added SQL functions
generateUUIDv7
,generateUUIDv7ThreadMonotonic
,generateUUIDv7NonMonotonic
(with different monotonicity/performance trade-offs) to generate version 7 UUIDs aka. timestamp-based UUIDs with random component. Also added a new functionUUIDToNum
to extract bytes from a UUID and a new functionUUIDv7ToDateTime
to extract timestamp component from a UUID version 7. #62852 (Alexey Petrunyaka). - Implement Dynamic data type that allows to store values of any type inside it without knowing all of them in advance. Dynamic type is available under a setting
allow_experimental_dynamic_type
. #63058 (Kruglov Pavel). - Allow to attach parts from a different disk. #63087 (Unalian).
- Allow proxy to be bypassed for hosts specified in
no_proxy
env variable and ClickHouse proxy configuration. #63314 (Arthur Passos). - Introduce bulk loading to StorageEmbeddedRocksDB by creating and ingesting SST file instead of relying on rocksdb build-in memtable. This help to increase importing speed, especially for long-running insert query to StorageEmbeddedRocksDB tables. Also, introduce StorageEmbeddedRocksDB table settings. #63324 (Duc Canh Le).
- Raw as a synonym for TSVRaw. #63394 (Unalian).
- Added possibility to do cross join in temporary file if size exceeds limits. #63432 (p1rattttt).
- Added a new table function
loop
to support returning query results in an infinite loop. #63452 (Sariel). - Added new SQL functions
generateSnowflakeID
for generating Twitter-style Snowflake IDs. #63577 (Danila Puzov). - Add the ability to reshuffle rows during insert to optimize for size without violating the order set by
PRIMARY KEY
. It's controlled by the settingoptimize_row_order
(off by default). #63578 (Igor Markelov). - On Linux and MacOS, if the program has STDOUT redirected to a file with a compression extension, use the corresponding compression method instead of nothing (making it behave similarly to
INTO OUTFILE
). #63662 (v01dXYZ). - Added
merge_workload
andmutation_workload
settings to regulate how resources are utilized and shared between merges, mutations and other workloads. #64061 (Sergei Trifonov). - Change warning on high number of attached tables to differentiate tables, views and dictionaries. #64180 (Francisco J. Jurado Moreno).
- Add support for comparing IPv4 and IPv6 types using the
=
operator. #64292 (Francisco J. Jurado Moreno). - Allow to store named collections in zookeeper. #64574 (Kseniia Sumarokova).
- Support decimal arguments in binary math functions (pow(), atan2(), max2, min2(), hypot(). #64582 (Mikhail Gorshkov).
- Add support for index analysis over
hilbertEncode
. #64662 (Artem Mustafin). - Added SQL functions
parseReadableSize
(along withOrNull
andOrZero
variants). #64742 (Francisco J. Jurado Moreno). - Add server settings
max_table_num_to_throw
andmax_database_num_to_throw
to limit the number of databases or tables onCREATE
queries. #64781 (Xu Jia). - Add _time virtual column to file alike storages (s3/file/hdfs/url/azureBlobStorage). #64947 (Ilya Golshtein).
- Introduced new functions
base64URLEncode
,base64URLDecode
andtryBase64URLDecode
. #64991 (Mikhail Gorshkov). - Add new function
editDistanceUTF8
, which calculates the edit distance between two UTF8 strings. #65269 (LiuNeng).
Performance Improvement
- Skip merging of newly created projection blocks during
INSERT
-s. #59405 (Nikita Taranov). - Add a native parquet reader, which can read parquet binary to ClickHouse Columns directly. It's controlled by the setting
input_format_parquet_use_native_reader
(disabled by default). #60361 (ZhiHong Zhang). - Reduce the number of virtual function calls in ColumnNullable::size(). #60556 (HappenLee).
- Process string functions XXXUTF8 'asciily' if input strings are all ascii chars. Inspired by https://github.com/apache/doris/pull/29799. Overall speed up by 1.07x~1.62x. Notice that peak memory usage had been decreased in some cases. #61632 (李扬).
- Improved performance of selection (
{}
) globs in StorageS3. #62120 (Andrey Zvonov). - HostResolver has each IP address several times. If remote host has several IPs and by some reason (firewall rules for example) access on some IPs allowed and on others forbidden, than only first record of forbidden IPs marked as failed, and in each try these IPs have a chance to be chosen (and failed again). Even if fix this, every 120 seconds DNS cache dropped, and IPs can be chosen again. #62652 (Anton Ivashkin).
- Speedup
splitByRegexp
when the regular expression argument is a single-character. #62696 (Robert Schulze). - Speed up FixedHashTable by keeping track of the min and max keys used. This allows to reduce the number of cells that need to be verified. #62746 (Jiebin Sun).
- Add a new configuration
prefer_merge_sort_block_bytes
to control the memory usage and speed up sorting 2 times when merging when there are many columns. #62904 (LiuNeng). clickhouse-local
will start faster. In previous versions, it was not deleting temporary directories by mistake. Now it will. This closes #62941. #63074 (Alexey Milovidov).- Micro-optimizations for the new analyzer. #63429 (Raúl Marín).
- Index analysis will work if
DateTime
is compared toDateTime64
. This closes #63441. #63443 (Alexey Milovidov). - Index analysis will work if
DateTime
is compared toDateTime64
. This closes #63441. #63532 (Raúl Marín). - Optimize the resolution of in(LowCardinality, ConstantSet). #64060 (Zhiguo Zhou).
- Speed up indices of type
set
a little (around 1.5 times) by removing garbage. #64098 (Alexey Milovidov). - Use a thread pool to initialize and destroy hash tables inside
ConcurrentHashJoin
. #64241 (Nikita Taranov). - Optimized vertical merges in tables with sparse columns. #64311 (Anton Popov).
- Enabled prefetches of data from remote filesystem during vertical merges. It improves latency of vertical merges in tables with data stored on remote filesystem. #64314 (Anton Popov).
- Reduce redundant calls to
isDefault()
ofColumnSparse::filter
to improve performance. #64426 (Jiebin Sun). - Speedup
find_super_nodes
andfind_big_family
keeper-client commands by making multiple asynchronous getChildren requests. #64628 (Alexander Gololobov). - Improve function least/greatest for nullable numberic type arguments. #64668 (KevinyhZou).
- Allow merging two consequent
FilterSteps
of a query plan. This improves filter-push-down optimization if the filter condition can be pushed down from the parent step. #64760 (Nikolai Kochetov). - Remove bad optimization in vertical final implementation and re-enable vertical final algorithm by default. #64783 (Duc Canh Le).
- Remove ALIAS nodes from the filter expression. This slightly improves performance for queries with
PREWHERE
(with new analyzer). #64793 (Nikolai Kochetov). - Fix performance regression in cross join introduced in #60459 (24.5). #65243 (Nikita Taranov).
Improvement
- Support empty tuples. #55061 (Amos Bird).
- Hot reload storage policy for distributed tables when adding a new disk. #58285 (Duc Canh Le).
- Maps can now have
Float32
,Float64
,Array(T)
,Map(K,V)
andTuple(T1, T2, ...)
as keys. Closes #54537. #59318 (李扬). - Avoid possible deadlock during MergeTree index analysis when scheduling threads in a saturated service. #59427 (Sean Haynes).
- Multiline strings with border preservation and column width change. #59940 (Volodyachan).
- Make rabbitmq nack broken messages. Closes #45350. #60312 (Kseniia Sumarokova).
- Support partial trivial count optimization when the query filter is able to select exact ranges from merge tree tables. #60463 (Amos Bird).
- Fix a crash in asynchronous stack unwinding (such as when using the sampling query profiler) while interpreting debug info. This closes #60460. #60468 (Alexey Milovidov).
- Reduce max memory usage of multithreaded
INSERT
s by collecting chunks of multiple threads in a single transform. #61047 (Yarik Briukhovetskyi). - Distinct messages for s3 error 'no key' for cases disk and storage. #61108 (Sema Checherinda).
- Less contention in filesystem cache (part 4). Allow to keep filesystem cache not filled to the limit by doing additional eviction in the background (controlled by
keep_free_space_size(elements)_ratio
). This allows to release pressure from space reservation for queries (ontryReserve
method). Also this is done in a lock free way as much as possible, e.g. should not block normal cache usage. #61250 (Kseniia Sumarokova). - The progress bar will work for trivial queries with LIMIT from
system.zeros
,system.zeros_mt
(it already works forsystem.numbers
andsystem.numbers_mt
), and thegenerateRandom
table function. As a bonus, if the total number of records is greater than themax_rows_to_read
limit, it will throw an exception earlier. This closes #58183. #61823 (Alexey Milovidov). - YAML Merge Key support. #62685 (Azat Khuzhin).
- Enhance error message when non-deterministic function is used with Replicated source. #62896 (Grégoire Pineau).
- Fix interserver secret for Distributed over Distributed from
remote
. #63013 (Azat Khuzhin). - Allow using
clickhouse-local
and its shortcutsclickhouse
andch
with a query or queries file as a positional argument. Examples:ch "SELECT 1"
,ch --param_test Hello "SELECT {test:String}"
,ch query.sql
. This closes #62361. #63081 (Alexey Milovidov). - Support configuration substitutions from YAML files. #63106 (Eduard Karacharov).
- Reduce the memory usage when using Azure object storage by using fixed memory allocation, avoiding the allocation of an extra buffer. #63160 (SmitaRKulkarni).
- Add TTL information in system parts_columns table. #63200 (litlig).
- Keep previous data in terminal after picking from skim suggestions. #63261 (FlameFactory).
- Width of fields now correctly calculate, ignoring ANSI escape sequences. #63270 (Shaun Struwig).
- Enable plain_rewritable metadata for local and Azure (azure_blob_storage) object storages. #63365 (Julia Kartseva).
- Support English-style Unicode quotes, e.g. “Hello”, ‘world’. This is questionable in general but helpful when you type your query in a word processor, such as Google Docs. This closes #58634. #63381 (Alexey Milovidov).
- Allowed to create MaterializedMySQL database without connection to MySQL. #63397 (Kirill).
- Remove copying data when writing to filesystem cache. #63401 (Kseniia Sumarokova).
- Update the usage of error code
NUMBER_OF_ARGUMENTS_DOESNT_MATCH
by more accurate error codes when appropriate. #63406 (Yohann Jardin). - Several minor corner case fixes to proxy support & tunneling. #63427 (Arthur Passos).
os_user
andclient_hostname
are now correctly set up for queries for command line suggestions in clickhouse-client. This closes #63430. #63433 (Alexey Milovidov).- Fixed tabulation from line numbering, correct handling of length when moving a line if the value has a tab, added tests. #63493 (Volodyachan).
- Add this
aggregate_function_group_array_has_limit_size
setting to support discarding data in some scenarios. #63516 (zhongyuankai). - Automatically mark a replica of Replicated database as lost and start recovery if some DDL task fails more than
max_retries_before_automatic_recovery
(100 by default) times in a row with the same error. Also, fixed a bug that could cause skipping DDL entries when an exception is thrown during an early stage of entry execution. #63549 (Alexander Tokmakov). - Add
http_response_headers
setting to support custom response headers in custom HTTP handlers. #63562 (Grigorii). - Automatically correct
max_block_size=0
to default value. #63587 (Antonio Andelic). - Account failed files in
s3queue_tracked_file_ttl_sec
ands3queue_traked_files_limit
forStorageS3Queue
. #63638 (Kseniia Sumarokova). - Add a build_id ALIAS column to trace_log to facilitate auto renaming upon detecting binary changes. This is to address #52086. #63656 (Zimu Li).
- Enable truncate operation for object storage disks. #63693 (MikhailBurdukov).
- Improve io_uring resubmits visibility. Rename profile event
IOUringSQEsResubmits
->IOUringSQEsResubmitsAsync
and add a new oneIOUringSQEsResubmitsSync
. #63699 (Tomer Shafir). - Introduce assertions to verify all functions are called with columns of the right size. #63723 (Raúl Marín).
- The loading of the keywords list is now dependent on the server revision and will be disabled for the old versions of ClickHouse server. CC @azat. #63786 (Nikita Mikhaylov).
SHOW CREATE TABLE
executed on top of system tables will now show the super handy comment unique for each table which will explain why this table is needed. #63788 (Nikita Mikhaylov).- Allow trailing commas in the columns list in the INSERT query. For example,
INSERT INTO test (a, b, c, ) VALUES ...
. #63803 (Alexey Milovidov). - Better exception messages for the
Regexp
format. #63804 (Alexey Milovidov). - Allow trailing commas in the
Values
format. For example, this query is allowed:INSERT INTO test (a, b, c) VALUES (4, 5, 6,);
. #63810 (Alexey Milovidov). - Clickhouse disks have to read server setting to obtain actual metadata format version. #63831 (Sema Checherinda).
- Disable pretty format restrictions (
output_format_pretty_max_rows
/output_format_pretty_max_value_width
) when stdout is not TTY. #63942 (Azat Khuzhin). - Exception handling now works when ClickHouse is used inside AWS Lambda. Author: Alexey Coolnev. #64014 (Alexey Milovidov).
- Throw
CANNOT_DECOMPRESS
instread ofCORRUPTED_DATA
on invalid compressed data passed via HTTP. #64036 (vdimir). - A tip for a single large number in Pretty formats now works for Nullable and LowCardinality. This closes #61993. #64084 (Alexey Milovidov).
- Now backups with azure blob storage will use multicopy. #64116 (alesapin).
- Added a new setting,
metadata_keep_free_space_bytes
to keep free space on the metadata storage disk. #64128 (MikhailBurdukov). - Add metrics, logs, and thread names around parts filtering with indices. #64130 (Alexey Milovidov).
- Allow to use native copy for azure even with different containers. #64154 (alesapin).
- Add metrics to track the number of directories created and removed by the plain_rewritable metadata storage, and the number of entries in the local-to-remote in-memory map. #64175 (Julia Kartseva).
- Finally enable native copy for azure. #64182 (alesapin).
- Ignore
allow_suspicious_primary_key
onATTACH
and verify onALTER
. #64202 (Azat Khuzhin). - The query cache now considers identical queries with different settings as different. This increases robustness in cases where different settings (e.g.
limit
oradditional_table_filters
) would affect the query result. #64205 (Robert Schulze). - Better Exception Message in Delete Table with Projection, users can understand the error and the steps should be taken. #64212 (jsc0218).
- Support the non standard error code
QpsLimitExceeded
in object storage as a retryable error. #64225 (Sema Checherinda). - Forbid converting a MergeTree table to replicated if the zookeeper path for this table already exists. #64244 (Kirill).
- If "replica group" is configured for a
Replicated
database, automatically create a cluster that includes replicas from all groups. #64312 (Alexander Tokmakov). - Added settings to disable materialization of skip indexes and statistics on inserts (
materialize_skip_indexes_on_insert
andmaterialize_statistics_on_insert
). #64391 (Anton Popov). - Use the allocated memory size to calculate the row group size and reduce the peak memory of the parquet writer in single-threaded mode. #64424 (LiuNeng).
- Added new configuration input_format_parquet_prefer_block_bytes to control the average output block bytes, and modified the default value of input_format_parquet_max_block_size to 65409. #64427 (LiuNeng).
- Always start Keeper with sufficient amount of threads in global thread pool. #64444 (Duc Canh Le).
- Settings from user config doesn't affect merges and mutations for MergeTree on top of object storage. #64456 (alesapin).
- Setting
replace_long_file_name_to_hash
is enabled by default forMergeTree
tables. #64457 (Anton Popov). - Improve the iterator of sparse column to reduce call of size(). #64497 (Jiebin Sun).
- Update condition to use copy for azure blob storage. #64518 (SmitaRKulkarni).
- Support the non standard error code
TotalQpsLimitExceeded
in object storage as a retryable error. #64520 (Sema Checherinda). - Optimized memory usage of vertical merges for tables with high number of skip indexes. #64580 (Anton Popov).
- Introduced two additional columns in the
system.query_log
:used_privileges
andmissing_privileges
.used_privileges
is populated with the privileges that were checked during query execution, andmissing_privileges
contains required privileges that are missing. #64597 (Alexey Katsman). - Add settings
parallel_replicas_custom_key_range_lower
andparallel_replicas_custom_key_range_upper
to control how parallel replicas with dynamic shards parallelizes queries when using a range filter. #64604 (josh-hildred). - Updated Advanced Dashboard for both open-source and ClickHouse Cloud versions to include a chart for 'Maximum concurrent network connections'. #64610 (Thom O'Connor).
- The second argument (scale) of functions
round()
,roundBankers()
,floor()
,ceil()
andtrunc()
can now be non-const. #64798 (Mikhail Gorshkov). - Improve progress report on zeros_mt and generateRandom. #64804 (Raúl Marín).
- Add an asynchronous metric jemalloc.profile.active to show whether sampling is currently active. This is an activation mechanism in addition to prof.active; both must be active for the calling thread to sample. #64842 (Unalian).
- Support statistics with ReplicatedMergeTree. #64934 (Han Fei).
- Don't mark of
allow_experimental_join_condition
as IMPORTANT. This may have prevented distributed queries in a mixed versions cluster from being executed successfully. #65008 (Nikita Mikhaylov). - Backported in #65716:
StorageS3Queue
related fixes and improvements. Deduce a default value ofs3queue_processing_threads_num
according to the number of physical cpu cores on the server (instead of the previous default value as 1). Set default value ofs3queue_loading_retries
to 10. Fix possible vague "Uncaught exception" in exception column ofsystem.s3queue
. Do not increment retry count onMEMORY_LIMIT_EXCEEDED
exception. Move files commit to a stage after insertion into table fully finished to avoid files being commited while not inserted. Add settingss3queue_max_processed_files_before_commit
,s3queue_max_processed_rows_before_commit
,s3queue_max_processed_bytes_before_commit
,s3queue_max_processing_time_sec_before_commit
, to better control commit and flush time. #65046 (Kseniia Sumarokova). - Added server Asynchronous metrics
DiskGetObjectThrottler*
andDiskGetObjectThrottler*
reflecting request per second rate limit defined withs3_max_get_rps
ands3_max_put_rps
disk settings and currently available number of requests that could be sent without hitting throttling limit on the disk. Metrics are defined for every disk that has a configured limit. #65050 (Sergei Trifonov). - Added a setting
output_format_pretty_display_footer_column_names
which when enabled displays column names at the end of the table for long tables (50 rows by default), with the threshold value for minimum number of rows controlled byoutput_format_pretty_display_footer_column_names_min_rows
. #65144 (Shaun Struwig). - Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings:
output_format_csv_serialize_tuple_into_separate_columns
,input_format_csv_deserialize_separate_columns_into_tuple
andinput_format_csv_try_infer_strings_from_quoted_tuples
. #65170 (Nikita Mikhaylov). - Initialize global trace collector for Poco::ThreadPool (needed for keeper, etc). #65239 (Kseniia Sumarokova).
- Add validation when creating a user with bcrypt_hash. #65242 (Raúl Marín).
Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)
- Fix a permission error where a user in a specific situation can escalate their privileges on the default database without necessary grants. #64769 (pufit).
- Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. #65188 (Raúl Marín).
- Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. #65198 (Aleksei Filatov).
- Forbid
QUALIFY
clause in the old analyzer. The old analyzer ignoredQUALIFY
, so it could lead to unexpected data removal in mutations. #65356 (Dmitry Novik). - Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. #65379 (Antonio Andelic).
- Backported in #65846: Check cyclic dependencies on CREATE/REPLACE/RENAME/EXCHANGE queries and throw an exception if there is a cyclic dependency. Previously such cyclic dependencies could lead to a deadlock during server startup. Closes #65355. Also fix some bugs in dependencies creation. #65405 (Kruglov Pavel).
- Backported in #65714: Fix crash in maxIntersections. #65689 (Raúl Marín).
Bug Fix (user-visible misbehavior in an official stable release)
- Fix making backup when multiple shards are used. This PR fixes #56566. #57684 (Vitaly Baranov).
- Fix passing projections/indexes from CREATE query into inner table of MV. #59183 (Azat Khuzhin).
- Fix boundRatio incorrect merge. #60532 (Tao Wang).
- Fix crash when using some functions with low-cardinality columns. #61966 (Michael Kolupaev).
- Fixed 'set' skip index not working with IN and indexHint(). #62083 (Michael Kolupaev).
- Fix queries with FINAL give wrong result when table does not use adaptive granularity. #62432 (Duc Canh Le).
- Improve the detection of cgroups v2 memory controller in unusual locations. This fixes a warning that the cgroup memory observer was disabled because no cgroups v1 or v2 current memory file could be found. #62903 (Robert Schulze).
- Fix subsequent use of external tables in client. #62964 (Azat Khuzhin).
- Fix crash with untuple and unresolved lambda. #63131 (Raúl Marín).
- Fix bug which could lead to server to accept connections before server is actually loaded. #63181 (alesapin).
- Fix intersect parts when restart after drop range. #63202 (Han Fei).
- Fix a misbehavior when SQL security defaults don't load for old tables during server startup. #63209 (pufit).
- JOIN filter push down filled join fix. Closes #63228. #63234 (Maksim Kita).
- Fix infinite loop while listing objects in Azure blob storage. #63257 (Julia Kartseva).
- CROSS join can be executed with any value
join_algorithm
setting, close #62431. #63273 (vdimir). - Fixed a potential crash caused by a
no space left
error when temporary data in the cache is used. #63346 (vdimir). - Fix bug which could potentially lead to rare LOGICAL_ERROR during SELECT query with message:
Unexpected return type from materialize. Expected type_XXX. Got type_YYY.
Introduced in #59379. #63353 (alesapin). - Fix
X-ClickHouse-Timezone
header returning wrong timezone when usingsession_timezone
as query level setting. #63377 (Andrey Zvonov). - Fix debug assert when using grouping WITH ROLLUP and LowCardinality types. #63398 (Raúl Marín).
- Fix logical errors in queries with
GROUPING SETS
andWHERE
andgroup_by_use_nulls = true
, close #60538. #63405 (vdimir). - Fix backup of projection part in case projection was removed from table metadata, but part still has projection. #63426 (Kseniia Sumarokova).
- Fix 'Every derived table must have its own alias' error for MYSQL dictionary source, close #63341. #63481 (vdimir).
- Insert QueryFinish on AsyncInsertFlush with no data. #63483 (Raúl Marín).
- Fix
system.query_log.used_dictionaries
logging. #63487 (Eduard Karacharov). - Support executing function during assignment of parameterized view value. #63502 (SmitaRKulkarni).
- Avoid segafult in
MergeTreePrefetchedReadPool
while fetching projection parts. #63513 (Antonio Andelic). - Fix rabbitmq heap-use-after-free found by clang-18, which can happen if an error is thrown from RabbitMQ during initialization of exchange and queues. #63515 (Kseniia Sumarokova).
- Fix crash on exit with sentry enabled (due to openssl destroyed before sentry). #63548 (Azat Khuzhin).
- Fixed parquet memory tracking. #63584 (Michael Kolupaev).
- Fix support for Array and Map with Keyed hashing functions and materialized keys. #63628 (Salvatore Mesoraca).
- Fixed Parquet filter pushdown not working with Analyzer. #63642 (Michael Kolupaev).
- It is forbidden to convert MergeTree to replicated if the zookeeper path for this table already exists. #63670 (Kirill).
- Read only the necessary columns from VIEW (new analyzer). Closes #62594. #63688 (Maksim Kita).
- Fix rare case with missing data in the result of distributed query. #63691 (vdimir).
- Fix #63539. Forbid WINDOW redefinition in new analyzer. #63694 (Dmitry Novik).
- Flatten_nested is broken with replicated database. #63695 (Nikolai Kochetov).
- Fix
SIZES_OF_COLUMNS_DOESNT_MATCH
error for queries witharrayJoin
function inWHERE
. Fixes #63653. #63722 (Nikolai Kochetov). - Fix
Not found column
andCAST AS Map from array requires nested tuple of 2 elements
exceptions for distributed queries which useMap(Nothing, Nothing)
type. Fixes #63637. #63753 (Nikolai Kochetov). - Fix possible
ILLEGAL_COLUMN
error inpartial_merge
join, close #37928. #63755 (vdimir). query_plan_remove_redundant_distinct
can break queries with WINDOW FUNCTIONS (withallow_experimental_analyzer
is on). Fixes #62820. #63776 (Igor Nikonov).- Fix possible crash with SYSTEM UNLOAD PRIMARY KEY. #63778 (Raúl Marín).
- Fix a query with a duplicating cycling alias. Fixes #63320. #63791 (Nikolai Kochetov).
- Fixed performance degradation of parsing data formats in INSERT query. This closes #62918. This partially reverts #42284, which breaks the original design and introduces more problems. #63801 (Alexey Milovidov).
- Add 'endpoint_subpath' S3 URI setting to allow plain_rewritable disks to share the same endpoint. #63806 (Julia Kartseva).
- Fix queries using parallel read buffer (e.g. with max_download_thread > 0) getting stuck when threads cannot be allocated. #63814 (Antonio Andelic).
- Allow JOIN filter push down to both streams if only single equivalent column is used in query. Closes #63799. #63819 (Maksim Kita).
- Remove the data from all disks after DROP with the Lazy database engines. Without these changes, orhpaned will remain on the disks. #63848 (MikhailBurdukov).
- Fix incorrect select query result when parallel replicas were used to read from a Materialized View. #63861 (Nikita Taranov).
- Fixes in
find_super_nodes
andfind_big_family
command of keeper-client: - do not fail on ZNONODE errors - find super nodes inside super nodes - properly calculate subtree node count. #63862 (Alexander Gololobov). - Fix a error
Database name is empty
for remote queries with lambdas over the cluster with modified default database. Fixes #63471. #63864 (Nikolai Kochetov). - Fix SIGSEGV due to CPU/Real (
query_profiler_real_time_period_ns
/query_profiler_cpu_time_period_ns
) profiler (has been an issue since 2022, that leads to periodic server crashes, especially if you were using distributed engine). #63865 (Azat Khuzhin). - Fixed
EXPLAIN CURRENT TRANSACTION
query. #63926 (Anton Popov). - Fix analyzer - IN function with arbitrary deep sub-selects in materialized view to use insertion block. #63930 (Yakov Olkhovskiy).
- Allow
ALTER TABLE .. MODIFY|RESET SETTING
andALTER TABLE .. MODIFY COMMENT
for plain_rewritable disk. #63933 (Julia Kartseva). - Fix Recursive CTE with distributed queries. Closes #63790. #63939 (Maksim Kita).
- Fixed reading of columns of type
Tuple(Map(LowCardinality(String), String), ...)
. #63956 (Anton Popov). - Fix resolve of unqualified COLUMNS matcher. Preserve the input columns order and forbid usage of unknown identifiers. #63962 (Dmitry Novik).
- Fix the
Not found column
error for queries withskip_unused_shards = 1
,LIMIT BY
, and the new analyzer. Fixes #63943. #63983 (Nikolai Kochetov). - (Low-quality third-party Kusto Query Language). Resolve Client Abortion Issue When Using KQL Table Function in Interactive Mode. #63992 (Yong Wang).
- Fix an
Cyclic aliases
error for cyclic aliases of different type (expression and function). #63993 (Nikolai Kochetov). - Deserialize untrusted binary inputs in a safer way. #64024 (Robert Schulze).
- Do not throw
Storage doesn't support FINAL
error for remote queries over non-MergeTree tables withfinal = true
and new analyzer. Fixes #63960. #64037 (Nikolai Kochetov). - Add missing settings to recoverLostReplica. #64040 (Raúl Marín).
- Fix unwind on SIGSEGV on aarch64 (due to small stack for signal). #64058 (Azat Khuzhin).
- This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline. #64079 (pufit).
- Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. #64096 (Yakov Olkhovskiy).
- Fix azure backup writing multipart blocks as 1mb (read buffer size) instead of max_upload_part_size. #64117 (Kseniia Sumarokova).
- Fix creating backups to S3 buckets with different credentials from the disk containing the file. #64153 (Antonio Andelic).
- Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. #64174 (Raúl Marín).
- The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. #64199 (Robert Schulze).
- Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. #64206 (Kruglov Pavel).
- Ignore
text_log
config when using Keeper. #64218 (Antonio Andelic). - Fix
duplicate alias
error for distributed queries withARRAY JOIN
. #64226 (Nikolai Kochetov). - Fix unexpected accurateCast from string to integer. #64255 (wudidapaopao).
- Fixed CNF simplification, in case any OR group contains mutually exclusive atoms. #64256 (Eduard Karacharov).
- Fix Query Tree size validation. #64377 (Dmitry Novik).
- Fix
Logical error: Bad cast
forBuffer
table withPREWHERE
. #64388 (Nikolai Kochetov). - Prevent recursive logging in
blob_storage_log
when it's stored on object storage. #64393 (vdimir). - Fixed
CREATE TABLE AS
queries for tables with default expressions. #64455 (Anton Popov). - Fixed
optimize_read_in_order
behaviour for ORDER BY ... NULLS FIRST / LAST on tables with nullable keys. #64483 (Eduard Karacharov). - Fix the
Expression nodes list expected 1 projection names
andUnknown expression or identifier
errors for queries with aliases toGLOBAL IN.
. #64517 (Nikolai Kochetov). - Fix an error
Cannot find column
in distributed queries with constant CTE in theGROUP BY
key. #64519 (Nikolai Kochetov). - Fixed ORC statistics calculation, when writing, for unsigned types on all platforms and Int8 on ARM. #64563 (Michael Kolupaev).
- Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. #64595 (pufit).
- Fix the output of function
formatDateTimeInJodaSyntax
when a formatter generates an uneven number of characters and the last character is0
. For example,SELECT formatDateTimeInJodaSyntax(toDate('2012-05-29'), 'D')
now correctly returns150
instead of previously15
. #64614 (LiuNeng). - Do not rewrite aggregation if
-If
combinator is already used. #64638 (Dmitry Novik). - Fix type inference for float (in case of small buffer, i.e.
--max_read_buffer_size 1
). #64641 (Azat Khuzhin). - Fix bug which could lead to non-working TTLs with expressions. #64694 (alesapin).
- Fix removing the
WHERE
andPREWHERE
expressions, which are always true (for the new analyzer). #64695 (Nikolai Kochetov). - Fixed excessive part elimination by token-based text indexes (
ngrambf
,full_text
) when filtering by result ofstartsWith
,endsWith
,match
,multiSearchAny
. #64720 (Eduard Karacharov). - Fixes incorrect behaviour of ANSI CSI escaping in the
UTF8::computeWidth
function. #64756 (Shaun Struwig). - Fix a case of incorrect removal of
ORDER BY
/LIMIT BY
across subqueries. #64766 (Raúl Marín). - Fix (experimental) unequal join with subqueries for sets which are in the mixed join conditions. #64775 (lgbo).
- Fix crash in a local cache over
plain_rewritable
disk. #64778 (Julia Kartseva). - Keeper fix: return correct value for
zk_latest_snapshot_size
inmntr
command. #64784 (Antonio Andelic). - Fix
Cannot find column
in distributed query withARRAY JOIN
byNested
column. Fixes #64755. #64801 (Nikolai Kochetov). - Fix memory leak in slru cache policy. #64803 (Kseniia Sumarokova).
- Fixed possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. #64844 (Anton Popov).
- Fix the
Block structure mismatch
error for queries reading withPREWHERE
from the materialized view when the materialized view has columns of different types than the source table. Fixes #64611. #64855 (Nikolai Kochetov). - Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. #64858 (alesapin).
- Fix duplicating
Delete
events inblob_storage_log
in case of large batch to delete. #64924 (vdimir). - Backported in #65544: Fix crash for
ALTER TABLE ... ON CLUSTER ... MODIFY SQL SECURITY
. #64957 (pufit). - Fixed
Session moved to another server
error from [Zoo]Keeper that might happen after server startup when the config has includes from [Zoo]Keeper. #64986 (Alexander Tokmakov). - Backported in #65582: Fix crash on destroying AccessControl: add explicit shutdown. #64993 (Vitaly Baranov).
- Fix
ALTER MODIFY COMMENT
query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. #65031 (Nikolay Degterinsky). - Fix
host_id
in DatabaseReplicated whencluster_secure_connection
parameter is enabled. Previously all the connections within the cluster created by DatabaseReplicated were not secure, even if the parameter was enabled. #65054 (Nikolay Degterinsky). - Fixing the
Not-ready Set
error after thePREWHERE
optimization for StorageMerge. #65057 (Nikolai Kochetov). - Avoid writing to finalized buffer in File-like storages. #65063 (Kruglov Pavel).
- Fix possible infinite query duration in case of cyclic aliases. Fixes #64849. #65081 (Nikolai Kochetov).
- Fix the
Unknown expression identifier
error for remote queries withINTERPOLATE (alias)
(new analyzer). Fixes #64636. #65090 (Nikolai Kochetov). - Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. #65104 (Dmitry Novik).
- Fix aggregate function name rewriting in the new analyzer. #65110 (Dmitry Novik).
- Respond with 5xx instead of 200 OK in case of receive timeout while reading (parts of) the request body from the client socket. #65118 (Julian Maicher).
- Backported in #65734: Eliminate injective function in argument of functions
uniq*
recursively. This used to work correctly but was broken in the new analyzer. #65140 (Duc Canh Le). - Fix possible crash for hedged requests. #65206 (Azat Khuzhin).
- Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. #65256 (jsc0218).
- This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. fix (#64487). #65315 (pn).
- Backported in #65665: Disable
non-intersecting-parts
optimization for queries withFINAL
in case ofread-in-order
optimization was enabled. This could lead to an incorrect query result. As a workaround, disabledo_not_merge_across_partitions_select_final
andsplit_parts_ranges_into_intersecting_and_non_intersecting_final
before this fix is merged. #65505 (Nikolai Kochetov). - Backported in #65606: Fix getting exception
Index out of bound for blob metadata
in case all files from list batch were filtered out. #65523 (Kseniia Sumarokova). - Backported in #65790: Fixed bug in MergeJoin. Column in sparse serialisation might be treated as a column of its nested type though the required conversion wasn't performed. #65632 (Nikita Taranov).
- Backported in #65814: Fix invalid exceptions in function
parseDateTime
with%F
and%D
placeholders. #65768 (Antonio Andelic). - Backported in #65830: Fix a bug in short circuit logic when old analyzer and dictGetOrDefault is used. #65802 (jsc0218).
Build/Testing/Packaging Improvement
- ClickHouse is built with clang-18. A lot of new checks from clang-tidy-18 have been enabled. #60469 (Alexey Milovidov).
- Make
network
service be required when using the rc init script to start the ClickHouse server daemon. #60650 (Chun-Sheng, Li). - Re-enable broken s390x build in CI. #63135 (Harry Lee).
- The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. #63400 (Mikhail f. Shiryaev).
- Information about every symbol in every translation unit will be collected in the CI database for every build in the CI. This closes #63494. #63495 (Alexey Milovidov).
- Experimentally support loongarch64 as a new platform for ClickHouse. #63733 (qiangxuhui).
- Update Apache Datasketches library. It resolves #63858. #63923 (Alexey Milovidov).
- Enable GRPC support for aarch64 linux while cross-compiling binary. #64072 (alesapin).
- Fix typo in test_hdfsCluster_unset_skip_unavailable_shards. The test writes data to unskip_unavailable_shards, but uses skip_unavailable_shards from the previous test. #64243 (Mikhail Artemenko).
- Reduce the size of some slow tests. #64387 (Raúl Marín).
- Reduce the size of some slow tests. #64452 (Raúl Marín).
- Fix test_lost_part_other_replica. #64512 (Raúl Marín).
- Add tests for experimental unequal joins and randomize new settings in clickhouse-test. #64535 (Nikita Fomichev).
- Upgrade tests: Update config and work with release candidates. #64542 (Raúl Marín).
- Add support for LLVM XRay. #64592 (Tomer Shafir).
- Speed up 02995_forget_partition. #64761 (Raúl Marín).
- Fix 02790_async_queries_in_query_log. #64764 (Raúl Marín).
- Support LLVM XRay on Linux amd64 only. #64837 (Tomer Shafir).
- Get rid of custom code in
tests/ci/download_release_packages.py
andtests/ci/get_previous_release_tag.py
to avoid issues after the https://github.com/ClickHouse/ClickHouse/pull/64759 is merged. #64848 (Mikhail f. Shiryaev). - Decrease the
unit-test
image a few times. #65102 (Mikhail f. Shiryaev).
NO CL CATEGORY
- Backported in #65568:. #65498 (Sergei Trifonov).
- Backported in #65693:. #65686 (Raúl Marín).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Do not remove server constants from GROUP BY key for secondary query."'. #63297 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Introduce bulk loading to StorageEmbeddedRocksDB"'. #63316 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Revert "Do not remove server constants from GROUP BY key for secondary query.""'. #63415 (Nikolai Kochetov).
- NO CL ENTRY: 'Revert "Fix index analysis for
DateTime64
"'. #63525 (Raúl Marín). - NO CL ENTRY: 'Revert "Update gui.md - Add ch-ui to open-source available tools."'. #64064 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Prevent conversion to Replicated if zookeeper path already exists"'. #64214 (Sergei Trifonov).
- NO CL ENTRY: 'Revert "Refactoring of Server.h: Isolate server management from other logic"'. #64425 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "Remove some unnecessary
UNREACHABLE
s"'. #64430 (Alexander Tokmakov). - NO CL ENTRY: 'Revert "CI: fix build_report selection in case of job reuse"'. #64516 (Max K.).
- NO CL ENTRY: 'Revert "Revert "CI: fix build_report selection in case of job reuse""'. #64531 (Max K.).
- NO CL ENTRY: 'Revert "Add
fromReadableSize
function"'. #64616 (Robert Schulze). - NO CL ENTRY: 'Update CHANGELOG.md'. #64816 (Paweł Kudzia).
- NO CL ENTRY: 'Revert "Reduce lock contention for MergeTree tables (by renaming parts without holding lock)"'. #64899 (alesapin).
- NO CL ENTRY: 'Revert "Add dynamic untracked memory limits for more precise memory tracking"'. #64969 (Sergei Trifonov).
- NO CL ENTRY: 'Revert "Fix duplicating Delete events in blob_storage_log"'. #65049 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "Revert "Fix duplicating Delete events in blob_storage_log""'. #65053 (vdimir).
- NO CL ENTRY: 'Revert "S3: reduce retires time for queries, increase retries count for backups"'. #65148 (Raúl Marín).
- NO CL ENTRY: 'Revert "Small fix for 02340_parts_refcnt_mergetree"'. #65149 (Raúl Marín).
- NO CL ENTRY: 'Revert "Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported"'. #65250 (Max K.).
- NO CL ENTRY: 'Revert "Fix AWS ECS"'. #65361 (Alexander Tokmakov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Try abort on current thread join. #42544 (Nikolai Kochetov).
- This change was reverted. #51008 (Michael Kolupaev).
- Analyzer fuzzer 2. #57098 (Nikolai Kochetov).
- Analyzer fuzzer 4. #57101 (Nikolai Kochetov).
- Check python code with flake8. #58349 (Azat Khuzhin).
- Unite s3/hdfs/azure storage implementations into a single class working with IObjectStorage. Same for *Cluster, data lakes and Queue storages. #59767 (Kseniia Sumarokova).
- Remove http_max_chunk_size setting (too internal). #60852 (Azat Khuzhin).
- Fix race in refreshable materialized views causing SELECT to fail sometimes. #60883 (Michael Kolupaev).
- Refactor KeyCondition and key analysis to improve PartitionPruner and trivial count optimization. This is separated from #60463 . #61459 (Amos Bird).
- Implement cumulative A Sync status. #61464 (Mikhail f. Shiryaev).
- Parallel replicas: table check failover. #61935 (Igor Nikonov).
- This change was reverted. #61973 (Azat Khuzhin).
- Avoid crashing on column type mismatch in a few dozen places. #62087 (Michael Kolupaev).
- Fix optimize_if_chain_to_multiif const NULL handling. #62104 (Michael Kolupaev).
- Use intrusive lists for
ResourceRequest
instead of deque. #62165 (Sergei Trifonov). - Analyzer: Fix validateAggregates for tables with different aliases. #62346 (vdimir).
- Improve code and tests of
DROP
of multiple tables. #62359 (zhongyuankai). - Fix exception message during writing to partitioned s3/hdfs/azure path with globs. #62423 (Kruglov Pavel).
- Support UBSan on Clang-19 (master). #62466 (Alexey Milovidov).
- Replay ZK logs using keeper-bench. #62481 (Antonio Andelic).
- Save the stacktrace of thread waiting on failing AsyncLoader job. #62719 (Sergei Trifonov).
- group_by_use_nulls strikes back. #62922 (Nikolai Kochetov).
- Analyzer: prefer column name to alias from array join. #62995 (vdimir).
- CI: try separate the workflows file for GitHub's Merge Queue. #63123 (Max K.).
- Try to fix coverage tests. #63130 (Raúl Marín).
- Fix azure backup flaky test. #63158 (SmitaRKulkarni).
- Merging #60920. #63159 (vdimir).
- QueryAnalysisPass improve QUALIFY validation. #63162 (Maksim Kita).
- Add numpy tests for different endianness. #63189 (Yarik Briukhovetskyi).
- Clean the
_work
directory between runner's launches. Fallback to auto-update actions runner if it fails to start. Make theinit-network.sh
sourceable and executable. #63195 (Mikhail f. Shiryaev). - Add ability to run Azure tests in PR with label. #63196 (alesapin).
- Fix possible endless loop while reading from azure. #63197 (Anton Popov).
- Add information about materialized view security bug fix into the changelog. #63204 (pufit).
- Disable one test from 02994_sanity_check_settings. #63208 (Raúl Marín).
- Enable custom parquet encoder by default, attempt 2. #63210 (Michael Kolupaev).
- Update version after release. #63215 (Alexey Milovidov).
- Update version_date.tsv and changelogs after v24.4.1.2088-stable. #63217 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.3.3.102-lts. #63226 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.2.3.70-stable. #63227 (robot-clickhouse).
- Return back #61551 (More optimal loading of marks). #63233 (Anton Popov).
- Hide CI options under a spoiler. #63237 (Konstantin Bogdanov).
- Add azure run with msan. #63238 (alesapin).
- Now syntax for this command is following:
TRUNCATE ALL TABLES FROM [IF EXISTS] <database_name>
. #63241 (Yarik Briukhovetskyi). - Minor follow-up to a renaming PR. #63260 (Robert Schulze).
- Followup for #62613 Adding back checks similar to these: https://github.com/ClickHouse/ClickHouse/pull/62613/files#diff-70859078da57ecdfc66d26f732c0d7718d269e82bdc80e62b39f5ffeab36c05bL99 https://github.com/ClickHouse/ClickHouse/pull/62613/files#diff-70859078da57ecdfc66d26f732c0d7718d269e82bdc80e62b39f5ffeab36c05bL144-L149. #63274 (Alexander Gololobov).
- This setting was added in 24.5, not 24.4. #63278 (Raúl Marín).
- Improve cloud backport script. #63282 (Raúl Marín).
- Update version_date.tsv and changelogs after v23.8.14.6-lts. #63285 (robot-clickhouse).
- Fix azure flaky test. #63286 (SmitaRKulkarni).
- Fix deadlock in
CacheDictionaryUpdateQueue
in case of exception in constructor. #63287 (Nikita Taranov). - DiskApp: fix 'list --recursive /' and crash on invalid arguments. #63296 (Michael Kolupaev).
- Fix terminate because of unhandled exception in
MergeTreeDeduplicationLog::shutdown
. #63298 (Nikita Taranov). - Move s3_plain_rewritable unit test to shell. #63317 (Julia Kartseva).
- Add tests for #63264. #63321 (Raúl Marín).
- Try fix segfault in
MergeTreeReadPoolBase::createTask
. #63323 (Antonio Andelic). - Reduce time-to-insert profiling data in case of logs cluster issues. #63325 (Mikhail f. Shiryaev).
- Update README.md. #63326 (Tyler Hannan).
- This should fix failures with error like
Permission denied ["/var/lib/clickhouse/disks/s3/store/364/3643ff83-0996-4a4a-a90b-a96e66a10c74"]
when table dir was chmod-ed by DatabaseCatalog. #63330 (Alexander Gololobov). - Use
/commit/
to have the URLs in reports like44f8bc5308
and not like44f8bc5308
. #63331 (Mikhail f. Shiryaev). - Add test for #56287. #63340 (Raúl Marín).
- Update README.md. #63350 (Tyler Hannan).
- Add test for #48049. #63351 (Raúl Marín).
- Add option
query_id_prefix
toclickhouse-benchmark
. #63352 (Anton Popov). - New version is fantatish (at least with Ubuntu 22.04.4 LTS): ``` azurite --version /usr/local/lib/node_modules/azurite/dist/src/common/persistence/MemoryExtentStore.js:53 return this._chunks.get(categoryName)?.chunks.get(id); ^. #63354 (alesapin).
- Randomize setting
enable_block_offset_column
in stress tests. #63355 (Anton Popov). - Fix AST parsing of invalid type names. #63357 (Michael Kolupaev).
- Fix some 00002_log_and_exception_messages_formatting flakiness. #63358 (Michael Kolupaev).
- Add tags for the test 03000_traverse_shadow_system_data_paths.sql to make it stable. #63366 (Aleksei Filatov).
- Add a test for #55655. #63380 (Alexey Milovidov).
- Fix data race in
reportBrokenPart
. #63396 (Antonio Andelic). - Workaround for
oklch()
inside canvas bug for firefox. #63404 (Sergei Trifonov). - Add test for issue #47862. #63424 (Robert Schulze).
- Fix parsing of
CREATE INDEX
query. #63425 (Anton Popov). - We are using Shared Catalog in the CI Logs cluster. #63442 (Alexey Milovidov).
- Fix collection of coverage data in the CI Logs cluster. #63453 (Alexey Milovidov).
- Fix flaky test for rocksdb bulk sink. #63457 (Duc Canh Le).
- Extra constraints for stress and fuzzer tests. #63470 (Raúl Marín).
- io_uring: refactor get reader from context. #63475 (Tomer Shafir).
- Analyzer setting max_streams_to_max_threads_ratio overflow fix. #63478 (Maksim Kita).
- Provides setting
output_format_pretty_preserve_border_for_multiline_string
which allows to render multiline strings in pretty format better. The default value for this setting is true. #63479 (Yarik Briukhovetskyi). - Fix logical error when reloading config with customly created web disk broken after #56367. #63484 (Kseniia Sumarokova).
- Add test for #49307. #63486 (Anton Popov).
- Remove leftovers of GCC support in cmake rules. #63488 (Azat Khuzhin).
- Fix ProfileEventTimeIncrement code. #63489 (Azat Khuzhin).
- MergeTreePrefetchedReadPool: Print parent name when logging projection parts. #63522 (Raúl Marín).
- Correctly stop
asyncCopy
tasks in all cases. #63523 (Antonio Andelic). - Almost everything should work on AArch64 (Part of #58061). #63527 (Alexey Milovidov).
- Update randomization of
old_parts_lifetime
. #63530 (Alexander Tokmakov). - Update 02240_system_filesystem_cache_table.sh. #63531 (Kseniia Sumarokova).
- Fix data race in
DistributedSink
. #63538 (Antonio Andelic). - Fix azure tests run on master. #63540 (alesapin).
- The commit
2b8254f987
was not synced into the cloud because it was falsely marked as a success byupstream_pr.head.sha
. Here we'll try our best to find a proper commit, and won't make anything if we can't. #63543 (Mikhail f. Shiryaev). - Add
no-s3-storage
tag to local_plain_rewritable ut. #63546 (Julia Kartseva). - Add
jwcrypto
to integration tests runner. #63551 (Konstantin Bogdanov). - Go back to upstream lz4. #63574 (Raúl Marín).
- Fix logical error in ColumnTuple::tryInsert(). #63583 (Michael Kolupaev).
- harmonize sumMap error messages on ILLEGAL_TYPE_OF_ARGUMENT. #63619 (Yohann Jardin).
- Refactor data part writer to remove dependencies on MergeTreeData and DataPart. #63620 (Alexander Gololobov).
- Update README.md. #63631 (Tyler Hannan).
- Ignore global profiler if system.trace_log is not enabled and fix really disable it for keeper standalone build. #63632 (Azat Khuzhin).
- Fixes for 00002_log_and_exception_messages_formatting. #63634 (Azat Khuzhin).
- Fix 02362_part_log_merge_algorithm flaky test. #63635 (Miсhael Stetsyuk).
- Fix tests flakiness due to long SYSTEM FLUSH LOGS (explicitly specify old_parts_lifetime). #63639 (Azat Khuzhin).
- Update clickhouse-test help section. #63663 (Ali).
- Fix bad test
02950_part_log_bytes_uncompressed
. #63672 (Alexey Milovidov). - Remove leftovers of
optimize_monotonous_functions_in_order_by
. #63674 (Nikita Taranov). - tests: attempt to fix 02340_parts_refcnt_mergetree flakiness. #63684 (Azat Khuzhin).
- Parallel replicas: simple cleanup. #63685 (Igor Nikonov).
- Cancel S3 reads properly when parallel reads are used. #63687 (Antonio Andelic).
- Explaining insertion order of the Map datatype. #63690 (Mark Needham).
- selectRangesToRead() simple cleanup. #63692 (Igor Nikonov).
- Fix fuzzed analyzer_join_with_constant query. #63702 (Nikolai Kochetov).
- Add missing explicit instantiations of ColumnUnique. #63718 (Raúl Marín).
- Better asserts in ColumnString.h. #63719 (Raúl Marín).
- Try to fix flaky s3 tests test_seekable_formats and test_seekable_formats_url. #63720 (Kruglov Pavel).
- Don't randomize some settings in 02941_variant_type_* tests to avoid timeouts. #63721 (Kruglov Pavel).
- Fix flaky 03145_non_loaded_projection_backup.sh. #63728 (Kseniia Sumarokova).
- Userspace page cache: don't collect stats if cache is unused. #63730 (Michael Kolupaev).
- Fix insignificant UBSAN error in QueryAnalyzer::replaceNodesWithPositionalArguments(). #63734 (Michael Kolupaev).
- Fix a bug in resolving matcher inside lambda inside ARRAY JOIN. #63744 (Nikolai Kochetov).
- Self explanatory. #63754 (Arthur Passos).
- Do not hide disk name. #63756 (Kseniia Sumarokova).
- CI: remove Cancel and Debug workflows as redundant. #63757 (Max K.).
- Security Policy: Add notification process. #63773 (Leticia Webb).
- Fix typo. #63774 (Anton Popov).
- Fix fuzzer when only explicit faults are used. #63775 (Raúl Marín).
- Settings typo. #63782 (Rory Crispin).
- Ref. #63479. #63783 (Yarik Briukhovetskyi).
- Fix test_odbc_interaction from aarch64 #61457. #63787 (alesapin).
- Fix test
test_catboost_evaluate
for aarch64. #61457. #63789 (alesapin). - Rewrite plan for parallel replicas in Planner. #63796 (Igor Nikonov).
- Follow-up for the
binary_symbols
table in CI. #63802 (Alexey Milovidov). - Support INSERT with VALUES in the ANTLR syntax file. #63811 (GG Bond).
- Fix race in
ReplicatedMergeTreeLogEntryData
. #63816 (Antonio Andelic). - Allow allocation during job destructor in
ThreadPool
. #63829 (Antonio Andelic). - Remove HDFS from disks config for one integration test for arm. #61457. #63832 (alesapin).
- io_uring: add basic io_uring clickhouse perf test. #63835 (Tomer Shafir).
- Bump version for old image in test_short_strings_aggregation to make it work on arm. #61457. #63836 (alesapin).
- fix typo. #63838 (Alexander Gololobov).
- Disable test
test_non_default_compression/test.py::test_preconfigured_deflateqpl_codec
on arm. #61457. #63839 (alesapin). - This PR was reverted. #63857 (Sema Checherinda).
- Remove unnecessary logging statements in MergeJoinTransform.cpp. #63860 (vdimir).
- Temporary disables 3 integration tcs on arm until https://github.com/clickhouse/clickhouse/issues/63855 is resolved. #63867 (Max K.).
- Fix some settings values in 02455_one_row_from_csv_memory_usage test to make it less flaky. #63874 (Kruglov Pavel).
- Randomise
allow_experimental_parallel_reading_from_replicas
in stress tests. #63899 (Nikita Taranov). - Fix logs test for binary data by converting it to a valid UTF8 string. #63909 (Alexey Katsman).
- More sanity checks for parallel replicas. #63910 (Nikita Taranov).
- Include checks like
Stateless tests (asan, distributed cache, meta storage in keeper, s3 storage) [2/3]
inMergeable Check
andA Sync
. #63945 (Mikhail f. Shiryaev). - Insignificant libunwind build fixes. #63946 (Azat Khuzhin).
- Revert multiline pretty changes due to performance problems. #63947 (Raúl Marín).
- Some usability improvements for c++expr script. #63948 (Azat Khuzhin).
- Fix 02124_insert_deduplication_token_multiple_blocks. #63950 (Han Fei).
- CI: aarch64: disable arm integration tests with kerberaized kafka. #63961 (Max K.).
- Make events like timeouts visible in CI DB. #63982 (Mikhail f. Shiryaev).
- Slightly better setting
force_optimize_projection_name
. #63997 (Anton Popov). - chore(ci-workers): remove reusable from tailscale key. #63999 (Gabriel Martinez).
- Better script to collect symbols statistics. #64013 (Alexey Milovidov).
- Fix a typo in Analyzer. #64022 (Alexey Milovidov).
- Fix libbcrypt for FreeBSD build. #64023 (Azat Khuzhin).
- Remove some unnecessary
UNREACHABLE
s. #64035 (Robert Schulze). - Add
ClickHouseVersion.copy
method. Create a branch release in advance without spinning out the release to increase the stability. #64039 (Mikhail f. Shiryaev). - Fix searching for libclang_rt.builtins.*.a on FreeBSD. #64051 (Azat Khuzhin).
- The mime type is not 100% reliable for Python and shell scripts without shebangs; add a check for file extension. #64062 (Mikhail f. Shiryaev).
- Fix waiting for mutations with retriable errors. #64063 (Alexander Tokmakov).
- harmonize h3PointDist* error messages. #64080 (Yohann Jardin).
- This log message is better in Trace. #64081 (Alexey Milovidov).
- Prevent stack overflow in Fuzzer and Stress test. #64082 (Alexey Milovidov).
- tests: fix expected error for 03036_reading_s3_archives (fixes CI). #64089 (Azat Khuzhin).
- Fix sanitizers. #64090 (Azat Khuzhin).
- Update llvm/clang to 18.1.6. #64091 (Azat Khuzhin).
- Set green Mergeable Check status only after all required checks are passed with success - All non-required checks are started at stage Test_3 when all required checks are passed in Test_1/2. #64093 (Max K.).
- Move
isAllASCII
from UTFHelper to StringUtils. #64108 (Robert Schulze). - Throw out some
inline
s. #64110 (Robert Schulze). - Clean up .clang-tidy after transition to Clang 18. #64111 (Robert Schulze).
- Ignore exception when checking for cgroupsv2. #64118 (Robert Schulze).
- Add retries in git submodule update. #64125 (Alexey Milovidov).
- See https://s3.amazonaws.com/clickhouse-test-reports/63946/86cf1e13d866333b8a511badd7f2fe186d810646/ast_fuzzer__ubsan_.html. #64127 (Alexey Milovidov).
- Refactoring of Server.h: Isolate server management from other logic. #64132 (TTPO100AJIEX).
- Syncing code. #64135 (Antonio Andelic).
- Losen build resource limits for unusual architectures. #64152 (Alexey Milovidov).
- fix clang tidy. #64179 (Han Fei).
- Fix: 02124_insert_deduplication_token_multiple_blocks_replica. #64181 (Igor Nikonov).
- Fix global query profiler. #64187 (Azat Khuzhin).
- CI: cancel running PR wf after adding to MQ. #64188 (Max K.).
- Add profile events for number of rows read during/after prewhere. #64198 (Nikita Taranov).
- Add debug logging to EmbeddedRocksDBBulkSink. #64203 (vdimir).
- Fix special builds (due to excessive resource usage - memory/CPU). #64204 (Azat Khuzhin).
- Update InterpreterCreateQuery.cpp. #64207 (Alexander Tokmakov).
- Remove unused storage_snapshot field from MergeTreeSelectProcessor. #64217 (Alexander Gololobov).
- Add test for #37090. #64220 (Yarik Briukhovetskyi).
- Small cli tool. #64227 (Nikita Mikhaylov).
- Make
settings_changes_history
const. #64230 (János Benjamin Antal). - test for #45804. #64245 (Denny Crane).
- Update version after release. #64283 (Raúl Marín).
- Followup for #63691. #64285 (vdimir).
- CI: dependency fix for changelog.py. #64293 (Max K.).
- Print query in explain plan with parallel replicas. #64298 (vdimir).
- CI: Cancel sync wf on new push. #64299 (Max K.).
- CI: master workflow with folded jobs. #64340 (Max K.).
- CI: Sync, Merge check, CI gh's statuses fixes. #64348 (Max K.).
- Enable 02494_query_cache_nested_query_bug for Analyzer. #64357 (Robert Schulze).
- Rename allow_deprecated_functions to allow_deprecated_error_prone_window_functions. #64358 (Raúl Marín).
- Change input_format_parquet_use_native_reader to 24.6. #64359 (Raúl Marín).
- Update description for settings
cross_join_min_rows_to_compress
andcross_join_min_bytes_to_compress
. #64360 (Nikita Fomichev). - Changed the unreleased setting
aggregate_function_group_array_has_limit_size
toaggregate_function_group_array_action_when_limit_is_reached
. #64362 (Raúl Marín). - Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. #64363 (Kruglov Pavel).
- Try to fix GWPAsan. #64365 (Antonio Andelic).
- CI: add secrets to reusable stage wf yml. #64366 (Max K.).
- Do not run tests tagged 'no-s3-storage-with-slow-build' with ASan. #64367 (vdimir).
- This change was reverted. #64386 (Francisco J. Jurado Moreno).
- Update s3queue.md. #64389 (Kseniia Sumarokova).
- test for #64211. #64390 (Denny Crane).
- Follow-up to #59767. #64398 (Kseniia Sumarokova).
- Remove wrong comment. #64403 (Sergei Trifonov).
- Follow up to #59767. #64404 (Kseniia Sumarokova).
- Refactor s3 settings (move settings parsing into single place). #64412 (Kseniia Sumarokova).
- This PR was reverted. #64423 (Sergei Trifonov).
- Fix test after #64404. #64432 (Kseniia Sumarokova).
- Faster TestKeeper shutdown. #64433 (Alexander Gololobov).
- Remove some logging. #64434 (Kseniia Sumarokova).
- Revert "Revert "Remove some unnecessary UNREACHABLEs"". #64435 (Robert Schulze).
- Clean settings in 02943_variant_read_subcolumns test. #64437 (Kruglov Pavel).
- Add a comment after #64226. #64449 (Nikolai Kochetov).
- CI: fix build_report selection in case of job reuse. #64459 (Max K.).
- Add Critical bugfix category in PR template. #64480 (Max K.).
- Remove
generateSnowflakeIDThreadMonotonic
. #64499 (Robert Schulze). - Move analyzer attempt 2. #64500 (Nikolai Kochetov).
- Sync some code back from internal to public repository. #64502 (Robert Schulze).
- Remove
generateUUIDv7(NonMonotonic|ThreadMonotonic)
functions. #64506 (Robert Schulze). - Fix bash completion for settings. #64521 (Azat Khuzhin).
- Use max_read_buffer_size for file descriptors as well in file(). #64532 (Azat Khuzhin).
- Temporarily disable
enable_vertical_final
setting by default. This feature should not be used in older releases because it might crash, but it's already fixed in 24.6 where this setting change has been reverted andenable_vertical_final
is again enabled by default. #64544 (Alexander Tokmakov). - Removed excessive calls to
flush logs
and disabled under sanitizers. #64550 (Nikita Taranov). - Sync code moved in private repo back back to public repo. #64551 (Robert Schulze).
- Add support for custom type to ASTLiteral, or else the type may be lost when parse the ast. E.g. set a ASTLiteral to DataTime32 with value 19870, then it will be parsed to Int16. #64562 (shuai.xu).
- Add a temporary known host for git over ssh. #64569 (Mikhail f. Shiryaev).
- Cache first analysis result in ReadFromMergeTree. #64579 (Igor Nikonov).
- Derive script parameters (labels) from the --repo/--from-repo - fix to not create backports for all release branches if backport for specific branch only. #64603 (Max K.).
- CI fixes. #64605 (Max K.).
- Double-checking #59318 and docs for
Map
. #64606 (Robert Schulze). - Update CHANGELOG.md. #64609 (Alexey Milovidov).
- Tests: Convert numeric to symbolic error codes. #64635 (Robert Schulze).
- Move NamedCollectionsFactory into a separate file. #64642 (Kseniia Sumarokova).
- Shuffle tests for parallel execution. #64646 (Nikita Mikhaylov).
- CI: Do not upload binaries for special builds in PRs. #64653 (Max K.).
- Update changelog. #64654 (Robert Schulze).
- Parallel replicas: simple cleanup. #64655 (Igor Nikonov).
- Be more graceful with existing tables with
inverted
indexes. #64656 (Robert Schulze). - CI: Build Report Check to verify only enabled builds. #64669 (Max K.).
- Tests: Convert error numbers to symbolic error codes, pt. II. #64670 (Robert Schulze).
- Split query analyzer. #64672 (Nikolai Kochetov).
- By the end of CI, CI_Running status must be SUCCESS or FAILURE never PENDING. #64693 (Max K.).
- The following list of merged PRs is not present in the release branch and was added to the changelog by mistake:. #64704 (Nikita Mikhaylov).
- CI: MergeQueue: add binary_release and unit tests. #64705 (Max K.).
- Fix to get first good enough GH token instead of getting and comparing all of them. #64709 (Max K.).
- Check for missing Upload ID in CreateMultipartUpload reply. #64714 (Michael Kolupaev).
- Update version_date.tsv and changelogs after v24.5.1.1763-stable. #64715 (robot-clickhouse).
- Fix (unreleased)
loop()
table function crashing on empty table name. #64716 (Michael Kolupaev). - Update CHANGELOG.md. #64730 (Alexey Milovidov).
- CI: ci.py refactoring. #64734 (Max K.).
- Return the explanation for session moved error. #64747 (Antonio Andelic).
- Adjust the version_helper and script to a new release scheme. #64759 (Mikhail f. Shiryaev).
- Do not try to write columns.txt if it does not exist for write-once storages. #64762 (Azat Khuzhin).
- Update 02482_load_parts_refcounts.sh. #64765 (Alexander Tokmakov).
- Fix crash with DISTINCT and window functions. #64767 (Igor Nikonov).
- Fix assert in IObjectStorageIteratorAsync. #64770 (Michael Kolupaev).
- Make table functions always report engine 'StorageProxy' in system.tables. #64771 (Michael Kolupaev).
- Ask about company name on GitHub. #64774 (Alexey Milovidov).
- Fix flaky tests about SQLite. #64776 (Alexey Milovidov).
- Remove iostream debug helpers. #64777 (Alexey Milovidov).
- Remove unnecessary comment. #64785 (Raúl Marín).
- Follow-ups to some PRs. #64787 (Mikhail f. Shiryaev).
- Attempt to fix 02228_merge_tree_insert_memory_usage.sql flakiness for s3. #64800 (Raúl Marín).
- Add regression test for filter propagation through
Merge
engine. #64806 (Nikita Taranov). - Migrate changelog.py to a descendant of fuzzywuzzy. #64807 (Mikhail f. Shiryaev).
- A follow-up for https://github.com/ClickHouse/ClickHouse/pull/64039 and #64759. #64813 (Mikhail f. Shiryaev).
- Make row order optimization non-experimental. #64814 (Robert Schulze).
- Didn't catch it at the time when all versions belonged to the current year. #64817 (Mikhail f. Shiryaev).
- Fix clang-tidy build. #64823 (Robert Schulze).
- Sets all builds that we run tests on to normal build list. #64824 (Max K.).
- CI: fix CI await feature. #64825 (Max K.).
- Fix clang-tidy. #64827 (Kseniia Sumarokova).
- Upload blob_storage_log from stateless tests. #64843 (alesapin).
- Follow-up to #64349. #64845 (Kseniia Sumarokova).
- Simplify handling of old 'inverted' indexes. #64846 (Robert Schulze).
- Use issue templates defined in YAML provide more user-friendly experience. #64850 (Nikita Mikhaylov).
- Handle logs from rocksdb by ClickHouse internal logging. #64856 (Azat Khuzhin).
- Follow-up for https://github.com/ClickHouse/ClickHouse/pull/59357. #64860 (Nikita Mikhaylov).
- added mlock and mlockall to aspell-dict to be ignored. #64863 (Ali).
- A tiny fix for fancy quotes. #64883 (Alexey Milovidov).
- Fix possible loss of "Query was cancelled" message in client. #64888 (Azat Khuzhin).
- We accidentally lost the way to set
PR Check
failure at some point. #64890 (Mikhail f. Shiryaev). - Fix global trace collector. #64896 (Antonio Andelic).
- Fix test_mask_sensitive_info/test.py::test_create_table. #64901 (Azat Khuzhin).
- Update 03165_string_functions_with_token_text_indexes.sql. #64903 (Alexander Tokmakov).
- When the branch is removed, it's impossible to get the diff by the labels.
print
in imported files spoils theipython
output. #64904 (Mikhail f. Shiryaev). - Disable transactions for unsupported storages even for materialized v…. #64918 (alesapin).
- additional log for cleanupDetachedTables. #64919 (Konstantin Morozov).
- Fix tupleConcat of two empty tuples. This fixes #64885. #64923 (Amos Bird).
- CI: Minor fixes in ci scripts. #64950 (Max K.).
- Fix error message (it was strange). #64952 (Alexey Milovidov).
- Update fmtlib version to 9.1.0. #64959 (Duc Canh Le).
- Test 02908_many_requests_to_system_replicas makes a lot of heavy requests and it overloads server if it's an ASAN build. #64966 (Alexander Gololobov).
- Fix (unreleased) bug in short circuit evaluation. #64967 (Raúl Marín).
- Update version_date.tsv and changelogs after v24.4.2.141-stable. #64968 (robot-clickhouse).
- Fix
test_attach_partition_using_copy
. #64977 (Alexander Tokmakov). - Faster processing of scheduler queue activations. #64985 (Sergei Trifonov).
- CI: Fix nightly workflow. #64987 (Max K.).
- Fix innocuous data race in detectLanguage. #64988 (Raúl Marín).
- CI: Builds in CI settings. #64994 (Max K.).
- REVERTED. #65009 (Yarik Briukhovetskyi).
- CI: Fix backports. #65010 (Max K.).
- Try fix 03143_prewhere_profile_events. #65014 (Nikita Taranov).
- Fix 03165_string_functions_with_token_text_indexes. #65018 (Julia Kartseva).
- This change was reverted. #65028 (Sergei Trifonov).
- Bump googletest to latest HEAD. #65038 (Robert Schulze).
- Improve comment about AsynchronousMetrics. #65040 (Antonio Andelic).
- CI: Remove fuzzer build from normal CI run (bugfix). #65041 (Max K.).
- CI config refactoring. #65045 (Max K.).
- Bump abseil to latest HEAD. #65048 (Robert Schulze).
- Capture weak_ptr of ContextAccess for safety. #65051 (Alexander Gololobov).
- Stateless tests: add test for SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT. #65056 (Nikita Fomichev).
- Increase timeout in wait_for_all_mutations. #65058 (Alexander Gololobov).
- Tests for _time virtual column in file alike storages. #65064 (Ilya Golshtein).
- Update odbc-bridge.md. #65099 (Alexander Gololobov).
- Small fix for 02340_parts_refcnt_mergetree. #65105 (Nikita Taranov).
- Re-enable OpenSSL session caching. #65111 (Robert Schulze).
- Update test_replicated_database/test.py. #65112 (Alexander Tokmakov).
- Fix false positives leaky memory warnings in OpenSSL. #65125 (Robert Schulze).
- Fix
Initiator received more initial requests than there are replicas
withloop
engine. #65133 (Nikita Taranov). - Fix 'Tasks in BackgroundSchedulePool cannot throw' caused by MergeTreeData::loadUnexpectedDataParts(). #65135 (Michael Kolupaev).
- Fix bad error message. #65137 (Alexey Milovidov).
- Just fixing flaky unit tests. #65152 (Sema Checherinda).
- This change was reverted. #65164 (Alexey Milovidov).
- Ensure submodules are named consistently. #65167 (Robert Schulze).
- Remove obsolete fix from aws submodule. #65168 (Robert Schulze).
- CI: Fix not-merged cherry-picks for backports. #65181 (Max K.).
- Add an assertion in ReplicatedMergeTreeQueue. #65184 (Alexander Tokmakov).
- Fix bug in unreleased code. #65185 (Raúl Marín).
- Fix docs for skipping-indexes.md. #65194 (morning-color).
- Fix the descriptions of some server settings. #65200 (Raúl Marín).
- Fix issue after #64813 with broken search in the changelog, and missing zstd in a style-check image. #65202 (Mikhail f. Shiryaev).
- Fix bug in unreleased code. #65203 (Raúl Marín).
- Add test prewhere merge. #65207 (Nikolai Kochetov).
- Sync ProfileEvents.h. #65208 (Kseniia Sumarokova).
- FinishCheck to set failure if workflow failed. #65228 (Max K.).
- Update version_date.tsv and changelogs after v24.3.4.147-lts. #65235 (robot-clickhouse).
- Update version_date.tsv and changelogs after v24.5.3.5-stable. #65240 (robot-clickhouse).
- Fails sometimes for debug build https://s3.amazonaws.com/clickhouse-test-reports/0/af6afd904316bfb771737faa147ce8aea72dd705/stateless_tests__debug__[4_5].html. #65245 (Antonio Andelic).
- Fix libunwind in CI. #65247 (Alexey Milovidov).
- CI: Do not skip FinishCheck in Merge Queue. #65249 (Max K.).
- Add a test just in case. #65271 (Alexey Milovidov).
- Disable 02581_share_big_sets_between_multiple_mutations_tasks_long in coverage run. #65295 (Alexander Gololobov).
- Update version_date.tsv and changelogs after v23.8.15.35-lts. #65300 (robot-clickhouse).
- mute test test_query_is_canceled_with_inf_retries. #65301 (Sema Checherinda).
- Fix silly typo that caused wrong tags messages. #65307 (Mikhail f. Shiryaev).
- Save server data for failed stateless tests. #65309 (Alexander Tokmakov).
- Fix 01246_buffer_flush flakiness (by tuning timeouts). #65310 (Azat Khuzhin).
- Remove outdated override in stress tests. #65323 (Alexey Milovidov).
- Fix bad code in
system.session_log
. #65332 (Alexey Milovidov). - add tests for 'boom filter index with map'. #65333 (iceFireser).
- Fix crash in 03036_dynamic_read_subcolumns. #65341 (Kruglov Pavel).
- Move tests 02942_variant_cast and 02944_variant_as_common_type to analyzer_tech_debt.txt. #65342 (Kruglov Pavel).
- REVERTED. #65384 (Yarik Briukhovetskyi).
- CI: Add Non-blocking (Woolen wolfdog) CI mode. #65385 (Max K.).
- Fix compatibility release check. #65394 (Alexey Katsman).
- Move a leaksan suppression from Poco into OpenSSL. #65396 (Robert Schulze).
- Fix tidy build. #65415 (Sergei Trifonov).
- Remove Tests dependency on Builds_2. No tests depend on Builds_2. #65416 (Max K.).
- CI: PR workflow dependencies fix. #65442 (Max K.).
- Fix test_storage_s3_queue/test.py::test_max_set_age. #65452 (Kseniia Sumarokova).
- CI: Rename A Sync status. #65456 (Max K.).
- CI: Rename sync status. #65464 (Max K.).
- This change was reverted. #65466 (Sergei Trifonov).
- Remove a feature wasn't part of any release yet. #65480 (Raúl Marín).
- Backported in #65657: Fix of
PlanSquashingTransform
: pipeline stuck. #65487 (Yarik Briukhovetskyi). - Backported in #65504: Fix bad test
02922_deduplication_with_zero_copy
. #65492 (Alexey Milovidov). - Backported in #65591: Setting
uniform_snowflake_conversion_functions
(not in any release yet) was replaced by settingallow_deprecated_snowflake_conversion_functions
. The latter controls if the legacy snowflake conversion functions are available (by default, they are not). #65522 (Robert Schulze). - Backported in #65759: Renames Build report jobs. #65554 (Max K.).
- Backported in #65773:
base64En/Decode64Url
-->base64En/Decode64URL
. #65760 (Robert Schulze). - Backported in #65805: CI: Fix for Builds report job in backports and releases. #65774 (Max K.).