mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
100 KiB
100 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.8.1.2684-lts (161c62fd29
) FIXME as compared to v24.8.1.1-new (3f8b27d7ac
)
Backward Incompatible Change
clickhouse-client
andclickhouse-local
now default to multi-query mode (instead single-query mode). As an example,clickhouse-client -q "SELECT 1; SELECT 2"
now works, whereas users previously had to add--multiquery
(or-n
). The--multiquery/-n
switch became obsolete. INSERT queries in multi-query statements are treated specially based on their FORMAT clause: If the FORMAT isVALUES
(the most common case), the end of the INSERT statement is represented by a trailing semicolon;
at the end of the query. For all other FORMATs (e.g.CSV
orJSONEachRow
), the end of the INSERT statement is represented by two newlines\n\n
at the end of the query. #63898 (FFish).- In previous versions, it was possible to use an alternative syntax for
LowCardinality
data types by appendingWithDictionary
to the name of the data type. It was an initial working implementation, and it was never documented or exposed to the public. Now, it is deprecated. If you have used this syntax, you have to ALTER your tables and rename the data types toLowCardinality
. #66842 (Alexey Milovidov). - Fix logical errors with storage
Buffer
used with distributed destination table. It's a backward incompatible change: queries usingBuffer
with a distributed destination table may stop working if the table appears more than once in the query (e.g., in a self-join). #67015 (vdimir). - In previous versions, calling functions for random distributions based on the Gamma function (such as Chi-Squared, Student, Fisher) with negative arguments close to zero led to a long computation or an infinite loop. In the new version, calling these functions with zero or negative arguments will produce an exception. This closes #67297. #67326 (Alexey Milovidov).
- The system table
text_log
is enabled by default. This is fully compatible with previous versions, but you may notice subtly increased disk usage on the local disk (this system table takes a tiny amount of disk space). #67428 (Alexey Milovidov). - In previous versions,
arrayWithConstant
can be slow if asked to generate very large arrays. In the new version, it is limited to 1 GB per array. This closes #32754. #67741 (Alexey Milovidov). - Fix REPLACE modifier formatting (forbid omitting brackets). #67774 (Azat Khuzhin).
- Backported in #68349: Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. #68132 (Kruglov Pavel).
New Feature
- Add new experimental Kafka storage engine to store offsets in Keeper instead of relying on committing them to Kafka. #57625 (János Benjamin Antal).
- Add new TimeSeries table engine: - by default:
CREATE TABLE tbl ENGINE=TimeSeries
- or with specifying engines of its internal tables:. #64183 (Vitaly Baranov). - Support more join strictnesses (
LEFT/RIGHT SEMI/ANTI/ANY JOIN
) with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y (see settingallow_experimental_join_condition
). #64281 (lgbo). - Add
_etag
virtual column for S3 table engine. Fixes #65312. #65386 (skyoct). - This pull request introduces Hive-style partitioning for different engines (
File
,URL
,S3
,AzureBlobStorage
,HDFS
). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). #65997 (Yarik Briukhovetskyi). - Add function printf for spark compatiability. #66257 (李扬).
- Backported in #68450: Implement new JSON data type. #66444 (Kruglov Pavel).
- Add a new server setting: disable_insertion_and_mutation Set it to true. This node will deny all insertions and mutations(Alter table delete/update/drop partition). Include async insertion. #66519 (Xu Jia).
- Add options
restore_replace_external_engines_to_null
andrestore_replace_external_table_functions_to_null
to replace external engines and table_engines to Null engine that can be useful for testing. It should work for RESTORE and explicit table creation. #66536 (Ilya Yatsishin). - Added support for reading MULTILINESTRING geometry in WKT format using function readWKTLineString. #67647 (Jacob Reckhard).
- Add a new table function
fuzzQuery
. This function allows the modification of a given query string with random variations. Example:SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;
. #67655 (pufit). - Support query
DROP DETACHED PARTITION ALL
to drop all detached partitions. #67885 (Duc Canh Le). - Added a tagging (namespace) mechanism for the query cache. The same queries with different tags are considered different by the query cache. Example:
SELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'abc'
andSELECT 1 SETTINGS use_query_cache = 1, query_cache_tag = 'def'
now create different query cache entries. #68235 (sakulali).
Performance Improvement
- Use adaptive read task size calculation method (adaptive meaning it depends on read column sizes) for parallel replicas. #60377 (Nikita Taranov).
- Store the
plain_rewritable
disk directory metadata in__meta
layout, separately from the merge tree data in the object storage. Move theplain_rewritable
disk to a flat directory structure. #65751 (Julia Kartseva). - Enable
compile_expressions
(JIT compiler for fragments of ordinary expressions) by default. This closes #51264 and #56386. #66486 (Alexey Milovidov). - Improve columns squashing for String/Array/Map/Variant/Dynamic types by reserving required memory in advance for all subcolumns. #67043 (Kruglov Pavel).
- Speed up system flush logs, flush logs on shutdown. #67472 (Sema Checherinda).
- Backported in #68496: Improved overall performance of merges by reducing the overhead of scheduling steps of merges. #68016 (Anton Popov).
- Setting
optimize_functions_to_subcolumns
is enabled by default. #68053 (Anton Popov).
Improvement
- ClickHouse server now supports new setting
max_keep_alive_requests
. For keep-alive HTTP connections to the server it works in tandem withkeep_alive_timeout
- if idle timeout not expired but there already more thanmax_keep_alive_requests
requests done through the given connection - it will be closed by the server. #61793 (Nikita Taranov). - As in the new version, SOURCES are checked based on Table Engine logic, even grant table engine is disabled by default, if a source is not granted, a prompt of table engine would popup instead, which is misleading. #65419 (jsc0218).
- Added statistics type
count_min
(count-min sketches) which provide selectivity estimations for equality predicates likecol = 'val'
. Supported data types are string, date, datatime and numeric types. #65521 (JackyWoo). - Do not pass logs for keeper explicitly in the image to allow overriding. #65564 (Azat Khuzhin).
- Use
Atomic
database by default inclickhouse-local
. Address items 1 and 5 from #50647. Closes #44817. #65860 (Alexey Milovidov). - Add the
rows_before_aggregation_at_least
statistic to the query response whenrows_before_aggregation
is enabled. This statistic represents the number of rows read before aggregation. In the context of a distributed query, when using thegroup by
ormax
aggregation function without alimit
,rows_before_aggregation_at_least
can reflect the number of rows hit by the query. #66084 (morning-color). - Introduced
use_same_password_for_base_backup
settings forBACKUP
andRESTORE
queries, allowing to create and restore incremental backups to/from password protected archives. #66214 (Samuele). - Ignore async_load_databases for ATTACH query (previously it was possible for ATTACH to return before the tables had been attached). #66240 (Azat Khuzhin).
- [Replicated]MergeTreeSink has to properly cancel its delayed_chunk on
onCancel()
method. #66279 (Sema Checherinda). - Added logs and metrics for rejected connections (where there are not enough resources). #66410 (Alexander Tokmakov).
- Setting
allow_experimental_analyzer
is renamed toenable_analyzer
. The old name is preserved in a form of an alias. #66438 (Nikita Mikhaylov). - Support true UUID type for MongoDB engine. #66671 (Azat Khuzhin).
- Added a new
MergeTree
settingdeduplicate_merge_projection_mode
to control the projections during merges (for specific engines) andOPTIMIZE DEDUPLICATE
query. Supported options:throw
(throw an exception in case the projection is not fully supported for *MergeTree engine),drop
(remove projection during merge if it can't be merged itself consistently) andrebuild
(rebuild projection from scratch, which is a heavy operation). #66672 (jsc0218). - Add replication lag and recovery time metrics. #66703 (Miсhael Stetsyuk).
- Add S3DiskNoKeyErrors metric. #66704 (Miсhael Stetsyuk).
- Ensure COMMENT clause works for all table engines. #66832 (Joe Lynch).
- Update the usage of error code
BAD_ARGUMENTS
andILLEGAL_TYPE_OF_ARGUMENT
by more accurate error codes when appropriate. #66851 (Yohann Jardin). - Function
mapFromArrays
now acceptsMap(K, V)
as first argument, for example:SELECT mapFromArrays(map('a', 4, 'b', 4), ['aa', 'bb'])
now works and returns{('a',4):'aa',('b',4):'bb'}
. Also, if the 1st argument is an Array, it can now also be of typeArray(Nullable(T))
orArray(LowCardinality(Nullable(T)))
as long as the actual array values are notNULL
. #67103 (李扬). - Read configuration for clickhouse-local from ~/.clickhouse-local. #67135 (Azat Khuzhin).
- Rename setting
input_format_orc_read_use_writer_time_zone
toinput_format_orc_reader_timezone
and allow the user to set the reader timezone. #67175 (kevinyhzou). - Decrease level of 'Socket is not connected' error when HTTP connection immediately reset by peer after connecting, close #34218. #67177 (vdimir).
- Speed up tables removal for
DROP DATABASE
query, increased the default value fordatabase_catalog_drop_table_concurrency
to 16. #67228 (Nikita Mikhaylov). - Add ability to load dashboards for system.dashboards from config (once set, they overrides the default dashboards preset). #67232 (Azat Khuzhin).
- The window functions in SQL are traditionally in snake case. ClickHouse uses camelCase, so new aliases
denseRank()
andpercentRank()
have been created. These new functions can be called the exact same as the originaldense_rank()
andpercent_rank()
functions. Both snake case and camelCase syntaxes remain usable. A new test for each of the functions has been added as well. This closes #67042 . #67334 (Peter Nguyen). - Autodetect configuration file format if is not .xml, .yml or .yaml. If the file begins with < it might be XML, otherwise it might be YAML. Non regular file just parse as XML such as PIPE: /dev/fd/X. #67391 (sakulali).
- Functions
formatDateTime
andformatDateTimeInJodaSyntax
now treat their format parameter as optional. If it is not specified, format strings%Y-%m-%d %H:%i:%s
andyyyy-MM-dd HH:mm:ss
are assumed. Example:SELECT parseDateTime('2021-01-04 23:12:34')
now returns DateTime value2021-01-04 23:12:34
(previously, this threw an exception). #67399 (Robert Schulze). - Automatically retry Keeper requests in KeeperMap if they happen because of timeout or connection loss. #67448 (Antonio Andelic).
- Rework usage of custom table's disks. #67684 (Sema Checherinda).
- Various improvements in the advanced dashboard. This closes #67697. This closes #63407. This closes #51129. This closes #61204. #67701 (Alexey Milovidov).
- Avoid allocate too much capacity for array column while writing orc & some minor refactors to make code cleaner. Performance speeds up 15% for array column. #67879 (李扬).
- Support OPTIMIZE query on Join table engine to reduce Join tables memory footprint. #67883 (Duc Canh Le).
- Add replication lag and recovery time metrics. #67913 (Miсhael Stetsyuk).
- Add '-no-pie' to aarch64 Linux builds to allow proper introspection and symbolizing of stacktraces after a ClickHouse restart. #67916 (filimonov).
- Backported in #68481: Added profile events for merges and mutations for better introspection. #68015 (Anton Popov).
- Fix settings/current_database in system.processes for async BACKUP/RESTORE. #68163 (Azat Khuzhin).
- Remove unnecessary logs for MergeTree that doesn't support replication. #68238 (Daniil Ivanik).
- Backported in #68430: Improve schema inference of date times. Now DateTime64 used only when date time has fractional part, otherwise regular DateTime is used. Inference of Date/DateTime is more strict now, especially when
date_time_input_format='best_effort'
to avoid inferring date times from strings in corner cases. #68382 (Kruglov Pavel).
Bug Fix (user-visible misbehavior in an official stable release)
- Fixed
Not-ready Set
in some system tables when filtering using subqueries. #66018 (Michael Kolupaev). - Fixed reading of subcolumns after
ALTER ADD COLUMN
query. #66243 (Anton Popov). - Fix boolean literals in query sent to external database (for engines like
PostgreSQL
). #66282 (vdimir). - Fix formatting of query with aliased JOIN ON expression, e.g.
... JOIN t2 ON (x = y) AS e ORDER BY x
should be formatted as... JOIN t2 ON ((x = y) AS e) ORDER BY x
. #66312 (vdimir). - Fix cluster() for inter-server secret (preserve initial user as before). #66364 (Azat Khuzhin).
- Fix possible runtime error while converting Array field with nulls to Array(Variant). #66727 (Kruglov Pavel).
- Fix for occasional deadlock in Context::getDDLWorker. #66843 (Alexander Gololobov).
- Fix creating KeeperMap table after an incomplete drop. #66865 (Antonio Andelic).
- Fix broken part error while restoring to a
s3_plain_rewritable
disk. #66881 (Vitaly Baranov). - In rare cases ClickHouse could consider parts as broken because of some unexpected projections on disk. Now it's fixed. #66898 (alesapin).
- Fix invalid format detection in schema inference that could lead to logical error Format {} doesn't support schema inference. #66899 (Kruglov Pavel).
- Fix possible deadlock on query cancel with parallel replicas. #66905 (Nikita Taranov).
- Forbid create as select even when database_replicated_allow_heavy_create is set. It was unconditionally forbidden in 23.12 and accidentally allowed under the setting in unreleased 24.7. #66980 (vdimir).
- Reading from the
numbers
could wrongly throw an exception when themax_rows_to_read
limit was set. This closes #66992. #66996 (Alexey Milovidov). - Add proper type conversion to lagInFrame and leadInFrame window functions - fixes msan test. #67091 (Yakov Olkhovskiy).
- Only relevant to the experimental Variant data type. Fix crash with Variant + AggregateFunction type. #67122 (Kruglov Pavel).
- TRUNCATE DATABASE used to stop replication as if it was a DROP DATABASE query, it's fixed. #67129 (Alexander Tokmakov).
- Use a separate client context in
clickhouse-local
. #67133 (Vitaly Baranov). - Fix error
Cannot convert column because it is non constant in source stream but must be constant in result.
for a query that reads from theMerge
table over theDistriburted
table with one shard. #67146 (Nikolai Kochetov). - Correct behavior of
ORDER BY all
with disabledenable_order_by_all
and parallel replicas (distributed queries as well). #67153 (Igor Nikonov). - Fix wrong usage of input_format_max_bytes_to_read_for_schema_inference in schema cache. #67157 (Kruglov Pavel).
- Fix the memory leak for count distinct, when exception issued during group by single nullable key. #67171 (Jet He).
- This closes #67156. This closes #66447. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/62907. #67178 (Maksim Kita).
- Fix crash in DistributedAsyncInsert when connection is empty. #67219 (Pablo Marcos).
- Fix error
Conversion from AggregateFunction(name, Type) to AggregateFunction(name, Nullable(Type)) is not supported
. The bug was caused by theoptimize_rewrite_aggregate_function_with_if
optimization. Fixes #67112. #67229 (Nikolai Kochetov). - Fix hung query when using empty tuple as lhs of function IN. #67295 (Duc Canh Le).
- Fix crash of
uniq
anduniqTheta
withtuple()
argument. Closes #67303. #67306 (flynn). - It was possible to create a very deep nested JSON data that triggered stack overflow while skipping unknown fields. This closes #67292. #67324 (Alexey Milovidov).
- Fix attaching ReplicatedMergeTree table after exception during startup. #67360 (Antonio Andelic).
- Fix segfault caused by incorrectly detaching from thread group in
Aggregator
. #67385 (Antonio Andelic). - Fix one more case when a non-deterministic function is specified in PK. #67395 (Nikolai Kochetov).
- Fixed
bloom_filter
index breaking queries with mildly weird conditions like(k=2)=(k=2)
orhas([1,2,3], k)
. #67423 (Michael Kolupaev). - Correctly parse file name/URI containing
::
if it's not an archive. #67433 (Antonio Andelic). - Fix wait for tasks in ~WriteBufferFromS3 in case WriteBuffer was cancelled. #67459 (Kseniia Sumarokova).
- Protect temporary part directories from removing during RESTORE. #67491 (Vitaly Baranov).
- Fix execution of nested short-circuit functions. #67520 (Kruglov Pavel).
- Fixes #66026. Avoid unresolved table function arguments traversal in
ReplaceTableNodeToDummyVisitor
. #67522 (Dmitry Novik). - Fix
Logical error: Expected the argument №N of type T to have X rows, but it has 0
. The error could happen in a remote query with constant expression inGROUP BY
(with a new analyzer). #67536 (Nikolai Kochetov). - Fix join on tuple with NULLs: Some queries with the new analyzer and
NULL
inside the tuple in theJOIN ON
section returned incorrect results. #67538 (vdimir). - Fix redundant reschedule of FileCache::freeSpaceRatioKeepingThreadFunc() in case of full non-evictable cache. #67540 (Kseniia Sumarokova).
- Fix inserting into stream like engines (Kafka, RabbitMQ, NATS) through HTTP interface. #67554 (János Benjamin Antal).
- Fix for function
toStartOfWeek
which returned the wrong result with a smallDateTime64
value. #67558 (Yarik Briukhovetskyi). - Fix creation of view with recursive CTE. #67587 (Yakov Olkhovskiy).
- Fix
Logical error: 'file_offset_of_buffer_end <= read_until_position'
in filesystem cache. Closes #57508. #67623 (Kseniia Sumarokova). - Fixes #62282. Removed the call to
convertFieldToString()
and added datatype specific serialization code. Parameterized view substitution was broken for multiple datatypes when parameter value was a function or expression returning datatype instance. #67654 (Shankar). - Fix crash on
percent_rank
.percent_rank
's default frame type is changed torange unbounded preceding and unbounded following
.IWindowFunction
's default window frame is considered and now window functions without window frame definition in sql can be put into differentWindowTransfomer
s properly. #67661 (lgbo). - Fix reloading SQL UDFs with UNION. Previously, restarting the server could make UDF invalid. #67665 (Antonio Andelic).
- Fix possible logical error "Unexpected return type from if" with experimental Variant type and enabled setting
use_variant_as_common_type
in function if with Tuples and Maps. #67687 (Kruglov Pavel). - Due to a bug in Linux Kernel, a query can hung in
TimerDescriptor::drain
. This closes #37686. #67702 (Alexey Milovidov). - Fix completion of
RESTORE ON CLUSTER
command. #67720 (Vitaly Baranov). - Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading. #67751 (Azat Khuzhin).
- Fix potential stack overflow in
JSONMergePatch
function. Renamed this function fromjsonMergePatch
toJSONMergePatch
because the previous name was wrong. The previous name is still kept for compatibility. Improved diagnostic of errors in the function. This closes #67304. #67756 (Alexey Milovidov). - Queries like
SELECT count() FROM t WHERE cast(c = 1 or c = 9999 AS Bool) SETTINGS use_skip_indexes=1
with bloom filter indexes onc
now work correctly. #67781 (jsc0218). - Fix wrong aggregation result in some queries with aggregation without keys and filter, close #67419. #67804 (vdimir).
- Validate experimental/suspicious data types in ALTER ADD/MODIFY COLUMN. #67911 (Kruglov Pavel).
- Fix DateTime64 parsing after constant folding in distributed queries, close #66773. #67920 (vdimir).
- Fix wrong
count()
result when there is non-deterministic function in predicate. #67922 (János Benjamin Antal). - Fixed the calculation of the maximum thread soft limit in containerized environments where the usable CPU count is limited. #67963 (Robert Schulze).
- Now ClickHouse doesn't consider part as broken if projection doesn't exist on disk but exists in
checksums.txt
. #68003 (alesapin). - Fixed skipping of untouched parts in mutations with new analyzer. Previously with enabled analyzer data in part could be rewritten by mutation even if mutation doesn't affect this part according to predicate. #68052 (Anton Popov).
- Fixed a NULL pointer dereference, triggered by a specially crafted query, that crashed the server via hopEnd, hopStart, tumbleEnd, and tumbleStart. #68098 (Salvatore Mesoraca).
- Removes an incorrect optimization to remove sorting in subqueries that use
OFFSET
. Fixes #67906. #68099 (Graham Campbell). - Attempt to fix
Block structure mismatch in AggregatingStep stream: different types
for aggregate projection optimization. #68107 (Nikolai Kochetov). - Backported in #68343: Try fix postgres crash when query is cancelled. #68288 (Kseniia Sumarokova).
- Backported in #68400: Fix missing sync replica mode in query
SYSTEM SYNC REPLICA
. #68326 (Duc Canh Le).
Build/Testing/Packaging Improvement
- Improved
test_storage_s3
tests: increaseds3_max_single_read_retries
for read from "unstable" s3 source and allowed all tests to run multiple times in a row. #66896 (Ilya Yatsishin). - Integration tests flaky check will not run each test case multiple times to find more issues in tests and make them more reliable. It is using
pytest-repeat
library to run test case multiple times for the same environment. It is important to cleanup tables and other entities in the end of a test case to pass. Repeat works much faster than several pytest runs as it starts necessary containers only once. #66986 (Ilya Yatsishin). - Allow to use CLion with ClickHouse. In previous versions, CLion freezed for a minute on every keypress. This closes #66994. #66995 (Alexey Milovidov).
- Getauxval: avoid crash under sanitizer re-exec due to high aslr entropy. #67081 (Raúl Marín).
- Some parts of client code are extracted to a single file and highest possible level optimization is applied to them even for debug builds. This closes: #65745. #67215 (Nikita Mikhaylov).
NO CL CATEGORY
- Backported in #68416:. #68386 (János Benjamin Antal).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Fix for 992 and friends"'. #66993 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Revert "Fix for 992 and friends""'. #67029 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "FuzzQuery table function"'. #67040 (Raúl Marín).
- NO CL ENTRY: 'Revert "Enable
compile_expressions
by default."'. #67299 (Alexey Milovidov). - NO CL ENTRY: 'Revert "Slightly better calculation of primary index"'. #67392 (alesapin).
- NO CL ENTRY: 'Revert "Add settings to replace external engines to Null during create"'. #67507 (Raúl Marín).
- NO CL ENTRY: 'Revert "Revert "Add settings to replace external engines to Null during create""'. #67511 (Ilya Yatsishin).
- NO CL ENTRY: 'Revert "Add replication lag and recovery time metrics"'. #67731 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Revert "Slightly better calculation of primary index""'. #67846 (Anton Popov).
- NO CL ENTRY: 'Revert "CI: Strict job timeout 1.5h for tests, 2h for builds"'. #67986 (Max K.).
- NO CL ENTRY: 'Revert "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing"'. #68014 (Nikita Mikhaylov).
- NO CL ENTRY: 'Revert "Use
Atomic
database by default inclickhouse-local
"'. #68023 (Alexey Milovidov). - NO CL ENTRY: 'Revert "Refactor tests for (experimental) statistics"'. #68156 (Alexander Tokmakov).
NOT FOR CHANGELOG / INSIGNIFICANT
- CI: enable libfuzzer (fixing build and docker). #61908 (Yakov Olkhovskiy).
- Initial implementation of vector similarity index. #63675 (Robert Schulze).
- Update zlib-ng from 2.0.2 to 2.1.7. #64489 (Michael Kolupaev).
- Fix 02444_async_broken_outdated_part_loading flakiness. #64956 (Azat Khuzhin).
- attach_gdb.lib: print more information before all stacks. #65253 (Michael Kolupaev).
- Fix some perf tests. #65320 (Nikita Taranov).
- Remove ActionsDAGPtr whenever it is possible. #65414 (Nikolai Kochetov).
- Scheduler queue throughput introspection, Fix CPU indication in client. #65654 (Sergei Trifonov).
- Increase timeout in 02122_join_group_by_timeout for tsan build. #65976 (vdimir).
- Remove default values for certificateFile/privateKeyFile/dhParamsFile in keeper config (to avoid annoying errors in logs). #65978 (Azat Khuzhin).
- Update version_date.tsv and changelogs after v24.3.5.46-lts. #66054 (robot-clickhouse).
- Fix flaky
02814_currentDatabase_for_table_functions
. #66111 (Nikita Mikhaylov). - Fix possible data-race StorageKafka with statistics_interval_ms>0. #66311 (Azat Khuzhin).
- Avoid unneeded calculation in SeriesPeriodDetect. #66320 (Ruihang Xia).
- It aims to complete #58630. This is made possible by #60463, #61459 and #60082. #66443 (Amos Bird).
- Allow run query instantly in play. #66457 (Aleksandr Musorin).
- Bump ICU from v70 to v75. #66474 (Robert Schulze).
- Bump RocksDB from v6.29.5 to v7.10.2. #66475 (Robert Schulze).
- Bump RocksDB from v7.10.2 to v8.9.1. #66479 (Robert Schulze).
- I believe the error code for this function should not be "NOT_ALLOWED" since it simply is an invalid query and "BAD_QUERY_PARAMETER" is a more reasonable error code for this. #66491 (Ali).
- Update gdb to 15.1 (by compiling from sources). #66494 (Azat Khuzhin).
- Ensure that llvm-symbolizer is used for symbolizing sanitizer reports. #66495 (Azat Khuzhin).
- Remove unused local variables. #66503 (Alexey Milovidov).
- This will solve a lot of problems with inconsistent formatting. And it opens the path for #65753. This closes #66807. This closes #61611. This closes #61711. This closes #67445. #66506 (Alexey Milovidov).
- Rename Context::getSettings() to Context::getSettingsCopy(). #66528 (Raúl Marín).
- Uninteresting change: introducing
ClientApplicationBase
. #66549 (Nikita Mikhaylov). - Respond to parallel replicas protocol requests with priority on initiator. #66618 (Igor Nikonov).
- Fix bad code: it was catching exceptions. #66628 (Alexey Milovidov).
- Dump all memory stats in CgroupsMemoryUsageObserver on hitting the limit. #66732 (Nikita Taranov).
- Save writer thread id in shared mutex for debugging. #66745 (Alexander Gololobov).
- Increase asio pool size in case the server is tiny. #66761 (alesapin).
- Looks like it runs too many mutations sometimes and fails to process them within the timeout. So if a query waits for mutations - the test fails. #66785 (Alexander Tokmakov).
- Better diagnostics in functional tests. #66790 (Alexey Milovidov).
- Fix bad test
01042_system_reload_dictionary_reloads_completely
. #66811 (Alexey Milovidov). - Something is strange with the test about refreshable materialized views. #66816 (Alexey Milovidov).
- Randomize
trace_profile_events
. #66821 (Alexey Milovidov). - Stop ignoring SIGSEGV in GDB. #66847 (Antonio Andelic).
- Followup #66725. #66869 (vdimir).
- When executing queries with parallel replicas that involve only a subset of nodes within a shard, the current behavior is that if all participating replicas are unavailable, the query completes without any errors but returns no results. Referencing issue #65467, this pull request addresses the issue where only a portion of the nodes in a shard are participating in the execution. #66880 (zoomxi).
- Speed up stateful tests setup. #66886 (Raúl Marín).
- Functions [h-r]*: Iterate over input_rows_count where appropriate. #66897 (Robert Schulze).
- Stateless tests: Change status of failed tests in case of server crash and add no-parallel to high-load tests. #66901 (Nikita Fomichev).
- Fix performance test about the generateRandom table function, supposedly. #66906 (Alexey Milovidov).
- Fix bad tests
share_big_sets
, CC @davenger. #66908 (Alexey Milovidov). - Speed up mutations for non-replicated MergeTree a bit. #66909 (Alexey Milovidov).
- Speed up mutations for non-replicated MergeTree significantly. #66911 (Alexey Milovidov).
- Fix views over distributed tables with Analyzer. #66912 (Alexey Milovidov).
- [CI fest] Try to fix
test_broken_projections/test.py::test_broken_ignored_replicated
. #66915 (Andrey Zvonov). - Decrease rate limit in
01923_network_receive_time_metric_insert
. #66924 (János Benjamin Antal). - Grouparrayintersect: fix serialization bug. #66928 (Raúl Marín).
- Update version after release branch. #66929 (Raúl Marín).
- Un-flake test_runtime_configurable_cache_size. #66934 (Robert Schulze).
- fix unit tests ResolvePoolTest with timeouts. #66953 (Sema Checherinda).
- Split slow test 03036_dynamic_read_subcolumns. #66954 (Nikita Taranov).
- CI: Fixes docker server build for release branches. #66955 (Max K.).
- Addressing issue #64936. #66973 (alesapin).
- Add initial 24.7 changelog. #66976 (Raúl Marín).
- Apply libunwind fix. #66977 (Michael Kolupaev).
- CI: Add logs for debugging. #66979 (Max K.).
- [CI Fest] Split dynamic tests and rewrite them from sh to sql to avoid timeouts. #66981 (Kruglov Pavel).
- Split 01508_partition_pruning_long. #66983 (Nikita Taranov).
- [CI Fest] Fix use-of-uninitialized-value in JSONExtract* numeric functions. #66984 (Kruglov Pavel).
- It should fix SQLancer checks, but for some reason we stopped invalidating cache for docker builds and fix was not published to our CI for a while. #66987 (Ilya Yatsishin).
- Fixes #66941. #66991 (Konstantin Bogdanov).
- Remove the support for Kerberized HDFS. #66998 (Alexey Milovidov).
- CI: Fix for workflow results parsing. #67000 (Max K.).
- Fix flaky
01454_storagememory_data_race_challenge
. #67003 (Antonio Andelic). - CI: Jepsen Workflow fix for skipped builds and observability. #67004 (Max K.).
- bugfix AttachedTable counting not symmetry, and adding some test logs…. #67007 (Xu Jia).
- CI: Automerge when required and non-required checks completed. #67008 (Max K.).
- Fix test
very_long_arrays
. #67009 (Alexey Milovidov). - Try to fix exception logging in destructors of static objects. #67016 (Antonio Andelic).
- [Green CI] Fix test test_storage_azure_blob_storage. #67019 (Daniil Ivanik).
- Integration tests: fix flaky tests
test_backup_restore_on_cluster/test_concurrency.py
&test_manipulate_statistics/test.py
. #67027 (Nikita Fomichev). - [Green CI] Fix test test_storage_s3_queue/test.py::test_max_set_age. #67035 (Pablo Marcos).
- Test for alter select with parallel replicas. #67041 (Igor Nikonov).
- Split query into multiple queries to consume less memory at once + use less data. Fixes #67034. #67044 (alesapin).
- Disable setting
optimize_functions_to_subcolumns
. #67046 (Anton Popov). - Increase max allocation size for sanitizers. #67049 (János Benjamin Antal).
- rewrite 01171 test. #67054 (Sema Checherinda).
- Add
**
tohdfs
docs, add test for**
inhdfs
. #67064 (Andrey Zvonov). - Very sad failure:
2024.07.24 13:28:45.517777 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> executeQuery: (from 172.16.11.1:55890) OPTIMIZE TABLE replicated_mt FINAL (stage: Complete) 2024.07.24 13:28:45.525945 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Trace> default.replicated_mt (ReplicatedMergeTreeQueue): Waiting for 4 entries to be processed: queue-0000000004, queue-0000000002, queue-0000000001, queue-0000000000 2024.07.24 13:29:15.528024 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e) (MergerMutator): Selected 3 parts from all_0_0_0 to all_2_2_0 2024.07.24 13:29:15.530736 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Trace> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Created log entry /clickhouse/tables/replicated_mt/log/log-0000000004 for merge all_0_2_1 2024.07.24 13:29:15.530873 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to process log entry 2024.07.24 13:29:15.530919 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for node1 to pull log-0000000004 to queue 2024.07.24 13:29:15.534286 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Looking for node corresponding to log-0000000004 in node1 queue 2024.07.24 13:29:15.534793 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> default.replicated_mt (6581a6fb-8458-466d-8350-89951eb1ac8e): Waiting for queue-0000000005 to disappear from node1 queue 2024.07.24 13:29:15.585533 [ 10 ] {08745bf9-4bc1-4946-b9a8-c03d82ec55dc} <Debug> TCPHandler: Processed in 30.067804125 sec.
. #67067 (alesapin). - Fix flaky
test_seekable_formats_url
andtest_seekable_formats
S3 storage tests. #67070 (János Benjamin Antal). - CI: Docker server build fix for new release workflow. #67075 (Max K.).
- Fix 2680 flasky. #67078 (jsc0218).
- [CI Fest] Fix flaky 02447_drop_replica test. #67085 (Nikolay Degterinsky).
- Fixes #67030. #67086 (Alexander Tokmakov).
- Increase timeout for curl in tests. #67089 (Anton Popov).
- Try calculating available memory if ClickHouse is bound to subset of NUMA nodes. #67098 (Antonio Andelic).
- A more precise way of tracking flushing time in 01246_buffer_flush. #67099 (Azat Khuzhin).
- Do not fail CheckReadyForMerge on failed Tests_2 (non-required jobs) Do not skip CiBuddy report step on failures. #67101 (Max K.).
- Tststs_1 - for all required checks Tststs_2 - for all non-required checks (normal mode) Tststs_2_ww - for all non-required checks (woolenwolfdog mode). #67104 (Max K.).
- Functions [s-t]*: Iterate over input_rows_count where appropriate. #67105 (Robert Schulze).
- Reintroduce 02805_distributed_queries_timeouts. #67106 (Azat Khuzhin).
- Added some tests in relation with #54881. #67110 (max-vostrikov).
- Reintroduce 03002_part_log_rmt_fetch_*_error tests without flakiness and less time. #67113 (Azat Khuzhin).
- Improve tag matching in backport scripts. #67118 (Raúl Marín).
- Fixes #67111. #67121 (Konstantin Bogdanov).
- Increase lock_acquire_timeout_for_background_operations setting in dynamic merges tests. #67126 (Kruglov Pavel).
- Attempt to fix flakiness of some window view tests. #67130 (Robert Schulze).
- Update assert in cache. #67138 (Kseniia Sumarokova).
- Fix test
00673_subquery_prepared_set_performance
. #67141 (Alexey Milovidov). - Fixes #67047. #67142 (Alexander Tokmakov).
- Lower max allocation size in query fuzzer. #67145 (Alexey Milovidov).
- Fixes #66966. #67147 (Alexander Tokmakov).
- Try fix
02481_async_insert_race_long
flakiness. #67148 (Julia Kartseva). - Rename (unreleased) bad setting. #67149 (Raúl Marín).
- Uncomment accidentally commented out code in QueryProfiler. #67152 (Michael Kolupaev).
- Try to fix 2572. #67158 (jsc0218).
- Fix benign data race in ZooKeeper. #67164 (Alexey Milovidov).
- Remove too long unit test. #67168 (Alexey Milovidov).
- Fix
00705_drop_create_merge_tree
. #67170 (Alexey Milovidov). - Fix stacktrace cache. #67173 (Antonio Andelic).
- Fixes #67151. #67174 (Konstantin Bogdanov).
- Make 02908_many_requests_to_system_replicas less stressful. #67176 (Alexander Gololobov).
- Reduce max time of 00763_long_lock_buffer_alter_destination_table. #67185 (Raúl Marín).
- Do not count AttachedTable for tables in information schema databases. #67187 (Sergei Trifonov).
- Verbose output for 03203_client_benchmark_options. #67188 (vdimir).
- Split test 02967_parallel_replicas_join_algo_and_analyzer. #67211 (Nikita Taranov).
- Fix flaky
test_pkill_query_log
(tsan). #67223 (Sergei Trifonov). - Remove integration test
test_broken_projections_in_backups_1
. #67231 (Vitaly Baranov). - Debug logging for #67002. #67233 (Nikita Taranov).
- Fix oss-fuzz build. #67235 (Nikita Taranov).
- Fix flaky 00180_no_seek_avoiding_when_reading_from_cache. #67236 (Yakov Olkhovskiy).
- English. #67258 (Alexey Milovidov).
- Remove strange code. #67260 (Alexey Milovidov).
- Fix MSan report in DatabaseReplicated. #67262 (Alexey Milovidov).
- Fix test
02310_clickhouse_local_INSERT_progress_profile_events
. #67264 (Alexey Milovidov). - Remove test
02982_aggregation_states_destruction
. #67266 (Alexey Milovidov). - Fix OOM in test runs. #67268 (Alexey Milovidov).
- Remove harmful stuff from tests. #67275 (Alexey Milovidov).
- Fix test
03201_variant_null_map_subcolumn
. #67276 (Alexey Milovidov). - Split
01651_lc_insert_tiny_log
. #67279 (Alexey Milovidov). - Debug test
02490_benchmark_max_consecutive_errors
. #67281 (Alexey Milovidov). - Fix bad test
02833_concurrrent_sessions
. #67282 (Alexey Milovidov). - Add a separate test for exception handling. #67283 (Alexey Milovidov).
- Debug test
01600_parts_states_metrics_long
. #67284 (Alexey Milovidov). - Faster test
02231_buffer_aggregate_states_leak
. #67285 (Alexey Milovidov). - Fix curiosities in
TimerDescriptor
. #67287 (Alexey Milovidov). - Add retries to test
02911_backup_restore_keeper_map
. #67290 (Alexey Milovidov). - Functions: Iterate over input_rows_count where appropriate. #67294 (Robert Schulze).
- Add documentation for
compile_expressions
. #67300 (Alexey Milovidov). - Wrap log lines in the CI report for functional tests. #67301 (Alexey Milovidov).
- Fix bad test
02050_client_profile_events
. #67309 (Alexey Milovidov). - I do not want to think about this code. #67312 (Alexey Milovidov).
- Fix test
00940_max_parts_in_total
. #67313 (Alexey Milovidov). - Tests for Kafka cannot run in parallel. #67315 (Alexey Milovidov).
- Add a test for #66285. #67325 (Alexey Milovidov).
- Follow-up for #67301. #67327 (Alexey Milovidov).
- Add a test for #61659. #67332 (Alexey Milovidov).
- Fix integration test
test_backup_restore_on_cluster/test_disallow_concurrency
. #67336 (Vitaly Baranov). - Faster and less flaky 01246_buffer_flush (by using HTTP over clickhouse-client). #67340 (Azat Khuzhin).
- Fix: data race in TCPHandler on socket timeouts settings. #67341 (Yakov Olkhovskiy).
- CI: Print stdout, stderr for docker pull command. #67343 (Max K.).
- Followup #67290. #67348 (vdimir).
- Skip parallel for
test_storage_kerberized_kafka
. #67349 (Andrey Zvonov). - Don't use PeekableReadBuffer in JSONAsObject format. #67354 (Kruglov Pavel).
- This closes: #57316. #67355 (Nikita Mikhaylov).
- Remove duplicated tests. #67357 (Kruglov Pavel).
- Release branch was not detected properly and job which must run on release branch could be reused from feature branches. PR Fixes detection of release branches. #67358 (Max K.).
- Disable some Dynamic tests under sanitizers, rewrite 03202_dynamic_null_map_subcolumn to sql. #67359 (Kruglov Pavel).
- Add no-distributed-cache tag in tests. #67361 (Kseniia Sumarokova).
- Trying to fix test_cache_evicted_by_temporary_data and print debug info. #67362 (vdimir).
- Try to fix: ALL_CONNECTION_TRIES_FAILED with parallel replicas. #67389 (Igor Nikonov).
- Fix bad test
01036_no_superfluous_dict_reload_on_create_database
. #67390 (Alexey Milovidov). - Adding
SelectedPartsTotal
andSelectedMarksTotal
as new ProfileEvents. #67393 (Jordi Villar). - Print debug info in
test_storage_s3_queue/test.py::test_shards_distributed
. #67394 (János Benjamin Antal). - Mute degraded perf test. #67396 (Nikita Taranov).
- Debug TimerDescriptor. #67397 (Alexey Milovidov).
- Remove has_single_port property from plan stream. #67398 (Nikolai Kochetov).
- Fix typo. #67400 (Halersson Paris).
- CI: Fix build results for release branches. #67402 (Max K.).
- Disable 02932_refreshable_materialized_views. #67404 (Michael Kolupaev).
- Follow-up to #67294. #67405 (Robert Schulze).
- Fix DWARF range list parsing in stack symbolizer. #67417 (Michael Kolupaev).
- Make Dwarf::findAddress() fallback slow path less slow. #67418 (Michael Kolupaev).
- Fix inconsistent formatting of CODEC and STATISTICS. #67421 (Alexey Milovidov).
- Reduced complexity of the test 02832_alter_max_sessions_for_user. #67425 (Alexey Gerasimchuck).
- Remove obsolete
--multiquery
parameter from tests. #67435 (Robert Schulze). - Fix public backports. #67439 (Raúl Marín).
- Bump Azure from v1.12 to v1.13. #67446 (Robert Schulze).
- 24.7 add missing documentation and testing. #67454 (Nikita Fomichev).
- Use correct order of fields in
StorageURLSource
. #67455 (Antonio Andelic). - run 01171 test in parallel. #67470 (Sema Checherinda).
- [Green CI] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown. #67474 (Alexey Katsman).
- Reduce 02473_multistep_prewhere run time. #67475 (Alexander Gololobov).
- Update version_date.tsv and changelogs after v24.7.1.2915-stable. #67483 (robot-clickhouse).
- Revert #61750 "Improve JSONEachRow reading by ignoring the keys case". #67484 (Michael Kolupaev).
- Disable parallel run for
01923_network_receive_time_metric_insert.sh
. #67492 (Julia Kartseva). - Fix test
test_backup_restore_on_cluster/test.py::test_mutation
. #67494 (Vitaly Baranov). - [Green CI] Fix potentially flaky test_mask_sensitive_info integration test. #67506 (Alexey Katsman).
- [Green CI] Test
test_storage_azure_blob_storage/test.py
is flaky. #67512 (Daniil Ivanik). - Prepare Release workflow for production. #67523 (Max K.).
- Fix upgrade check. #67524 (Raúl Marín).
- [Green CI] test 03164_s3_settings_for_queries_and_merges is flaky. #67535 (Daniil Ivanik).
- Log message and increased concurrency for table removal. #67537 (Nikita Mikhaylov).
- Fix AsyncLoader destruction race. #67553 (Sergei Trifonov).
- Add an assert into TimerDescriptor. #67555 (Alexey Milovidov).
- Re-enable ICU on s390/x. #67557 (Robert Schulze).
- Update version_date.tsv and changelogs after v24.4.4.107-stable. #67559 (robot-clickhouse).
- Sometimes we fail with timeout in stateless tests and the reason for that seems to be in
stop_logs_replication
step. Add a check for timeout here. #67560 (Nikolay Degterinsky). - Miscellaneous. #67564 (Alexey Milovidov).
- enable parallel_view_processing in perf tests. #67565 (Sema Checherinda).
- Fix flaky
test_system_kafka_consumers_rebalance
. #67566 (János Benjamin Antal). - Update version_date.tsv and changelogs after v24.7.2.13-stable. #67586 (robot-clickhouse).
- Fix 01811_storage_buffer_flush_parameters flakiness. #67589 (Azat Khuzhin).
- Fix test_zookeeper_config_load_balancing after adding the xdist worker name to the instance. #67590 (Pablo Marcos).
- Update minio in integration tests. #67595 (Antonio Andelic).
- added tests for page index in parquet files. #67596 (max-vostrikov).
- Update check_rabbitmq_is_available. #67597 (Kseniia Sumarokova).
- Fix 02434_cancel_insert_when_client_dies. #67600 (vdimir).
- Fix 02910_bad_logs_level_in_local in fast tests. #67603 (Raúl Marín).
- Fix 01605_adaptive_granularity_block_borders. #67605 (Nikita Taranov).
- Update CHANGELOG.md. #67607 (Alexey Milovidov).
- Remove some
no-parallel
tags from tests. #67610 (Raúl Marín). - Update README.md. #67613 (Tyler Hannan).
- Try fix 03143_asof_join_ddb_long. #67620 (Nikita Taranov).
- Don't run ASAN unit tests under gdb. #67622 (Raúl Marín).
- Fix crash in KeyCondition::cloneASTWithInversionPushDown() caused by type change. #67641 (Michael Kolupaev).
- Fix race condition between ProcessList and Settings. #67645 (Alexey Milovidov).
02481_async_insert_race_long.sh
flakiness fixes. #67650 (Julia Kartseva).- Fixes #67651. #67653 (pufit).
- Fix flaky
test_replicated_table_attach
. #67658 (Antonio Andelic). - Update version_date.tsv and changelogs after v24.4.4.113-stable. #67659 (robot-clickhouse).
- Even better healthcheck for ldap. #67667 (Andrey Zvonov).
- Fix 03203_client_benchmark_options. #67671 (vdimir).
- Integration tests: fix ports clashing problem. #67672 (Nikita Fomichev).
- Remove some
no-parallel
tags from tests (Part 2). #67673 (Raúl Marín). - Use FunctionArgumentDescriptors for bitSlice. #67674 (Lennard Eijsackers).
- Update version_date.tsv and changelog after v24.3.6.48-lts. #67677 (robot-clickhouse).
- Avoid ddl queue timeout in 02313_filesystem_cache_seeks. #67680 (Nikita Taranov).
- Fix bad log message in sort description. #67690 (Alexey Milovidov).
- Update version_date.tsv and changelog after v23.8.16.40-lts. #67692 (robot-clickhouse).
- Fix check names in test reports and the CI Logs database. #67696 (Alexey Milovidov).
- CI: Minor refactoring in ci_utils. #67706 (Max K.).
- Fix 01042_system_reload_dictionary_reloads_completely flakiness. #67719 (Azat Khuzhin).
- Fix test
00002_log_and_exception_messages_formatting
. #67723 (Alexey Milovidov). - Fix test
02789_reading_from_s3_with_connection_pool
. #67726 (Alexey Milovidov). - Fix strange code in HostResolvePool. #67727 (Alexey Milovidov).
- Fix a typo. #67729 (Alexey Milovidov).
- Smart handling of processes leftovers in tests. #67737 (Azat Khuzhin).
- Fix test retries. #67738 (Azat Khuzhin).
- Fill only selected columns from system.clusters. #67739 (Azat Khuzhin).
- Bump NuRaft (to properly catch thread exceptions). #67740 (Azat Khuzhin).
- Try to fix RabbitMQ test failures. #67743 (Azat Khuzhin).
- Stateless tests: attempt to fix timeouts of
02473_multistep_prewhere* 00411_long_accurate_number_comparison*
. #67746 (Nikita Fomichev). - Fix test_ttl_move::test_alter_with_merge_work flakiness. #67747 (Azat Khuzhin).
- ci: better stateless runner (correctly collect artifacts and also some basic errors capturing). #67752 (Azat Khuzhin).
- Introduce
no-flaky-check
tag. #67755 (Alexey Milovidov). - Whitespaces. #67771 (Alexey Milovidov).
- [RFC] Print original query for AST formatting check on CI. #67776 (Azat Khuzhin).
- Fix test
02833_concurrent_sessions
, Fix test02835_drop_user_during_session
. #67779 (Alexey Milovidov). - Fix diagnostics in the test script. #67780 (Alexey Milovidov).
- Fix test
02231_bloom_filter_sizing
. #67784 (Alexey Milovidov). - Fixed session_log related tests race condition on logout. #67785 (Alexey Gerasimchuck).
- ci/stateless: fix artifacts post-processing and detect if something failed there. #67791 (Azat Khuzhin).
- Integration tests: fix flaky
test_dictionaries_update_and_reload::test_reload_after_fail_by_timer
. #67793 (Nikita Fomichev). - Fix possible CANNOT_READ_ALL_DATA during server startup in performance tests. #67795 (Azat Khuzhin).
- Reduce table size in 03037_dynamic_merges_2* tests. #67797 (Kruglov Pavel).
- Disable 03038_nested_dynamic_merges* under sanitizers because it's too slow. #67798 (Kruglov Pavel).
- Revert "Merge pull request #66510 from canhld94/fix_trivial_count_non_deterministic_func". #67800 (János Benjamin Antal).
- Update comment. #67801 (Kseniia Sumarokova).
- Fix bad test
03032_redundant_equals
. #67822 (Alexey Milovidov). - Update tags for a few tests. #67829 (Kseniia Sumarokova).
- Add debug logging for window view tests. #67841 (Kseniia Sumarokova).
- Closes #67621. #67843 (Ilya Yatsishin).
- Fix query cache randomization in stress tests. #67855 (Azat Khuzhin).
- Update version_date.tsv and changelogs after v24.5.5.78-stable. #67863 (robot-clickhouse).
- Un-flake 02524_fuzz_and_fuss_2. #67867 (Robert Schulze).
- Misc fixes. #67869 (Alexey Katsman).
- Fixes #67444. #67873 (Konstantin Bogdanov).
- no-msan 00314_sample_factor_virtual_column. #67874 (Michael Kolupaev).
- Revert "Revert "Add a test for #47892"". #67877 (Alexey Milovidov).
- Lazily create invalid files in S3. #67882 (Antonio Andelic).
- Do not try to create azure container if not needed. #67896 (Anton Popov).
- CI: Fix for setting Mergeable Check from sync. #67898 (Max K.).
- Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing. #67904 (Robert Schulze).
- Update version_date.tsv and changelogs after v24.6.3.95-stable. #67910 (robot-clickhouse).
- Remove some no-parallel tags from tests (Part 3). #67914 (Raúl Marín).
- Follow up #67235. #67917 (Nikita Taranov).
- CI: Changelog: Critical Bug Fix to Bug Fix. #67919 (Max K.).
- CI: Multi-channel CiBuddy. #67923 (Max K.).
- more logs to debug logical error from async inserts. #67928 (Han Fei).
- Fix stress test error with TDigest statistics. #67930 (Robert Schulze).
- Remove some no-parallel tags from tests (Part 4). #67932 (Raúl Marín).
- Upgrade QPL to v1.6.0. #67933 (Maria Zhukova).
- CI: Strict job timeout 1.5h for tests, 2h for builds. #67934 (Max K.).
- Remove slow tests from fasttest check. #67941 (Raúl Marín).
- Fix memory corruption in usearch. #67942 (Robert Schulze).
- Backported in #68547: Timeout handling for functional and integration tests, store artifacts and report if timed out - sets 2h default timeout for all jobs. #67944 (Max K.).
- Unflake 02099_tsv_raw_format.sh. #67947 (Robert Schulze).
- This closes: #67866. #67950 (Nikita Mikhaylov).
- Change log level of an insignificant message in clickhouse-local. #67952 (Alexey Milovidov).
- Add an explicit error for
ALTER MODIFY SQL SECURITY
on non-view tables. #67953 (pufit). - Fix flaky
test_storage_s3_queue/test.py::test_multiple_tables_streaming_sync_distributed
. #67959 (Julia Kartseva). - tests: fix 03002_part_log_rmt_fetch_merge_error flakiness. #67960 (Azat Khuzhin).
- Fix timeout of 02310_clickhouse_local_INSERT_progress_profile_events. #67961 (Robert Schulze).
- Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. III. #67964 (Robert Schulze). - Update minio in stateless tests. #67975 (Antonio Andelic).
- CI: Integration tests uncover some logging. #67978 (Max K.).
- Fix 03130_convert_outer_join_to_inner_join. #67980 (vdimir).
- Collect minio audit logs in stateless tests. #67998 (Antonio Andelic).
- Remove some no-parallel tags from tests (Part 5). #68002 (Raúl Marín).
- Minor fixes in tables.md. #68004 (Ilya Yatsishin).
- Follow up for #67843. #68007 (Ilya Yatsishin).
- Remove unused CLI option. #68008 (Alexey Milovidov).
- Fix test
02845_threads_count_in_distributed_queries
. #68011 (Alexey Milovidov). - CI: Pass job timeout into tests config. #68013 (Nikita Fomichev).
- Add a test for #57420. #68017 (Alexey Milovidov).
- Revert "Revert "Bump rocksdb from v8.10 to v9.4 + enable jemalloc and liburing"". #68021 (Robert Schulze).
- CI: Fix for filtering jobs in PRs. #68022 (Max K.).
- Docs: Update 3rd party library guide. #68027 (Robert Schulze).
- Refactor tests for (experimental) statistics. #68034 (Robert Schulze).
- Split
00284_external_aggregation.sql
. #68037 (Robert Schulze). - Update version_date.tsv and changelog after v24.7.3.42-stable. #68045 (robot-clickhouse).
- Update test_drop_is_lock_free/test.py. #68051 (Alexander Tokmakov).
- Fixes #67865. #68054 (Konstantin Bogdanov).
- Disable randomization of
trace_profile_events
in clickhouse-test. #68058 (Alexey Milovidov). - Minor CMake cleanup. #68069 (Robert Schulze).
- If the test cluster is overloaded, sometimes simple query execution can take more time than
Buffer
's max time to flush. This PR doubles the timeout and allows to skip the check in case of significant latency. #68072 (pufit). - Fix flaky
02675_profile_events_from_query_log_and_client
. #68097 (Antonio Andelic). - Fix race in
WithRetries
. #68106 (Antonio Andelic). - Add empty cell to reports when time is missing. #68112 (János Benjamin Antal).
- Fix test
00900_long_parquet_load
. #68130 (Alexey Milovidov). - tests: fix 01246_buffer_flush flakiness due to slow trace_log flush. #68134 (Azat Khuzhin).
- Only use Field::safeGet - Field::get prone to type punning. #68135 (Yakov Olkhovskiy).
- tests: attempt to fix 01600_parts_states_metrics_long (by forbid parallel run). #68136 (Azat Khuzhin).
- Fix01710 Timeout. #68138 (jsc0218).
- Remove the extra cell from reports when it is not necessary. #68145 (János Benjamin Antal).
- Remove "Processing configuration file" message from clickhouse-local. #68157 (Azat Khuzhin).
- tests: fix 02122_join_group_by_timeout flakiness. #68160 (Azat Khuzhin).
- Fix
test_cluster_all_replicas
. #68178 (Alexey Milovidov). - Fix leftovers. #68181 (Alexey Milovidov).
- Fix test
01172_transaction_counters
. #68182 (Alexey Milovidov). - Refactor tests for (experimental) statistics. #68186 (Robert Schulze).
- Remove Log engine from Kafka integration tests. #68200 (János Benjamin Antal).
- [Green CI] Fix test 01903_correct_block_size_prediction_with_default. #68203 (Pablo Marcos).
- Replace segfault in Replicated database with logical error. #68250 (Alexander Tokmakov).
- Backported in #68423: tests: make 01600_parts_states_metrics_long better. #68265 (Azat Khuzhin).
- Backported in #68374: Rename: S3DiskNoKeyErrors -> DiskS3NoSuchKeyErrors. #68361 (Miсhael Stetsyuk).
- Backported in #68637: Check for invalid regexp in JSON SKIP REGEXP section. #68451 (Kruglov Pavel).
- Backported in #68485: Better inference of date times 2. #68452 (Kruglov Pavel).
- Backported in #68540: CI: Native build for package_aarch64. #68457 (Max K.).
- Backported in #68518: Minor update in Dynamic/JSON serializations. #68459 (Kruglov Pavel).
- Backported in #68558: CI: Minor release workflow fix. #68536 (Max K.).
- Backported in #68576: CI: Tidy build timeout from 2h to 3h. #68567 (Max K.).