mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
89 KiB
89 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.10.1.2812-stable (9cd0a3738d
) FIXME as compared to v24.10.1.1-new (b12a367741
)
Backward Incompatible Change
- Allow to write
SETTINGS
beforeFORMAT
in a chain of queries withUNION
when subqueries are inside parentheses. This closes #39712. Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. #68614 (Alexey Milovidov). - Reordering of filter conditions from
[PRE]WHERE
clause is now allowed by default. It could be disabled by settingallow_reorder_prewhere_conditions
tofalse
. #70657 (Nikita Taranov). - Fix
optimize_functions_to_subcolumns
optimization (previously could lead toInvalid column type for ColumnUnique::insertRangeFrom. Expected String, got LowCardinality(String)
error), by preservingLowCardinality
type inmapKeys
/mapValues
. #70716 (Azat Khuzhin). - Remove the
idxd-config
library, which has an incompatible license. This also removes the experimental Intel DeflateQPL codec. #70987 (Alexey Milovidov).
New Feature
- MongoDB integration refactored: migration to new driver mongocxx from deprecated Poco::MongoDB, remove support for deprecated old protocol, support for connection by URI, support for all MongoDB types, support for WHERE and ORDER BY statements on MongoDB side, restriction for expression unsupported by MongoDB. #63279 (Kirill Nikiforov).
- A new
--progress-table
option in clickhouse-client prints a table with metrics changing during query execution; a new--enable-progress-table-toggle
is associated with the--progress-table
option, and toggles the rendering of the progress table by pressing the control key (Space). #63689 (Maria Khristenko). - This allows to grant access to the wildcard prefixes.
GRANT SELECT ON db.table_pefix_* TO user
. #65311 (pufit). - Add system.query_metric_log which contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk. #66532 (Pablo Marcos).
- A simple SELECT query can be written with implicit SELECT to enable calculator-style expressions, e.g.,
ch "1 + 2"
. This is controlled by a new setting,implicit_select
. #68502 (Alexey Milovidov). - Support --copy mode for clickhouse local as a shortcut for format conversion #68503. #68583 (Denis Hananein).
- Add support for
arrayUnion
function. #68989 (Peter Nguyen). - Support aggreate function
quantileExactWeightedInterpolated
, which is a interpolated version based on quantileExactWeighted. Some people may wonder why we need a newquantileExactWeightedInterpolated
since we already havequantileExactInterpolatedWeighted
. The reason is the new one is more accurate than the old one. BTW, it is for spark compatiability in Apache Gluten. #69619 (李扬). - Support function arrayElementOrNull. It returns null if array index is out of range or map key not found. #69646 (李扬).
- Allows users to specify regular expressions through new
message_regexp
andmessage_regexp_negative
fields in theconfig.xml
file to filter out logging. The logging is applied to the formatted un-colored text for the most intuitive developer experience. #69657 (Peter Nguyen). - Support Dynamic type in most functions by executing them on internal types inside Dynamic. #69691 (Pavel Kruglov).
- Re-added
RIPEMD160
function, which computes the RIPEMD-160 cryptographic hash of a string. Example:SELECT HEX(RIPEMD160('The quick brown fox jumps over the lazy dog'))
returns37F332F68DB77BD9D7EDD4969571AD671CF9DD3B
. #70087 (Dergousov Maxim). - Allow to cache read files for object storage table engines and data lakes using hash from ETag + file path as cache key. #70135 (Kseniia Sumarokova).
- Support reading Iceberg tables on HDFS. #70268 (flynn).
- Allow to read/write JSON type as binary string in RowBinary format under settings
input_format_binary_read_json_as_string/output_format_binary_write_json_as_string
. #70288 (Pavel Kruglov). - Allow to serialize/deserialize JSON column as single String column in Native format. For output use setting
output_format_native_write_json_as_string
. For input, use serialization version1
before the column data. #70312 (Pavel Kruglov). - Supports standard CTE,
with insert
, as previously only supportsinsert ... with ...
. #70593 (Shichao Jin).
Performance Improvement
- Support minmax index for
pointInPolygon
. #62085 (JackyWoo). - Add support for parquet bloom filters. #62966 (Arthur Passos).
- Lock-free parts rename to avoid INSERT affect SELECT (due to parts lock) (under normal circumstances with
fsync_part_directory
, QPS of SELECT with INSERT in parallel, increased 2x, under heavy load the effect is even bigger). Note, this only includesReplicatedMergeTree
for now. #64955 (Azat Khuzhin). - Respect
ttl_only_drop_parts
onmaterialize ttl
; only read necessary columns to recalculate TTL and drop parts by replacing them with an empty one. #65488 (Andrey Zvonov). - Refactor
IDisk
andIObjectStorage
for better performance. Tables fromplain
andplain_rewritable
object storages will initialize faster. #68146 (Alexey Milovidov). - Optimized thread creation in the ThreadPool to minimize lock contention. Thread creation is now performed outside of the critical section to avoid delays in job scheduling and thread management under high load conditions. This leads to a much more responsive ClickHouse under heavy concurrent load. #68694 (filimonov).
- Enable reading LowCardinality string columns from ORC. #69481 (李扬).
- Added an ability to parse data directly into sparse columns. #69828 (Anton Popov).
- Supports parallel reading of parquet row groups and prefetching of row groups in single-threaded mode. #69862 (LiuNeng).
- Improved performance of parsing formats with high number of missed values (e.g.
JSONEachRow
). #69875 (Anton Popov). - Use
LowCardinality
forProfileEvents
in system logs such aspart_log
,query_views_log
,filesystem_cache_log
. #70152 (Alexey Milovidov). - Improve performance of FromUnixTimestamp/ToUnixTimestamp functions. #71042 (kevinyhzou).
Improvement
- Allow parametrised SQL aliases. #50665 (Anton Kozlov).
- Fixed #57616 this problem occurs because all positive number arguments are automatically identified as
uint64
type, leading to an inability to match int type data insummapfiltered
. the issue of non-matching is indeed confusing, as theuint64
parameters are not specified by the user. additionally, if the arguments are[1,2,3,toint8(-3)]
, due to thegetleastsupertype()
, these parameters will be uniformly treated asint
type, causing'1,2,3'
to also fail in matching theuint
type data insummapfiltered
. #58408 (Chen768959). ALTER TABLE .. REPLACE PARTITION
doesn't wait anymore for mutations/merges that happen in other partitions. #59138 (Vasily Nemkov).- Refreshable materialized views are now supported in Replicated databases. #60669 (Michael Kolupaev).
- Symbolic links for tables in the
data/database_name/
directory are created for the actual paths to the table's data, depending on the storage policy, instead of thestore/...
directory on the default disk. #61777 (Kirill). - Apply configuration updates in global context object. It fixes issues like #62308. #62944 (Amos Bird).
- Reworked settings that control the behavior of parallel replicas algorithms. A quick recap: ClickHouse has four different algorithms for parallel reading involving multiple replicas, which is reflected in the setting
parallel_replicas_mode
, the default value for it isread_tasks
Additionally, the toggle-switch settingenable_parallel_replicas
has been added. #63151 (Alexey Milovidov). - Fix
ReadSettings
not using user set values, because defaults were only used. #65625 (Kseniia Sumarokova). - While parsing an Enum field from JSON, a string containing an integer will be interpreted as the corresponding Enum element. This closes #65119. #66801 (scanhex12).
- Allow
TRIM
-ingLEADING
orTRAILING
empty string as a no-op. Closes #67792. #68455 (Peter Nguyen). - Support creating a table with a query:
CREATE TABLE ... CLONE AS ...
. It clones the source table's schema and then attaches all partitions to the newly created table. This feature is only supported with tables of theMergeTree
family Closes #65015. #69091 (tuanpach). - In Gluten ClickHouse, Spark's timestamp type is mapped to ClickHouse's datetime64(6) type. When casting timestamp '2012-01-01 00:11:22' as a string, Spark returns '2012-01-01 00:11:22', while Gluten ClickHouse returns '2012-01-01 00:11:22.000000'. #69179 (Wenzheng Liu).
- Always use the new analyzer to calculate constant expressions when
enable_analyzer
is set totrue
. Support calculation ofexecutable()
table function arguments without usingSELECT
query for constant expression. #69292 (Dmitry Novik). - Add
enable_secure_identifiers
to disallow insecure identifiers. #69411 (tuanpach). - Add
show_create_query_identifier_quoting_rule
to define identifier quoting behavior of the show create query result. Possible values: -user_display
: When the identifiers is a keyword. -when_necessary
: When the identifiers is one of{"distinct", "all", "table"}
, or it can cause ambiguity: column names, dictionary attribute names. -always
: Always quote identifiers. #69448 (tuanpach). - Follow-up to https://github.com/ClickHouse/ClickHouse/pull/69346 Point 4 described there will work now as well:. #69563 (Vitaly Baranov).
- Implement generic SerDe between Avro Union and ClickHouse Variant type. Resolves #69713. #69712 (Jiří Kozlovský).
-
- CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses. Now it is supported only for the MergeTree family of table engines. 2. For example, the follow SQL statements will trigger exception in the past, but this PR fixes it: if the destination table do not provide an
ORDER BY
orPRIMARY KEY
expression in the table definition, we will copy that from source table. #69739 (sakulali).
- CREATE TABLE AS will copy PRIMARY KEY, ORDER BY, and similar clauses. Now it is supported only for the MergeTree family of table engines. 2. For example, the follow SQL statements will trigger exception in the past, but this PR fixes it: if the destination table do not provide an
- Added user-level settings
min_free_disk_bytes_to_throw_insert
andmin_free_disk_ratio_to_throw_insert
to prevent insertions on disks that are almost full. #69755 (Marco Vilas Boas). - If you run
clickhouse-client
or other CLI application and it starts up slowly due to an overloaded server, and you start typing your query, such asSELECT
, the previous versions will display the remaining of the terminal echo contents before printing the greetings message, such asSELECTClickHouse local version 24.10.1.1.
instead ofClickHouse local version 24.10.1.1.
. Now it is fixed. This closes #31696. #69856 (Alexey Milovidov). - Add new column readonly_duration to the system.replicas table. Needed to be able to distinguish actual readonly replicas from sentinel ones in alerts. #69871 (Miсhael Stetsyuk).
- Change the join to sort settings type to unsigned int. #69886 (kevinyhzou).
- Support 64-bit XID in Keeper. It can be enabled with
use_xid_64
config. #69908 (Antonio Andelic). - New function getSettingOrDefault() added to return the default value and avoid exception if a custom setting is not found in the current profile. #69917 (Shankar).
- Allow empty needle in function replace, the same behavior with PostgreSQL. #69918 (zhanglistar).
- Enhance OpenTelemetry span logging to include query settings. #70011 (sharathks118).
- Allow empty needle in functions replaceRegexp*, like https://github.com/ClickHouse/ClickHouse/pull/69918. #70053 (zhanglistar).
- Add info to higher-order array functions if lambda result type is unexpected. #70093 (ttanay).
- Keeper improvement: less blocking during cluster changes. #70275 (Antonio Andelic).
- Embedded documentation for settings will be strictly more detailed and complete than the documentation on the website. This is the first step before making the website documentation always auto-generated from the source code. This has long-standing implications: - it will be guaranteed to have every setting; - there is no chance of having default values obsolete; - we can generate this documentation for each ClickHouse version; - the documentation can be displayed by the server itself even without Internet access. Generate the docs on the website from the source code. #70289 (Alexey Milovidov).
- Add
WITH IMPLICIT
andFINAL
keywords to theSHOW GRANTS
command. Fix a minor bug with implicit grants: #70094. #70293 (pufit). - Don't disable nonblocking read from page cache for the entire server when reading from a blocking I/O. #70299 (Antonio Andelic).
- Respect
compatibility
for MergeTree settings. Thecompatibility
value is taken from thedefault
profile on server startup, and default MergeTree settings are changed accordingly. Further changes of thecompatibility
setting do not affect MergeTree settings. #70322 (Nikolai Kochetov). - Clickhouse-client realtime metrics follow-up: restore cursor when ctrl-c cancels query; immediately stop intercepting keystrokes when the query is canceled; display the metrics table if
--progress-table
is on, and toggling is disabled. #70423 (Julia Kartseva). - Command-line arguments for Bool settings are set to true when no value is provided for the argument (e.g.
clickhouse-client --optimize_aggregation_in_order --query "SELECT 1"
). #70459 (davidtsuk). - Avoid spamming the logs with large HTTP response bodies in case of errors during inter-server communication. #70487 (Vladimir Cherkasov).
- Added a new setting
max_parts_to_move
to control the maximum number of parts that can be moved at once. #70520 (Vladimir Cherkasov). - Limit the frequency of certain log messages. #70601 (Alexey Milovidov).
- Don't do validation when synchronizing user_directories from keeper. #70644 (Raúl Marín).
- Introduced a special (experimental) mode of a merge selector for MergeTree tables which makes it more aggressive for the partitions that are close to the limit by the number of parts. It is controlled by the
merge_selector_use_blurry_base
MergeTree-level setting. #70645 (Nikita Mikhaylov). CHECK TABLE
withPART
qualifier was incorrectly formatted in the client. #70660 (Alexey Milovidov).- Support write column index and offset index using parquet native writer. #70669 (LiuNeng).
- Support parse
DateTime64
for microseond and timezone in joda syntax. #70737 (kevinyhzou). - Changed an approach to figure out if a cloud storage supports batch delete or not. #70786 (Vitaly Baranov).
- Support for Parquet page V2 on native reader. #70807 (Arthur Passos).
- Add an HTML page for visualizing merges. #70821 (Alexey Milovidov).
- Backported in #71234: Do not call the object storage API when listing directories, as this may be cost-inefficient. Instead, store the list of filenames in the memory. The trade-offs are increased initial load time and memory required to store filenames. #70823 (Julia Kartseva).
- A check if table has both
storage_policy
anddisk
set after alter query is added. A check if a new storage policy is compatible with an old one when usingdisk
setting is added. #70839 (Kirill). - Add system.s3_queue_settings and system.azure_queue_settings. #70841 (Kseniia Sumarokova).
- Functions
base58Encode
andbase58Decode
now accept arguments of typeFixedString
. Example:SELECT base58Encode(toFixedString('plaintext', 9));
. #70846 (Faizan Patel). - Add the
partition
column to every entry type of the part log. Previously, it was set only for some entries. This closes #70819. #70848 (Alexey Milovidov). - Add merge start and mutate start events into
system.part_log
which helps with merges analysis and visualization. #70850 (Alexey Milovidov). - Do not call the LIST object storage API when determining if a file or directory exists on the plain rewritable disk, as it can be cost-inefficient. #70852 (Julia Kartseva).
- Add a profile event about the number of merged source parts. It allows the monitoring of the fanout of the merge tree in production. #70908 (Alexey Milovidov).
- Reduce the number of object storage HEAD API requests in the plain_rewritable disk. #70915 (Julia Kartseva).
- Background downloads to filesystem cache was enabled back. #70929 (Nikita Taranov).
- Add a new merge selector algorithm, named
Trivial
, for professional usage only. It is worse than theSimple
merge selector. #70969 (Alexey Milovidov).
Bug Fix (user-visible misbehavior in an official stable release)
- Fix toHour-like conversion functions' monotonicity when optional time zone argument is passed. #60264 (Amos Bird).
- Relax
supportsPrewhere
check for StorageMerge. This fixes #61064. It was hardened unnecessarily in #60082. #61091 (Amos Bird). - Fix
use_concurrency_control
setting handling for properconcurrent_threads_soft_limit_num
limit enforcing. This enables concurrency control by default because previously it was broken. #61473 (Sergei Trifonov). - Fix incorrect JOIN ON section optimization in case of
IS NULL
check under any other function (likeNOT
) that may lead to wrong results. Closes #67915. #68049 (Vladimir Cherkasov). - Prevent
ALTER
queries that would make theCREATE
query of tables invalid. #68574 (János Benjamin Antal). - Fix inconsistent AST formatting for
negate
(-
) andNOT
functions with tuples and arrays. #68600 (Vladimir Cherkasov). - Fix insertion of incomplete type into Dynamic during deserialization. It could lead to
Parameter out of bound
errors. #69291 (Pavel Kruglov). - Fix inf loop after
restore replica
in the replicated merge tree with zero copy. #69293 (MikhailBurdukov). - Return back default value of
processing_threads_num
as number of cpu cores in storageS3Queue
. #69384 (Kseniia Sumarokova). - Bypass try/catch flow when de/serializing nested repeated protobuf to nested columns ( fixes #41971 ). #69556 (Eliot Hautefeuille).
- Fix vrash during insertion into FixedString column in PostgreSQL engine. #69584 (Pavel Kruglov).
- Fix crash when executing
create view t as (with recursive 42 as ttt select ttt);
. #69676 (Han Fei). - Added
strict_once
mode to aggregate functionwindowFunnel
to avoid counting one event several times in case it matches multiple conditions, close #21835. #69738 (Vladimir Cherkasov). - Fixed
maxMapState
throwing 'Bad get' if value type is DateTime64. #69787 (Michael Kolupaev). - Fix
getSubcolumn
withLowCardinality
columns by overridinguseDefaultImplementationForLowCardinalityColumns
to returntrue
. #69831 (Miсhael Stetsyuk). - Fix permanent blocked distributed sends if DROP of distributed table fails. #69843 (Azat Khuzhin).
- Fix non-cancellable queries containing WITH FILL with NaN keys. This closes #69261. #69845 (Alexey Milovidov).
- Fix analyzer default with old compatibility value. #69895 (Raúl Marín).
- Don't check dependencies during CREATE OR REPLACE VIEW during DROP of old table. Previously CREATE OR REPLACE query failed when there are dependent tables of the recreated view. #69907 (Pavel Kruglov).
- Implement missing decimal cases for
zeroField
. Fixes #69730. #69978 (Arthur Passos). - Now SQL security will work with parameterized views correctly. #69984 (pufit).
- Closes #69752. #69985 (pufit).
- Fixed a bug when the timezone could change the result of the query with a
Date
orDate32
arguments. #70036 (Yarik Briukhovetskyi). - Fixes
Block structure mismatch
for queries with nested views andWHERE
condition. Fixes #66209. #70054 (Nikolai Kochetov). - Avoid reusing columns among different named tuples when evaluating
tuple
functions. This fixes #70022. #70103 (Amos Bird). - Fix wrong LOGICAL_ERROR when replacing literals in ranges. #70122 (Pablo Marcos).
- Check for Nullable(Nothing) type during ALTER TABLE MODIFY COLUMN/QUERY to prevent tables with such data type. #70123 (Pavel Kruglov).
- Proper error message for illegal query
JOIN ... ON *
, close #68650. #70124 (Vladimir Cherkasov). - Fix wrong result with skipping index. #70127 (Raúl Marín).
- Fix data race in ColumnObject/ColumnTuple decompress method that could lead to heap use after free. #70137 (Pavel Kruglov).
- Fix possible hung in ALTER COLUMN with Dynamic type. #70144 (Pavel Kruglov).
- Now ClickHouse will consider more errors as retriable and will not mark data parts as broken in case of such errors. #70145 (alesapin).
- Use correct
max_types
parameter during Dynamic type creation for JSON subcolumn. #70147 (Pavel Kruglov). - Fix the password being displayed in
system.query_log
for users with bcrypt password authentication method. #70148 (Nikolay Degterinsky). - Fix event counter for native interface (InterfaceNativeSendBytes). #70153 (Yakov Olkhovskiy).
- Fix possible crash in JSON column. #70172 (Pavel Kruglov).
- Fix multiple issues with arrayMin and arrayMax. #70207 (Raúl Marín).
- Respect setting allow_simdjson in JSON type parser. #70218 (Pavel Kruglov).
- Fix server segfault on creating a materialized view with two selects and an
INTERSECT
, e.g.CREATE MATERIALIZED VIEW v0 AS (SELECT 1) INTERSECT (SELECT 1);
. #70264 (Konstantin Bogdanov). - Don't modify global settings with startup scripts. Previously, changing a setting in a startup script would change it globally. #70310 (Antonio Andelic).
- Fix ALTER of Dynamic type with reducing max_types parameter that could lead to server crash. #70328 (Pavel Kruglov).
- Fix crash when using WITH FILL incorrectly. #70338 (Raúl Marín).
- Fix possible use-after-free in
SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf
. #70358 (Azat Khuzhin). - Fix crash during GROUP BY JSON sub-object subcolumn. #70374 (Pavel Kruglov).
- Don't prefetch parts for vertical merges if part has no rows. #70452 (Antonio Andelic).
- Fix crash in WHERE with lambda functions. #70464 (Raúl Marín).
- Fix table creation with
CREATE ... AS table_function()
with databaseReplicated
and unavailable table function source on secondary replica. #70511 (Kseniia Sumarokova). - Ignore all output on async insert with
wait_for_async_insert=1
. Closes #62644. #70530 (Konstantin Bogdanov). - Ignore frozen_metadata.txt while traversing shadow directory from system.remote_data_paths. #70590 (Aleksei Filatov).
- Fix creation of stateful window functions on misaligned memory. #70631 (Raúl Marín).
- Fixed rare crashes in
SELECT
-s and merges after adding a column ofArray
type with non-empty default expression. #70695 (Anton Popov). - Insert into table function s3 respect query settings. #70696 (Vladimir Cherkasov).
- Fix infinite recursion when infering a proto schema with skip unsupported fields enabled. #70697 (Raúl Marín).
- Backported in #71122:
GroupArraySortedData
uses a PODArray with non-POD elements, manually calling constructors and destructors for the elements as needed. But it wasn't careful enough: in two places it forgot to call destructor, in one place it left elements uninitialized if an exception is thrown when deserializing previous elements. ThenGroupArraySortedData
's destructor called destructors on uninitialized elements and crashed:2024.10.17 22:58:23.523790 [ 5233 ] {} <Fatal> BaseDaemon: ########## Short fault info ############ 2024.10.17 22:58:23.523834 [ 5233 ] {} <Fatal> BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) Received signal 11 2024.10.17 22:58:23.523862 [ 5233 ] {} <Fatal> BaseDaemon: Signal description: Segmentation fault 2024.10.17 22:58:23.523883 [ 5233 ] {} <Fatal> BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523908 [ 5233 ] {} <Fatal> BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.523936 [ 5233 ] {} <Fatal> BaseDaemon: ######################################## 2024.10.17 22:58:23.523959 [ 5233 ] {} <Fatal> BaseDaemon: (version 24.6.1.4609 (official build), build id: 5423339A6571004018D55BBE05D464AFA35E6718, git hash: fa6cdfda8a94890eb19bc7f22f8b0b56292f7a26) (from thread 682) (query_id: 6c8a33a2-f45a-4a3b-bd71-ded6a1c9ccd3::202410_534066_534078_2) (query: ) Received signal Segmentation fault (11) 2024.10.17 22:58:23.523977 [ 5233 ] {} <Fatal> BaseDaemon: Address: 0x8f. Access: . Address not mapped to object. 2024.10.17 22:58:23.523993 [ 5233 ] {} <Fatal> BaseDaemon: Stack trace: 0x0000aaaac4b78308 0x0000ffffb7701850 0x0000aaaac0104855 0x0000aaaac01048a0 0x0000aaaac501e84c 0x0000aaaac7c510d0 0x0000aaaac7c4ba20 0x0000aaaac968bbfc 0x0000aaaac968fab0 0x0000aaaac969bf50 0x0000aaaac9b7520c 0x0000aaaac9b74c74 0x0000aaaac9b8a150 0x0000aaaac9b809f0 0x0000aaaac9b80574 0x0000aaaac9b8e364 0x0000aaaac9b8e4fc 0x0000aaaac94f4328 0x0000aaaac94f428c 0x0000aaaac94f7df0 0x0000aaaac98b5a3c 0x0000aaaac950b234 0x0000aaaac49ae264 0x0000aaaac49b1dd0 0x0000aaaac49b0a80 0x0000ffffb755d5c8 0x0000ffffb75c5edc 2024.10.17 22:58:23.524817 [ 5233 ] {} <Fatal> BaseDaemon: 0. signalHandler(int, siginfo_t*, void*) @ 0x000000000c6f8308 2024.10.17 22:58:23.524917 [ 5233 ] {} <Fatal> BaseDaemon: 1. ? @ 0x0000ffffb7701850 2024.10.17 22:58:23.524962 [ 5233 ] {} <Fatal> BaseDaemon: 2. DB::Field::~Field() @ 0x0000000007c84855 2024.10.17 22:58:23.525012 [ 5233 ] {} <Fatal> BaseDaemon: 3. DB::Field::~Field() @ 0x0000000007c848a0 2024.10.17 22:58:23.526626 [ 5233 ] {} <Fatal> BaseDaemon: 4. DB::IAggregateFunctionDataHelper<DB::(anonymous namespace)::GroupArraySortedData<DB::Field, (DB::(anonymous namespace)::GroupArraySortedStrategy)0>, DB::(anonymous namespace)::GroupArraySorted<DB::(anonymous namespace)::GroupArraySortedData<DB::Field, (DB::(anonymous namespace)::GroupArraySortedStrategy)0>, DB::Field>>::destroy(char*) const (.5a6a451027f732f9fd91c13f4a13200c) @ 0x000000000cb9e84c 2024.10.17 22:58:23.527322 [ 5233 ] {} <Fatal> BaseDaemon: 5. DB::SerializationAggregateFunction::deserializeBinaryBulk(DB::IColumn&, DB::ReadBuffer&, unsigned long, double) const @ 0x000000000f7d10d0 2024.10.17 22:58:23.528470 [ 5233 ] {} <Fatal> BaseDaemon: 6. DB::ISerialization::deserializeBinaryBulkWithMultipleStreams(COW<DB::IColumn>::immutable_ptr<DB::IColumn>&, unsigned long, DB::ISerialization::DeserializeBinaryBulkSettings&, std::shared_ptr<DB::ISerialization::DeserializeBinaryBulkState>&, std::unordered_map<String, COW<DB::IColumn>::immutable_ptr<DB::IColumn>, std::hash<String>, std::equal_to<String>, std::allocator<std::pair<String const, COW<DB::IColumn>::immutable_ptr<DB::IColumn>>>>*) const @ 0x000000000f7cba20 2024.10.17 22:58:23.529213 [ 5233 ] {} <Fatal> BaseDaemon: 7. DB::MergeTreeReaderCompact::readData(DB::NameAndTypePair const&, COW<DB::IColumn>::immutable_ptr<DB::IColumn>&, unsigned long, std::function<DB::ReadBuffer* (DB::ISerialization::SubstreamPath const&)> const&) @ 0x000000001120bbfc 2024.10.17 22:58:23.529277 [ 5233 ] {} <Fatal> BaseDaemon: 8. DB::MergeTreeReaderCompactSingleBuffer::readRows(unsigned long, unsigned long, bool, unsigned long, std::vector<COW<DB::IColumn>::immutable_ptr<DB::IColumn>, std::allocator<COW<DB::IColumn>::immutable_ptr<DB::IColumn>>>&) @ 0x000000001120fab0 2024.10.17 22:58:23.529319 [ 5233 ] {} <Fatal> BaseDaemon: 9. DB::MergeTreeSequentialSource::generate() @ 0x000000001121bf50 2024.10.17 22:58:23.529346 [ 5233 ] {} <Fatal> BaseDaemon: 10. DB::ISource::tryGenerate() @ 0x00000000116f520c 2024.10.17 22:58:23.529653 [ 5233 ] {} <Fatal> BaseDaemon: 11. DB::ISource::work() @ 0x00000000116f4c74 2024.10.17 22:58:23.529679 [ 5233 ] {} <Fatal> BaseDaemon: 12. DB::ExecutionThreadContext::executeTask() @ 0x000000001170a150 2024.10.17 22:58:23.529733 [ 5233 ] {} <Fatal> BaseDaemon: 13. DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic<bool>*) @ 0x00000000117009f0 2024.10.17 22:58:23.529763 [ 5233 ] {} <Fatal> BaseDaemon: 14. DB::PipelineExecutor::executeStep(std::atomic<bool>*) @ 0x0000000011700574 2024.10.17 22:58:23.530089 [ 5233 ] {} <Fatal> BaseDaemon: 15. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001170e364 2024.10.17 22:58:23.530277 [ 5233 ] {} <Fatal> BaseDaemon: 16. DB::PullingPipelineExecutor::pull(DB::Block&) @ 0x000000001170e4fc 2024.10.17 22:58:23.530295 [ 5233 ] {} <Fatal> BaseDaemon: 17. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() @ 0x0000000011074328 2024.10.17 22:58:23.530318 [ 5233 ] {} <Fatal> BaseDaemon: 18. DB::MergeTask::ExecuteAndFinalizeHorizontalPart::execute() @ 0x000000001107428c 2024.10.17 22:58:23.530339 [ 5233 ] {} <Fatal> BaseDaemon: 19. DB::MergeTask::execute() @ 0x0000000011077df0 2024.10.17 22:58:23.530362 [ 5233 ] {} <Fatal> BaseDaemon: 20. DB::SharedMergeMutateTaskBase::executeStep() @ 0x0000000011435a3c 2024.10.17 22:58:23.530384 [ 5233 ] {} <Fatal> BaseDaemon: 21. DB::MergeTreeBackgroundExecutor<DB::DynamicRuntimeQueue>::threadFunction() @ 0x000000001108b234 2024.10.17 22:58:23.530410 [ 5233 ] {} <Fatal> BaseDaemon: 22. ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::worker(std::__list_iterator<ThreadFromGlobalPoolImpl<false, true>, void*>) @ 0x000000000c52e264 2024.10.17 22:58:23.530448 [ 5233 ] {} <Fatal> BaseDaemon: 23. void std::__function::__policy_invoker<void ()>::__call_impl<std::__function::__default_alloc_func<ThreadFromGlobalPoolImpl<false, true>::ThreadFromGlobalPoolImpl<void ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>(void&&)::'lambda'(), void ()>>(std::__function::__policy_storage const*) @ 0x000000000c531dd0 2024.10.17 22:58:23.530476 [ 5233 ] {} <Fatal> BaseDaemon: 24. void* std::__thread_proxy[abi:v15000]<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>>(void*) @ 0x000000000c530a80 2024.10.17 22:58:23.530514 [ 5233 ] {} <Fatal> BaseDaemon: 25. ? @ 0x000000000007d5c8 2024.10.17 22:58:23.530534 [ 5233 ] {} <Fatal> BaseDaemon: 26. ? @ 0x00000000000e5edc 2024.10.17 22:58:23.530551 [ 5233 ] {} <Fatal> BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read. 2024.10.17 22:58:23.531083 [ 5233 ] {} <Fatal> BaseDaemon: Report this error to https://github.com/ClickHouse/ClickHouse/issues 2024.10.17 22:58:23.531294 [ 5233 ] {} <Fatal> BaseDaemon: Changed settings: max_insert_threads = 4, max_threads = 42, use_hedged_requests = false, distributed_foreground_insert = true, alter_sync = 0, enable_memory_bound_merging_of_aggregation_results = true, cluster_for_parallel_replicas = 'default', do_not_merge_across_partitions_select_final = false, log_queries = true, log_queries_probability = 1., max_http_get_redirects = 10, enable_deflate_qpl_codec = false, enable_zstd_qat_codec = false, query_profiler_real_time_period_ns = 0, query_profiler_cpu_time_period_ns = 0, max_bytes_before_external_group_by = 90194313216, max_bytes_before_external_sort = 90194313216, max_memory_usage = 180388626432, backup_restore_keeper_retry_max_backoff_ms = 60000, cancel_http_readonly_queries_on_client_close = true, max_table_size_to_drop = 1000000000000, max_partition_size_to_drop = 1000000000000, default_table_engine = 'ReplicatedMergeTree', mutations_sync = 0, optimize_trivial_insert_select = false, database_replicated_allow_only_replicated_engine = true, cloud_mode = true, cloud_mode_engine = 2, distributed_ddl_output_mode = 'none_only_active', distributed_ddl_entry_format_version = 6, async_insert_max_data_size = 10485760, async_insert_busy_timeout_max_ms = 1000, enable_filesystem_cache_on_write_operations = true, load_marks_asynchronously = true, allow_prefetched_read_pool_for_remote_filesystem = true, filesystem_prefetch_max_memory_usage = 18038862643, filesystem_prefetches_limit = 200, compatibility = '24.6', insert_keeper_max_retries = 20, allow_experimental_materialized_postgresql_table = false, date_time_input_format = 'best_effort'
. #70820 (Michael Kolupaev). - Disable enable_named_columns_in_function_tuple by default. #70833 (Raúl Marín).
- Fix S3Queue table engine setting processing_threads_num not being effective in case it was deduced from the number of cpu cores on the server. #70837 (Kseniia Sumarokova).
- Normalize named tuple arguments in aggregation states. This fixes #69732 . #70853 (Amos Bird).
- Fix a logical error due to negative zeros in the two-level hash table. This closes #70973. #70979 (Alexey Milovidov).
- Backported in #71214: Fix logical error in
StorageS3Queue
"Cannot create a persistent node in /processed since it already exists". #70984 (Kseniia Sumarokova). - Backported in #71243: Fixed named sessions not being closed and hanging on forever under certain circumstances. #70998 (Márcio Martins).
- Backported in #71157: Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. #71089 (Shichao Jin).
- Backported in #71265: Fix wrong value in system.query_metric_log due to unexpected race condition. #71124 (Pablo Marcos).
- Backported in #71331: Fix async inserts with empty blocks via native protocol. #71312 (Anton Popov).
Build/Testing/Packaging Improvement
- Docker in integration tests runner is updated to latest version. It was previously pinned u until patch release 24.0.3 was out. https://github.com/moby/moby/issues/45770#issuecomment-1618255130. - HDFS image was deprecated and not running with current docker version. Switched to newer version of a derivative image based on ubuntu. - HDFS tests were hardened to allow them to run with python-repeat. #66867 (Ilya Yatsishin).
- Alpine docker images now use ubuntu 22.04 as glibc donor, results in upgrade of glibc version delivered with alpine images from 2.31 to 2.35. #69033 (filimonov).
- Makes dbms independent from clickhouse_functions. #69914 (Raúl Marín).
- Fix FreeBSD compilation of the MariaDB connector. #70007 (Raúl Marín).
- Building on Apple Mac OS X Darwin does not produce strange warnings anymore. #70411 (Alexey Milovidov).
- Fix building with ARCH_NATIVE CMake flag. #70585 (Daniil Gentili).
- The universal installer will download Musl build on Alpine Linux. Some Docker containers are using Alpine Linux, but it was not possible to install ClickHouse there with
curl https://clickhouse.com/ | sh
. #70767 (Alexey Milovidov).
NO CL CATEGORY
- Backported in #71259:. #71220 (Raúl Marín).
NO CL ENTRY
- NO CL ENTRY: 'Revert "JSONCompactWithProgress query output format"'. #69989 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Support CREATE OR REPLACE VIEW atomically"'. #70535 (Raúl Marín).
- NO CL ENTRY: 'Revert "Revert "Support CREATE OR REPLACE VIEW atomically""'. #70536 (Raúl Marín).
- NO CL ENTRY: 'Revert "Add projections size to system.projections"'. #70858 (Alexey Milovidov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Allow writing argument of
has
orhasAny
orhasAll
as string values if array element type isEnum
. #56555 (Duc Canh Le). - Rename FileSegmentKind::Ephemeral and other changes. #66600 (Vladimir Cherkasov).
- Closes #67345. #67346 (KrJin).
- Because it is too complicated to support. #68410 (Nikolai Kochetov).
- Fix 01600_parts_states_metrics_long flakiness. #68521 (Azat Khuzhin).
- Reduce client start time in debug/sanitizer mode. #68980 (Raúl Marín).
- Closes #69038. #69040 (Nikolay Degterinsky).
- Better exception for unsupported full_text index with non-full parts. #69067 (Vladimir Cherkasov).
- Catch additional zk connection erros while creating table and make sure to cleanup dirs if necessary for retries. #69093 (Sumit).
- Update version_date.tsv and changelog after v24.7.5.37-stable. #69185 (robot-clickhouse).
- DOCS: Replace live view with refreshable since the former is deprecated. #69392 (Damian Kula).
- Update ORC to the current HEAD. #69473 (Nikita Taranov).
- Make a test ready for flaky check. #69586 (Alexander Tokmakov).
- Support antlr parser to parse sql with some keywords as alias, make the behaviour same as the clickhouse-server - remove redundant
for
in thekeyword
field. #69614 (Z.H.). - Allow default implementations for null in function mapFromArrays for spark compatiability in apache gluten. Current change doesn't have any side effects on clickhouse in theory. #69715 (李扬).
- Fix exception message in AzureBlobStorage. #69728 (Pavel Kruglov).
- Add test parsing s3 URL with a bucket name including a dot. #69743 (Kaushik Iska).
- Make
clang-tidy
happy. #69765 (Konstantin Bogdanov). - Prepare to enable
clang-tidy
readability-else-after-return
. #69768 (Konstantin Bogdanov). - S3Queue: support having deprecated settings to not fail server startup. #69769 (Kseniia Sumarokova).
- Use only adaptive heuristic to choose task sizes for remote reading. #69778 (Nikita Taranov).
- Remove unused buggy code. #69780 (Raúl Marín).
- Fix bugfix check. #69789 (Antonio Andelic).
- Followup for #63279. #69790 (Vladimir Cherkasov).
- Update version after release. #69816 (robot-clickhouse).
- Update ext-dict-functions.md. #69819 (kurikuQwQ).
- Allow cyrillic characters in generated contributor names. #69820 (Raúl Marín).
- CI: praktika integration 1. #69822 (Max Kainov).
- Fix
test_delayed_replica_failover
. #69826 (Antonio Andelic). - minor change, less conflicts. #69830 (Vladimir Cherkasov).
- Improve error message DDLWorker.cpp. #69835 (Denny Crane).
- Fix typo in description: mutation_sync -> mutations_sync. #69838 (Alexander Gololobov).
- Fix changelog. #69841 (Alexey Milovidov).
- This closes #49940. #69842 (Alexey Milovidov).
- This closes #51036. #69844 (Alexey Milovidov).
- Update README.md - Update meetups. #69849 (Tanya Bragin).
- Revert #69790 and #63279. #69850 (Alexey Milovidov).
- See #63279. #69851 (Alexey Milovidov).
- Add a test for #50928. #69852 (Alexey Milovidov).
- Add a test for #55981. #69853 (Alexey Milovidov).
- Add a test for #56823. #69854 (Alexey Milovidov).
- This closes #62350. #69855 (Alexey Milovidov).
- Refactor functions and variables in statistics code. #69860 (Robert Schulze).
- Resubmit #63279. #69861 (Vladimir Cherkasov).
- Improve stateless test runner. #69864 (Alexey Katsman).
- Adjust fast test time limit a bit. #69874 (Raúl Marín).
- Add initial 24.9 CHANGELOG. #69876 (Raúl Marín).
- Fix test
01278_random_string_utf8
. #69878 (Alexey Milovidov). - Fix minor fuzzer issue with experimental statistics. #69881 (Robert Schulze).
- Fix linking after settings refactoring. #69882 (Robert Schulze).
- Add Proj Obsolete Setting. #69883 (Shichao Jin).
- Improve remote queries startup time. #69884 (Igor Nikonov).
- Revert "Merge pull request #69032 from alexon1234/include_real_time_execution_in_http_header". #69885 (Alexey Milovidov).
- A dedicated commits from https://github.com/ClickHouse/ClickHouse/pull/61473. #69896 (Mikhail f. Shiryaev).
- Added aliases
time_bucket
(from TimescaleDB) anddate_bin
(from PostgreSQL) fortoStartOfInterval
. #69900 (Yarik Briukhovetskyi). - RIPE is an acronym and thus should be capital. RIPE stands for RACE Integrity Primitives Evaluation and RACE stands for Research and Development in Advanced Communications Technologies in Europe. #69901 (Nikita Mikhaylov).
- Replace error codes with error names in stateless tests. #69906 (Dmitry Novik).
- Move setting to 24.10. #69913 (Raúl Marín).
- Minor: Reduce diff between public and private repo. #69928 (Robert Schulze).
- Followup for #69861. #69930 (Vladimir Cherkasov).
- Fix test_dictionaries_all_layouts_separate_sources. #69962 (Vladimir Cherkasov).
- Fix test_keeper_mntr_data_size. #69965 (Antonio Andelic).
- This closes #49823. #69981 (Alexey Milovidov).
- Add changelog for 24.9. #69982 (Alexey Milovidov).
- Add a test for #45303. #69987 (Alexey Milovidov).
- Update CHANGELOG.md. #69988 (Alexey Milovidov).
- Update README.md. #69991 (Tyler Hannan).
- Disable
03215_parallel_replicas_crash_after_refactoring.sql
for Azure. #69992 (Nikita Mikhaylov). - Update CHANGELOG.md. #69993 (Alexey Milovidov).
- Update CHANGELOG.md. #70004 (Alexey Milovidov).
- Revert "Add RIPEMD160 function". #70005 (Robert Schulze).
- Update CHANGELOG.md. #70009 (Alexey Milovidov).
- Update CHANGELOG.md. #70010 (Alexey Milovidov).
- Make the pylint stricter. #70013 (Mikhail f. Shiryaev).
- Added a setting
restore_replace_external_dictionary_source_to_null
which enables replacing dictionary source with Null on restore for external dictionaries (useful for testing). #70032 (Alexander Tokmakov). isort
is a simple import sorter for the python to comply pep-8 requirements. It will allow to decrease conflicts during sync and beautify the code. The import block is divided into three sub-blocks:standard library
->third-party libraries
->local imports
->.local imports
. Each sub-block is ordered alphabetically with sub-sub-blocksimport X
->from X import Y
. #70038 (Mikhail f. Shiryaev).- Update version_date.tsv and changelog after v24.9.1.3278-stable. #70049 (robot-clickhouse).
- Despite the fact that we set the org-level workflow parameter
PYTHONUNBUFFERED
, it's not inherited in workflows. #70050 (Mikhail f. Shiryaev). - Fix ubsan issue in function sqid. #70061 (Robert Schulze).
- Delete a setting change. #70071 (Nikita Mikhaylov).
- Fix
test_distributed_ddl
. #70075 (Alexander Tokmakov). - Remove unused placeholder from exception message string. #70086 (Alsu Giliazova).
- Better exception message when some of the permission is missing. #70088 (pufit).
- Make vector similarity indexes work with adaptive granularity. #70101 (Robert Schulze).
- Add missing columns
total_rows
,data_compressed_bytes
, anddata_uncompressed_bytes
tosystem.projections
. Part of https://github.com/ClickHouse/ClickHouse/pull/68901. #70106 (Jordi Villar). - Make
00938_fix_rwlock_segfault_long
non flaky. #70109 (Alexey Milovidov). - Remove TODO. #70110 (Alexey Milovidov).
- Change the default threshold to enable hyper threading. #70111 (Jiebin Sun).
- Fixed #69092: if
materialized_postgresql_tables_list=table1(id, code),table(id,name)
(table1
has name that is a substring fortable
)getTableAllowedColumns
method returns[id, code]
fortable
before this fix. #70114 (Kruglov Kirill). - Reduce log level. #70117 (Kseniia Sumarokova).
- Rename
getNumberOfPhysicalCPUCores
and fix its decription. #70130 (Nikita Taranov). - Adding 24.10. #70132 (Tyler Hannan).
- (Re?)-enable libcxx asserts for debug builds. #70134 (Robert Schulze).
- Refactor reading from object storage. #70141 (Kseniia Sumarokova).
- Silence UBSAN for integer overflows in some datetime functions. #70142 (Michael Kolupaev).
- Improve pipdeptree generator for docker images. - Update requirements.txt for the integration tests runner container - Remove some small dependencies, improve
helpers/retry_decorator.py
- Upgrade docker-compose from EOL version 1 to version 2. #70146 (Mikhail f. Shiryaev). - Fix 'QueryPlan was not initialized' in 'loop' with empty MergeTree. #70149 (Michael Kolupaev).
- Remove QueryPlan DataStream. #70158 (Nikolai Kochetov).
- Update test_storage_s3_queue/test.py. #70159 (Kseniia Sumarokova).
- Small docs fix. #70160 (Yarik Briukhovetskyi).
- Test: PR local plan, non-constant in source stream. #70173 (Igor Nikonov).
- Fix performance checks. #70175 (Antonio Andelic).
- Simplify test 03246_range_literal_replacement_works. #70176 (Pablo Marcos).
- Update 01079_parallel_alter_add_drop_column_zookeeper.sh. #70196 (Alexander Tokmakov).
- Require bugfix job for a set of labels. #70197 (Mikhail f. Shiryaev).
- CI: Praktika integration, fast test. #70239 (Max Kainov).
- Avoid
Cannot schedule a task
error when loading parts. #70257 (Kseniia Sumarokova). - Bump usearch to v2.15.2 and SimSIMD to v5.0.0. #70270 (Robert Schulze).
- Instead of balancing tests by
crc32(file_name)
we'll useadd tests to a group with a minimal number of tests
. #70272 (Mikhail f. Shiryaev). - Closes #70263. #70273 (flynn).
- Hide MergeTreeSettings implementation. #70285 (Raúl Marín).
- CI: Remove await feature from release branches. #70294 (Max Kainov).
- Fix
test_keeper_four_word_command
. #70298 (Antonio Andelic). - Update version_date.tsv and changelog after v24.9.2.42-stable. #70301 (robot-clickhouse).
- Synchronize settings with private. #70320 (Alexey Milovidov).
- Add Ignore Option In DeduplicateMergeProjectionMode. #70327 (Shichao Jin).
- CI: Enable Integration Tests for backport PRs. #70329 (Max Kainov).
- There is a failed CI job which is triggered by 03237_create_or_replace_view_atomically_with_atomic_engine. #70330 (tuanpach).
- Fix flaky test
03237_insert_sparse_columns_mem
. #70333 (Anton Popov). - Rename enable_secure_identifiers -> enforce_strict_identifier_format. #70335 (Vladimir Cherkasov).
- Attempt to fix flaky RabbitMQ tests. Maybe closes #45160. #70336 (filimonov).
- Don't fail the stateless check script if we can't collect minio logs. #70350 (Raúl Marín).
- Fix tiny mistake, responsible for some of kafka test flaps. Example report. #70352 (filimonov).
- Closes #69634. #70354 (pufit).
- Fix 02346_fulltext_index_bug52019. #70357 (Vladimir Cherkasov).
- Use new JSON for collecting minio logs. #70359 (Antonio Andelic).
- Update comments in VectorSimilarityCondition (WHERE is not supported). #70360 (Azat Khuzhin).
- Remove 02492_clickhouse_local_context_uaf test. #70363 (Azat Khuzhin).
- Fix
clang-19
build issues. #70412 (Konstantin Bogdanov). - Ignore "Invalid multibyte data detected" error during completion. #70422 (Azat Khuzhin).
- Make QueryPlan explain methods const. #70444 (Alexander Gololobov).
- Fix 0.1 second delay for interactive queries (due to keystroke interceptor). #70445 (Azat Khuzhin).
- Increase lock timeout in attempt to fix 02125_many_mutations. #70448 (Azat Khuzhin).
- Fix order in 03249_dynamic_alter_consistency. #70453 (Alexander Gololobov).
- Fix refreshable MV in system database breaking server startup. #70460 (Michael Kolupaev).
- Fix flaky test_refreshable_mv_in_replicated_db. #70462 (Michael Kolupaev).
- Update version_date.tsv and changelog after v24.8.5.115-lts. #70463 (robot-clickhouse).
- Decrease probability of "Server died" due to 00913_many_threads. #70473 (Azat Khuzhin).
- Fixes for killing leftovers in clikhouse-test. #70474 (Azat Khuzhin).
- Update version_date.tsv and changelog after v24.3.12.75-lts. #70485 (robot-clickhouse).
- Use logging instead of print. #70505 (János Benjamin Antal).
- Remove slow poll() logs in keeper. #70508 (Raúl Marín).
- Add timeouts for retry loops in test_storage_rabbitmq. It should prevent cascading failures of the whole test suite caused by deadloop in one of the test scenarios. Also added small sleeps in a 'tight' loops to make retries bit less agressive. #70510 (filimonov).
- CI: Fix for canceled Sync workflow. #70521 (Max Kainov).
- Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/70412, don't know why it's ok in release build, simply changing
_
to_1
is ok for both release and debug build. #70532 (Chang chen). - Refreshable materialized views are not experimental anymore. #70550 (Michael Kolupaev).
- Fix 24.9 setting compatibility
database_replicated_allow_explicit_uuid
. #70565 (Nikita Fomichev). - Fix typos. #70588 (Alexey Milovidov).
- Vector search: allow to specify HNSW parameter
ef_search
at query time. #70616 (Robert Schulze). - Increase max_rows_to_read limit in some tests. #70617 (Raúl Marín).
- Reduce sync efforts with private. #70634 (Raúl Marín).
- Fix parsing of some formats into sparse columns. #70635 (Anton Popov).
- Fix typos. #70637 (Konstantin Bogdanov).
- Try fix 00180_no_seek_avoiding_when_reading_from_cache. #70640 (Kseniia Sumarokova).
- When the
PR Check
status is set, it's a valid RunConfig job failure. #70643 (Mikhail f. Shiryaev). - Fix timeout in materialized pg tests. #70646 (Kseniia Sumarokova).
- Introduced MergeTree setting which allow to change merge selecting algorithm. However we still have only one algorithm and it's mostly for future experiments. #70647 (alesapin).
- Docs: Follow-up for #70585. #70654 (Robert Schulze).
- Remove strange file. #70662 (Alexey Milovidov).
- Locally I had lots of errors like
'AllocList' does not refer to a value
around places which usedoffsetof
. Changing it to__builtin_offsetof
helped and I didn't debug any further. #70671 (Nikita Mikhaylov). - Adding the report link to a test result and files' list. #70677 (Mikhail f. Shiryaev).
- materialized postgres: minor fixes. #70710 (Kseniia Sumarokova).
- Probably fix flaky test_refreshable_mv_in_replicated_db. #70714 (Michael Kolupaev).
- Move more setting structs to pImpl. #70739 (Raúl Marín).
- Reduce sync effort. #70747 (Raúl Marín).
- Backported in #71198: Check number of arguments for function with Dynamic argument. #70749 (Nikita Taranov).
- Add s3queue settings check for cloud. #70750 (Kseniia Sumarokova).
- Fix readiness/health check for OpenLDAP container. #70755 (Julian Maicher).
- Allow update plan headers for all the steps. #70761 (Nikolai Kochetov).
- Autogenerate documentation for settings. #70768 (Alexey Milovidov).
- Not a logical error. #70770 (Alexey Milovidov).
- CI: Aarch64 build with Asan. #70778 (Max Kainov).
- Minor fix. #70783 (Anton Popov).
- The docs for settings should be located in the source code. Now, the CI supports that. #70784 (Alexey Milovidov).
- Update style-test image. #70785 (Mikhail f. Shiryaev).
- Avoid double finalization of
WriteBuffer
in library bridge. #70799 (Nikolai Kochetov). - Make Array Field serialization consistent. #70803 (Nikolai Kochetov).
- A follow-up for #70785, jwt looks very outdated, and we have issue with conflicting paths. #70815 (Mikhail f. Shiryaev).
- Remove inneficient code. #70816 (Raúl Marín).
- Allow large object files if OMIT_HEAVY_DEBUG_SYMBOLS = 0. #70818 (Michael Kolupaev).
- Add test with distributed queries for 15768. #70834 (Nikita Taranov).
- More setting structs to pImpl and reuse code. #70840 (Raúl Marín).
- Update default HNSW parameter settings. #70873 (Robert Schulze).
- Limiting logging some lines about configs. #70879 (Yarik Briukhovetskyi).
- Fix
limit by
,limit with ties
for distributed and parallel replicas. #70880 (Nikita Taranov). - Fix darwin build. #70894 (Kseniia Sumarokova).
- Add dots for consistency. #70909 (Alexey Milovidov).
- Logical error fix for substrings, found by fuzzer. #70914 (Yarik Briukhovetskyi).
- More setting structs to pImpl. #70942 (Raúl Marín).
- Add logging for mock HTTP servers used in minio integration tests. #70943 (Vitaly Baranov).
- Minor fixups of #70011 and #69918. #70959 (Robert Schulze).
- CI: Do not skip Build report and status fix. #70965 (Max Kainov).
- Fix Keeper entry serialization compatibility. #70972 (Antonio Andelic).
- Update exception message. #70975 (Kseniia Sumarokova).
- Fix
utils/c++expr
option-b
. #70978 (Sergei Trifonov). - Fix
test_keeper_broken_logs
. #70982 (Antonio Andelic). - Fix
01039_test_setting_parse
. #70986 (Alexey Milovidov). - Tests for languages support for Embedded Dictionaries. #71004 (Max Vostrikov).
- Required for internal test runs with the same image build in public CI. #71008 (Ilya Yatsishin).
- Move remaining settings objects to pImpl and start simplification. #71019 (Raúl Marín).
- CI: Rearrange directories for praktika ci. #71029 (Max Kainov).
- Fix assert in RemoteSource::onAsyncJobReady(). #71034 (Igor Nikonov).
- Fix showing error message in ReadBufferFromS3 when retrying. Without this PR information about a retryable failure in
ReadBufferFromS3
could look like this:. #71038 (Vitaly Baranov). - Fix
test_truncate_database
. #71057 (Antonio Andelic). - Fix clickhouse-test useless 5 second delay in case of multiple threads are used. #71069 (Azat Khuzhin).
- Backported in #71142: Followup #70520. #71129 (Vladimir Cherkasov).
- Backported in #71189: Update compatibility setting for
hnsw_candidate_list_size_for_search
. #71133 (Robert Schulze). - Backported in #71222: Fixes for interactive metrics. #71173 (Julia Kartseva).
- Backported in #71205: Maybe not GWPAsan by default. #71174 (Antonio Andelic).
- Backported in #71277: Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. #71216 (Raúl Marín).
- Backported in #71253: Disable enable_named_columns_in_function_tuple for 24.10. #71219 (Raúl Marín).
- Backported in #71303: Improve system.query_metric_log to remove flakiness. #71295 (Pablo Marcos).
- Backported in #71317: Fix debug log timestamp. #71311 (Pablo Marcos).