mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-15 20:24:07 +00:00
96 KiB
96 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.9.1.3278-stable (6d058d82a8
) FIXME as compared to v24.9.1.1-new (e02b434d2f
)
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. #60197 (Alexey Milovidov). - Do not allow explicitly specifying UUID when creating a table in Replicated database. Also, do not allow explicitly specifying ZooKeeper path and replica name for *MergeTree tables in Replicated databases. #66104 (Alexander Tokmakov).
- 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 (Pavel Kruglov).
- Expressions like
a[b].c
are supported for named tuples, as well as named subscripts from arbitrary expressions, e.g.,expr().name
. This is useful for processing JSON. This closes #54965. In previous versions, an expression of formexpr().name
was parsed astupleElement(expr(), name)
, and the query analyzer was searching for a columnname
rather than for the corresponding tuple element; while in the new version, it is changed totupleElement(expr(), 'name')
. In most cases, the previous version was not working, but it is possible to imagine a very unusual scenario when this change could lead to incompatibility: if you stored names of tuple elements in a column or an alias, that was named differently than the tuple element's name:SELECT 'b' AS a, CAST([tuple(123)] AS 'Array(Tuple(b UInt8))') AS t, t[1].a
. It is very unlikely that you used such queries, but we still have to mark this change as potentially backward incompatible. #68435 (Alexey Milovidov). - When the setting
print_pretty_type_names
is enabled, it will printTuple
data type in a pretty form inSHOW CREATE TABLE
statements,formatQuery
function, and in the interactive mode inclickhouse-client
andclickhouse-local
. In previous versions, this setting was only applied toDESCRIBE
queries andtoTypeName
. This closes #65753. #68492 (Alexey Milovidov).
New Feature
- Function
toStartOfInterval()
now has a new overload which emulates TimescaleDB'stime_bucket()
function, respectively PostgreSQL'sdate_bin()
function. (#55619). It allows to align date or timestamp values to multiples of a given interval from an arbitrary origin (instead of 0000-01-01 00:00:00.000 as fixed origin). For example,SELECT toStartOfInterval(toDateTime('2023-01-01 14:45:00'), INTERVAL 1 MINUTE, toDateTime('2023-01-01 14:35:30'));
returns2023-01-01 14:44:30
which is a multiple of 1 minute intervals, starting from origin2023-01-01 14:35:30
. #56738 (Yarik Briukhovetskyi). - Add support for
ATTACH PARTITION ALL FROM
. #61987 (Kirill Nikiforov). - Adds a setting
input_format_try_infer_variants
which allows Variant type to be inferred during schema inference for text formats when there is more than one possible type for column/array elements. #63798 (Shaun Struwig). - Introduced JSONCompactWithProgress format where ClickHouse outputs each row as a newline-delimited JSON object, including metadata, data, progress, totals, and statistics. #66205 (Alexey Korepanov).
- Implement new JSON data type. #66444 (Pavel Kruglov).
- Add the
input_format_json_empty_as_default
setting which, when enabled, treats empty fields in JSON inputs as default values. Closes #59339. #66782 (Alexis Arnaud). - Added functions
overlay
andoverlayUTF8
which replace parts of a string by another string. Example:SELECT overlay('Hello New York', 'Jersey', 11)
returnsHello New Jersey
. #66933 (李扬). - Add new Command, Lightweight Delete In Partition
DELETE FROM [db.]table [ON CLUSTER cluster] [IN PARTITION partition_expr] WHERE expr;
``` VM-114-29-tos :) select * from ads_app_poster_ip_source_channel_di_replicated_local;. #67805 (sunny). - Implemented comparison for
Interval
data type values so they are converting now to the least supertype. #68057 (Yarik Briukhovetskyi). - Add create_if_not_exists setting to default to IF NOT EXISTS behavior during CREATE statements. #68164 (Peter Nguyen).
- Makes possible to read Iceberg tables in Azure and locally. #68210 (Daniil Ivanik).
- Add aggregate functions distinctDynamicTypes/distinctJSONPaths/distinctJSONPathsAndTypes for better introspection of JSON column type content. #68463 (Pavel Kruglov).
- Query cache entries can now be dropped by tag. For example, the query cache entry created by
SELECT 1 SETTINGS use_query_cache = true, query_cache_tag = 'abc'
can now be dropped bySYSTEM DROP QUERY CACHE TAG 'abc'
(or of course just:SYSTEM DROP QUERY CACHE
which will clear the entire query cache). #68477 (Michał Tabaszewski). - Add storage encryption for named collections. #68615 (Pablo Marcos).
- 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
. #68639 (Dergousov Maxim). - Add virtual column _headers for url table engine. Closes #65026. #68867 (flynn).
- Adding
system.projections
table to track available projections. #68901 (Jordi Villar). - Add new function
arrayZipUnaligned
for spark compatiablity(arrays_zip), which allowed unaligned arrays based on originalarrayZip
. ``` sql SELECT arrayZipUnaligned([1], [1, 2, 3]). #69030 (李扬). - Adding
RealTimeMicroseconds
metric in the HTTP HeaderX-ClickHouse-Summary
. This way we can know the CPU time of a request without having to go to check it insystem.query_log
. #69032 (Alejandro). - Added cp/mv commands for keeper client which atomically copies/moves node. #69034 (Mikhail Artemenko).
- Adds argument
scale
(default:true
) to functionarrayAUC
which allows to skip the normalization step (issue #69609). #69717 (gabrielmcg44).
Performance Improvement
- Improve the join performance by rerange the right table by keys while the table keys are dense in left or inner hash join. #60341 (kevinyhzou).
- Improve all join perfromance by append
RowRefList
orRowRef
to AddedColumns for lazy output, while buildOutput, we useRowRefList
/RowRef
for output, and removeis_join_get
condition frombuildOutput
for loop. #63677 (kevinyhzou). - Load filesystem cache metadata asynchronously to boot process, in order to make restarts faster. #65736 (Daniel Pozo Escalona).
- Functions
array
andmap
were optimized to process certain common cases much faster. #67707 (李扬). - Trivial optimize on orc string reading especially when column contains no NULLs. #67794 (李扬).
- Improved overall performance of merges by reducing the overhead of scheduling steps of merges. #68016 (Anton Popov).
- Speed up requests to S3 when a profile is not set, credentials are not set, and IMDS is not available (for example, when you are querying a public bucket on a machine outside of a cloud). This closes #52771. #68082 (Alexey Milovidov).
- New algorithm to determine the unit of marks distribution between replicas by consistent hash. Different numbers of marks chosen for different read patterns to improve performance. #68424 (Nikita Taranov).
- Try to devirtualize format reader in RowInputFormatWithNamesAndTypes, and see if it could bring some performance improvement. #68437 (李扬).
- Add the parallel merge with key implementation to maximize the CPU utilization. #68441 (Jiebin Sun).
- Add settings
output_format_orc_dictionary_key_size_threshold
to allow user to enable dict encoding for string column in ORC output format. It helps reduce the output orc file size and improve reading performance significantly. #68591 (李扬). - Implemented reading of required files only during hive partitioning. #68963 (Yarik Briukhovetskyi).
- Introduce new Keeper request RemoveRecursive which removes node with all it's subtree. #69332 (Mikhail Artemenko).
- Speedup insert performance with vector similarity index by adding data to vector index parallel. #69493 (flynn).
- Previously the algorithmic complexity of part deduplication logic in parallel replica announcement handling was O(n^2) which could take noticeable time for tables with many part (or partitions). This change makes the complexity O(n*log(n)). #69596 (Alexander Gololobov).
Improvement
- Hardened parts of the codebase related to parsing of small entities. The following (minor) bugs were found and fixed: - if a
DeltaLake
table is partitioned by Bool, the partition value is always interpreted as false; -ExternalDistributed
table was using only a single shard in the provided addresses; the value ofmax_threads
setting and similar were printed as'auto(N)'
instead ofauto(N)
. #52503 (Alexey Milovidov). - Refreshable materialized view improvements: append mode (
... REFRESH EVERY 1 MINUTE APPEND ...
) to add rows to existing table instead of overwriting the whole table, retries (disabled by default, configured in SETTINGS section of the query),SYSTEM WAIT VIEW <name>
query that waits for the currently running refresh, some fixes. #58934 (Michael Kolupaev). - Use cgroup-specific metrics for CPU usage accounting instead of system-wide metrics. #62003 (Nikita Taranov).
- IO scheduling for remote S3 disks is now done on the level of HTTP socket streams (instead of the whole S3 requests) to resolve
bandwidth_limit
throttling issues. #65182 (Sergei Trifonov). - Allow a user to have multiple authentication methods instead of only one. Allow authentication methods to be reset to most recently added method. If you want to run instances on 24.8 and one on 24.9 for some time, it's better to set
max_authentication_methods_per_user = 1
for that period to avoid potential errors. #65277 (Arthur Passos). - Functions
upperUTF8
andlowerUTF8
were previously only able to uppercase / lowercase Cyrillic characters. This limitation is now removed and characters in arbitrary languages are uppercased/lowercased. Example:SELECT upperUTF8('Süden')
now returnsSÜDEN
. #65761 (李扬). - When lightweight delete happens on a table with projection(s), despite users have choices either throw an exception (by default) or drop the projection lightweight delete would happen, now the third option is to still have lightweight delete and then rebuild projection(s). #66169 (Shichao).
- Two options (
dns_allow_resolve_names_to_ipv4
anddns_allow_resolve_names_to_ipv6
) have been added, to allow block connections ip family. #66895 (MikhailBurdukov). - Added
min_max
as a new type of (experimental) statistics. It supports estimating range predicates over numeric columns, e.g.x < 100
. #67013 (JackyWoo). - Make C-z ignorance configurable (ignore_shell_suspend) in clickhouse-client. #67134 (Azat Khuzhin).
- Improve castOrDefault from Variant/Dynamic columns so it works when inner types are not convertable at all. #67150 (Pavel Kruglov).
- Improve unicode encoding in JSON output formats. Ensures that valid JSON is generated in the case of certain byte sequences in the result data. #67938 (mwoenker).
- Added profile events for merges and mutations for better introspection. #68015 (Anton Popov).
- Odbc: get http_max_tries from server configuration. #68128 (Rodolphe Dugé de Bernonville).
- Add wildcard support for user identification in x509 SubjectAltName extension. #68236 (Marco Vilas Boas).
- 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 (Pavel Kruglov). - Delete old code of named collections from dictionaries and substitute it to the new, which allows to use DDL created named collections in dictionaries. Closes #60936, closes #36890. #68412 (Kseniia Sumarokova).
- Use HTTP/1.1 instead of HTTP/1.0 (set by default) for external HTTP authentication. #68456 (Aleksei Filatov).
- Functions
upperUTF8
andlowerUTF8
were previously only able to uppercase / lowercase Cyrillic characters. This limitation is now removed and arbitrary characters are uppercased/lowercased. Example:SELECT upperUTF8('Süden')
now returnsSÜDEN
. #68523 (Robert Schulze). - Added a new set of metrics for Thread Pool introspection, providing deeper insights into thread pool performance and behavior. #68674 (filimonov).
- Support query parameters in async inserts with format
Values
. #68741 (Anton Popov). - Support Date32 on dateTrunc and toStartOfInterval. #68874 (LiuNeng).
- Add a new setting:
output_format_always_quote_identifiers
to always quote identifiers if it is enabled. - Add a new setting:output_format_identifier_quoting_style
to set the identifier quoting style. Applicable values:'None'
,'Backticks'
(default),'DoubleQuotes'
,'BackticksMySQL'
. #68896 (tuanpach). - Add
plan_step_name
andplan_step_description
columns tosystem.processors_profile_log
. #68954 (Alexander Gololobov). - Support for the Spanish language in the embedded dictionaries. #69035 (Vasily Okunev).
- Add CPU arch to short fault info. #69037 (Konstantin Bogdanov).
- CREATE TABLE AS copy PRIMARY KEY, ORDER BY, and similar clauses. Now it supports only for MergeTree family of table engines. #69076 (sakulali).
- Replication of subset of columns is now available through MaterializedPostgreSQL. Closes #33748. #69092 (Kruglov Kirill).
- Add
std::string getHost(const std::string & key) const;
method to "base/poco/Util/include/Poco/Util/AbstractConfiguration.h" which does the same asPoco::Util::AbstractConfiguration::getString
method but additionally does validity check to make sure value is a correct IP address or domain name. #69130 (Maxim). - Do not block retries when establishing a new keeper connection. #69148 (Raúl Marín).
- Update DatabaseFactory so it would be possible for user defined database engines to have arguments, settings and table overrides (similar to StorageFactory). #69201 (Барыкин Никита Олегович).
- Restore mode that replaces all external table engines and functions to Null (
restore_replace_external_engines_to_null
,restore_replace_external_table_functions_to_null
settings) was failing if table had SETTINGS. Now it removes settings from table definition in this case and allows to restore such tables. #69253 (Ilya Yatsishin). - Reduce memory usage of inserts to JSON by using adaptive write buffer size. A lot of files created by JSON column in wide part contains small amount of data and it doesn't make sense to allocate 1MB buffer for them. #69272 (Pavel Kruglov).
- CLICKHOUSE_PASSWORD is escaped for XML in clickhouse image's entrypoint. #69301 (aohoyd).
- 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. #69376 (Marco Vilas Boas). - Not retaining thread in concurrent hash join threadpool to avoid query excessively spawn threads. #69406 (Duc Canh Le).
- Allow empty arguments for arrayZip/arrayZipUnaligned, as concat did in https://github.com/ClickHouse/ClickHouse/pull/65887. It is for spark compatiability in Gluten CH Backend. #69576 (李扬).
- Support more advanced SSL options for Keeper's internal communication (e.g. private keys with passphrase). #69582 (Antonio Andelic).
- Sleep for 10ms before retrying to acquire a lock in
databasereplicatedddlworker::enqueuequeryimpl
. #69588 (Miсhael Stetsyuk). - Index analysis can take noticeable time for big tables with many parts or partitions. This change should enable killing a heavy query at that stage. #69606 (Alexander Gololobov).
- Masking sensitive info in
gcs()
table function. #69611 (Vitaly Baranov). - Sync changes to
interpreterdropquery::executetodatabaseimpl
from the private fork. #69670 (Miсhael Stetsyuk). - Backported in #69905: Change the join to sort settings type to unsigned int. #69886 (kevinyhzou).
Bug Fix (user-visible misbehavior in an official stable release)
- Rebuild projection for merges that reduce number of rows. #62364 (cangyin).
- Fix attaching table when pg dbname contains "-" in MaterializedPostgreSQL. #62730 (takakawa).
- Storage Join support for nullable columns in left table, close #61247. #66926 (vdimir).
- Incorrect query result with parallel replicas (distribute queries as well) when
IN
operator contains conversion to Decimal(). The bug was introduced with the new analyzer. #67234 (Igor Nikonov). - Fix the problem that alter modfiy order by causes inconsistent metadata. #67436 (iceFireser).
- Fix the upper bound of the function
fromModifiedJulianDay
. It was supposed to be9999-12-31
but was mistakenly set to9999-01-01
. #67583 (PHO). - Fix when the index is not at the beginning of the tuple during
IN
query. #67626 (Yarik Briukhovetskyi). - Fixed error on generated columns in MaterializedPostgreSQL when adnum ordering is broken #63161. Fixed error on id column with nextval expression as default MaterializedPostgreSQL when there are generated columns in table. Fixed error on dropping publication with symbols except [a-z1-9-]. #67664 (Kruglov Kirill).
- Fix expiration in
RoleCache
. #67748 (Vitaly Baranov). - Fix window view missing blocks due to slow flush to view. #67983 (Raúl Marín).
- Fix MSAN issue caused by incorrect date format. #68105 (JackyWoo).
- Fixed crash in Parquet filtering when data types in the file substantially differ from requested types (e.g.
... FROM file('a.parquet', Parquet, 'x String')
, but the file hasx Int64
). Without this fix, useinput_format_parquet_filter_push_down = 0
as a workaround. #68131 (Michael Kolupaev). - Fix crash in
lag
/lead
which is introduced in #67091. #68262 (lgbo). - Try fix postgres crash when query is cancelled. #68288 (Kseniia Sumarokova).
- After https://github.com/ClickHouse/ClickHouse/pull/61984
schema_inference_make_columns_nullable=0
still can make columnsNullable
in Parquet/Arrow formats. The change was backward incompatible and users noticed the changes in the behaviour. This PR makesschema_inference_make_columns_nullable=0
to work as before (no Nullable columns will be inferred) and introduces new valueauto
for this setting that will make columnsNullable
only if data has information about nullability. #68298 (Pavel Kruglov). - Fixes #50868. Small DateTime64 constant values returned by a nested subquery inside a distributed query were wrongly transformed to Nulls, thus causing errors and possible incorrect query results. #68323 (Shankar).
- Fix missing sync replica mode in query
SYSTEM SYNC REPLICA
. #68326 (Duc Canh Le). - Fix bug in key condition. #68354 (Han Fei).
- Fix crash on drop or rename a role that is used in LDAP external user directory. #68355 (Andrey Zvonov).
- Fix Progress column value of system.view_refreshes greater than 1 #68377. #68378 (megao).
- Process regexp flags correctly. #68389 (Han Fei).
- PostgreSQL-style cast operator (
::
) works correctly even for SQL-style hex and binary string literals (e.g.,SELECT x'414243'::String
). This closes #68324. #68482 (Alexey Milovidov). - Minor patch for https://github.com/ClickHouse/ClickHouse/pull/68131. #68494 (Chang chen).
- Fix #68239 SAMPLE n where n is an integer. #68499 (Denis Hananein).
- Fix bug in mann-whitney-utest when the size of two districutions are not equal. #68556 (Han Fei).
- After unexpected restart, fail to start replication of ReplicatedMergeTree due to abnormal handling of covered-by-broken part. #68584 (baolin).
- Fix
LOGICAL_ERROR
s when functionssipHash64Keyed
,sipHash128Keyed
, orsipHash128ReferenceKeyed
are applied to empty arrays or tuples. #68630 (Robert Schulze). - Full text index may filter out wrong columns when index multiple columns, it didn't reset row_id between different columns, the reproduce procedure is in tests/queries/0_stateless/03228_full_text_with_multi_col.sql. Without this. #68644 (siyuan).
- Fix invalid character '\t' and '\n' in replica_name when creating a Replicated table, which causes incorrect parsing of 'source replica' in LogEntry. Mentioned in issue #68640. #68645 (Zhigao Hong).
- Added back virtual columns
_table
and_database
to distributed tables. They were available until version 24.3. #68672 (Anton Popov). - Fix possible error
Size of permutation (0) is less than required (...)
during Variant column permutation. #68681 (Pavel Kruglov). - Fix possible error
DB::Exception: Block structure mismatch in joined block stream: different columns:
with new JSON column. #68686 (Pavel Kruglov). - Fix issue with materialized constant keys when hashing maps with arrays as keys in functions
sipHash(64/128)Keyed
. #68731 (Salvatore Mesoraca). - Make
ColumnsDescription::toString
format each column using the sameIAST::FormatState object
. This results in uniform columns metadata being written to disk and ZooKeeper. #68733 (Miсhael Stetsyuk). - Fix merging of aggregated data for grouping sets. #68744 (Nikolai Kochetov).
- Fix logical error, when we create a replicated merge tree, alter a column and then execute modify statistics. #68820 (Han Fei).
- Fix resolving dynamic subcolumns from subqueries in analyzer. #68824 (Pavel Kruglov).
- Fix complex types metadata parsing in DeltaLake. Closes #68739. #68836 (Kseniia Sumarokova).
- Fixed asynchronous inserts in case when metadata of table is changed (by
ALTER ADD/MODIFY COLUMN
queries) after insert but before flush to the table. #68837 (Anton Popov). - Fix unexpected exception when passing empty tuple in array. This fixes #68618. #68848 (Amos Bird).
- Fix parsing pure metadata mutations commands. #68935 (János Benjamin Antal).
- Fix possible wrong result during anyHeavy state merge. #68950 (Raúl Marín).
- Fixed writing to Materialized Views with enabled setting
optimize_functions_to_subcolumns
. #68951 (Anton Popov). - Don't use serializations cache in const Dynamic column methods. It could let to use-of-unitialized value or even race condition during aggregations. #68953 (Pavel Kruglov).
- Fix parsing error when null should be inserted as default in some cases during JSON type parsing. #68955 (Pavel Kruglov).
- Fix
Content-Encoding
not sent in some compressed responses. #64802. #68975 (Konstantin Bogdanov). - There were cases when path was concatenated incorrectly and had the
//
part in it, solving this problem using path normalization. #69066 (Yarik Briukhovetskyi). - Fix logical error when we have empty async insert. #69080 (Han Fei).
- Fixed data race of progress indication in clickhouse-client during query canceling. #69081 (Sergei Trifonov).
- Fix a bug that the vector similarity index (currently experimental) was not utilized when used with cosine distance as distance function. #69090 (flynn).
- This change addresses an issue where attempting to create a Replicated database again after a server failure during the initial creation process could result in error. #69102 (Miсhael Stetsyuk).
- Don't infer Bool type from String in CSV when
input_format_csv_try_infer_numbers_from_strings = 1
because we don't allow reading bool values from strings. #69109 (Pavel Kruglov). - Fix explain ast insert queries parsing errors on client when
--multiquery
is enabled. #69123 (wxybear). UNION
clause in subqueries wasn't handled correctly in queries with parallel replicas and lead to LOGICAL_ERRORDuplicate announcement received for replica
. #69146 (Igor Nikonov).- Fix propogating structure argument in s3Cluster. Previously the
DEFAULT
expression of the column could be lost when sending the query to the replicas in s3Cluster. #69147 (Pavel Kruglov). - Respect format settings in Values format during conversion from expression to the destination type. #69149 (Pavel Kruglov).
- Fix
clickhouse-client --queries-file
for readonly users (previously fails withCannot modify 'log_comment' setting in readonly mode
). #69175 (Azat Khuzhin). - Fix data race in clickhouse-client when it's piped to a process that terminated early. #69186 (vdimir).
- Fix incorrect results of Fix uniq and GROUP BY for JSON/Dynamic types. #69203 (Pavel Kruglov).
- Fix the INFILE format detection for asynchronous inserts. If the format is not explicitly defined in the FORMAT clause, it can be detected from the INFILE file extension. #69237 (Julia Kartseva).
- After this issue there are quite a few table replicas in production such that their
metadata_version
node value is both equal to0
and is different from the respective table'smetadata
node version. This leads toalter
queries failing on such replicas. #69274 (Miсhael Stetsyuk). - Backported in #69986: Fix inf loop after
restore replica
in the replicated merge tree with zero copy. #69293 (MikhailBurdukov). - Mark Dynamic type as not safe primary key type to avoid issues with Fields. #69311 (Pavel Kruglov).
- Improve restoring of access entities' dependencies. #69346 (Vitaly Baranov).
- Fix undefined behavior when all connection attempts fail getting a connection for insertions. #69390 (Pablo Marcos).
- Close #69135. If we try to reuse joined data for
cross
join, but this could not happen in ClickHouse at present. It's better to keephave_compressed
inreuseJoinedData
. #69404 (lgbo). - Make
materialize()
function return full column when parameter is a sparse column. #69429 (Alexander Gololobov). - Fixed a
LOGICAL_ERROR
with functionsqidDecode
(#69450). #69451 (Robert Schulze). - Quick fix for s3queue problem on 24.6 or create query with database replicated. #69454 (Kseniia Sumarokova).
- Fixed case when memory consumption was too high because of the squashing in
INSERT INTO ... SELECT
orCREATE TABLE AS SELECT
queries. #69469 (Yarik Briukhovetskyi). - Statements
SHOW COLUMNS
andSHOW INDEX
now work properly if the table has dots in its name. #69514 (Salvatore Mesoraca). - Usage of the query cache for queries with an overflow mode != 'throw' is now disallowed. This prevents situations where potentially truncated and incorrect query results could be stored in the query cache. (issue #67476). #69549 (Robert Schulze).
- Keep original order of conditions during move to prewhere. Previously the order could change and it could lead to failing queries when the order is important. #69560 (Pavel Kruglov).
- Fix Keeper multi-request preprocessing after ZNOAUTH error. #69627 (Antonio Andelic).
- Fix METADATA_MISMATCH that might have happened due to TTL with a WHERE clause in DatabaseReplicated when creating a new replica. #69736 (Nikolay Degterinsky).
- Fix
StorageS3(Azure)Queue
settingstracked_file_ttl_sec
. We wrote it to keeper with keytracked_file_ttl_sec
, but read astracked_files_ttl_sec
, which was a typo. #69742 (Kseniia Sumarokova). - Make getHyperrectangleForRowGroup not throw an exception when the data type in parquet file is not convertable into the requested data type. Solved the user's problem when the Parquet file had Decimal64 data type and the column data type was DateTime. #69745 (Miсhael Stetsyuk).
- Backported in #70021: Fix analyzer default with old compatibility value. #69895 (Raúl Marín).
- Backported in #69943: 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).
- Backported in #70003: Now SQL security will work with parameterized views correctly. #69984 (pufit).
Build/Testing/Packaging Improvement
- Allow to specify min and max for random settings in the test after tags in a form
setting=(min, max)
. For some tests really low value of a setting can lead to timeout (for exampleindex_granularity=1
) but we still want to randomize this setting in such tests. To avoid timeouts we can specify custom minimum value for this setting. #67875 (Pavel Kruglov). - Integration tests flaky check now runs each module(file) as a separate group. It allows to apply timeout per module. Previously slow tests could use all the time and other tests were not run in flaky check with successful result. #68380 (Ilya Yatsishin).
- Backported in #69980: Makes dbms independent from clickhouse_functions. #69914 (Raúl Marín).
NO CL CATEGORY
- Backported in #69995:. #69989 (Alexey Milovidov).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Revert "Fix AWS ECS""'. #65362 (Pavel Kruglov).
- NO CL ENTRY: 'Revert "[RFC] Fix settings/current_database in system.processes for async BACKUP/RESTORE"'. #68386 (János Benjamin Antal).
- NO CL ENTRY: 'Revert "Improve compatibility of
upper/lowerUTF8
with Spark"'. #68510 (Robert Schulze). - NO CL ENTRY: 'Revert "Fix unexpected behavior with
FORMAT
andSETTINGS
parsing"'. #68608 (Alexander Tokmakov). - NO CL ENTRY: 'Revert "Fix prewhere without columns and without adaptive index granularity (almost w/o anything)"'. #68897 (Alexander Gololobov).
- NO CL ENTRY: 'Revert "Speed up some Kafka tests with multiprocessing"'. #69356 (Nikita Mikhaylov).
- NO CL ENTRY: 'Revert "Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. V"'. #69393 (Alexander Tokmakov). - NO CL ENTRY: 'Revert "Add user-level settings min_free_diskspace_bytes_to_throw_insert and min_free_diskspace_ratio_to_throw_insert"'. #69705 (Raúl Marín).
- NO CL ENTRY: 'Revert "Support more oss endpoints"'. #69779 (Raúl Marín).
NOT FOR CHANGELOG / INSIGNIFICANT
- Followup for #56996. #58289 (vdimir).
- Add chunked wrapper to native protocol. #63781 (Yakov Olkhovskiy).
- Local plan for parallel replicas. #64448 (Igor Nikonov).
- Disallow creating refreshable MV on Linux < 3.15. #64953 (Michael Kolupaev).
- Allow query profiler period to be longer than 4 seconds. #65255 (Michael Kolupaev).
- Improved getting of alter conversions for queries. #65832 (Anton Popov).
- Simplify some memory tracking parts. #66648 (Antonio Andelic).
- What if we tighten limits for functional tests?. #66837 (Alexey Milovidov).
- Fix flaky
test_delayed_replica_failover
. #67541 (Sergei Trifonov). - Fix(asan): access destroyed shared context from handleCrash(). #67700 (Igor Nikonov).
- Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. II. #67749 (Robert Schulze). - Make
ColumnLowCardinality::getName()
consistent with other columns. #67789 (李扬). - Catch exception in destructor of LocalFileHolder. #67891 (Duc Canh Le).
- Add keeper error description to the message. #67935 (Alexander Gololobov).
- Timeout handling for functional and integration tests, store artifacts and report if timed out - sets 2h default timeout for all jobs. #67944 (Max K.).
- Add test cases to 03217_datetime64_constant_to_ast. #67966 (vdimir).
- Remove some no-parallel tags from tests (Part 6). #68048 (Raúl Marín).
- Keeper improvements package. #68108 (Antonio Andelic).
- CI: Docker images clean from test scripts. #68117 (Max K.).
- Output an operation error for ZK Multi request failed operation into log. #68127 (Aleksei Filatov).
- Better test for Not-ready Set is passed in system.* tables. #68139 (Azat Khuzhin).
- ci: add more logs in the functional tests reports. #68153 (Azat Khuzhin).
- Fix fundamentally broken
test_throttling
. #68180 (Alexey Milovidov). - Fix test
00600_replace_running_query
. #68184 (Alexey Milovidov). - Fix non-deterministic result order in
test_storage_mysql.test_mysql_distributed
. #68188 (Robert Schulze). - Try to unflaky
test_storage_s3_queue/test.py::test_shards_distributed
. #68233 (János Benjamin Antal). - Bump usearch to v2.13.2. #68248 (Robert Schulze).
- Fix 'Refresh set entry already exists'. #68249 (Michael Kolupaev).
- tests: make 01600_parts_states_metrics_long better. #68265 (Azat Khuzhin).
- Avoid ignoring errors of execute_process() (set COMMAND_ERROR_IS_FATAL=ANY). #68267 (Azat Khuzhin).
- Fix FullSortingJoinTest.AsofGreaterGeneratedTestData with empty data. #68280 (vdimir).
- Fix min/max time columns in TimeSeries table. #68282 (Antonio Andelic).
- Update fuzzer dictionary. #68284 (Pablo Marcos).
- [Green CI] Test 00652_mergetree_mutations is flaky. #68286 (Daniil Ivanik).
- Fix test storage_join_direct_join. #68287 (Nikita Taranov).
- Add execution status to PipelineExecutor. Avoid LOGICAL_ERROR if the pushing pipeline was canceled by timeout. Replace
Pipeline for PushingPipelineExecutor was finished before all data was inserted. (LOGICAL_ERROR)
toQUERY_WAS_CANCELLED
. Closes #63979. #68291 (Nikolai Kochetov). - CI: Create new release branch workflow updates. #68294 (Max K.).
- Update version after release. #68306 (robot-clickhouse).
- Various vector similarity index related fixes. #68308 (Robert Schulze).
- Fix: min marks to read overflow with parallel replicas. #68309 (Igor Nikonov).
- Apply libunwind changes needed for musl. #68312 (Michael Kolupaev).
- Update musl to 1.2.5 with unwind info. #68313 (Michael Kolupaev).
- Fix data race on SampleKey. #68321 (Antonio Andelic).
- Fix part name in 00961_check_table. #68325 (vdimir).
- Fix 02995_index_10 timeout. #68329 (vdimir).
- CI: Fix for change log critical bug fix regex. #68332 (Max K.).
- Optionally re-enable compilation with
-O0
. #68352 (Robert Schulze). - Add debug info for
00180_no_seek_avoiding_when_reading_from_cache
. #68353 (Kseniia Sumarokova). - Fix data race in
DynamicResourceManager::updateConfiguration
. #68356 (Sergei Trifonov). - Update version_date.tsv and changelog after v24.3.7.30-lts. #68357 (robot-clickhouse).
- More improvements in integration tests. #68358 (Ilya Yatsishin).
- Rename: S3DiskNoKeyErrors -> DiskS3NoSuchKeyErrors. #68361 (Miсhael Stetsyuk).
- CI: Minor fixes for changelog and release exceptions. #68362 (Max K.).
- Check that setProcessListElement() is not called on global context. #68364 (Michael Kolupaev).
- Fix off-by-one inline function info in stack traces. #68365 (Michael Kolupaev).
- Check that merge entries are valid. #68366 (Alexander Tokmakov).
- performance comparison test for output_format_parquet_write_page_index. #68367 (max-vostrikov).
- Add a test for #57324. #68376 (Alexey Milovidov).
- CI: Auto release workflow. #68402 (Max K.).
- Update delta lake test. #68404 (Kseniia Sumarokova).
- Add a test for #59118. #68406 (Kseniia Sumarokova).
- Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. IV. #68407 (Robert Schulze). - Try to fix test 03221_mutation_analyzer_skip_part. #68409 (Anton Popov).
- Add
NotStarted
status to not-prepared processors. This is for better diagnostics in the case of PipelineStuck. #68413 (Nikolai Kochetov). - Update log message. #68426 (Kseniia Sumarokova).
- Fix 01119_session_log flakiness. #68433 (Michael Kolupaev).
- Try to mitigate 02818_memory_profiler_sample_min_max_allocation_size flakiness. #68434 (Michael Kolupaev).
- Update analyzer_tech_debt.txt. #68443 (Robert Schulze).
- Remove obsolete
-n
/--multiquery
from tests. #68447 (Robert Schulze). - Check for invalid regexp in JSON SKIP REGEXP section. #68451 (Pavel Kruglov).
- Better inference of date times 2. #68452 (Pavel Kruglov).
- CI: Native build for package_aarch64. #68457 (Max K.).
- Minor update in Dynamic/JSON serializations. #68459 (Pavel Kruglov).
- Fix test
02122_join_group_by_timeout
. #68462 (Alexey Milovidov). - Add check for
min_number_of_marks = 0
in parallel replicas requests. #68476 (Nikita Taranov). - Fix
Broken pipe
error for03149_numbers_max_block_size_zero.sh
. #68478 (Julia Kartseva). - Use temporary tables for input and output in
clickhouse-local
. #68483 (Alexey Milovidov). - Pass-through RENAME and UUID-related operations in Overlay database to underlying databases. #68486 (Alexey Milovidov).
- Fix output of clickhouse-test in case of tests timeouts. #68487 (Azat Khuzhin).
- Part of #68024. #68488 (Alexey Milovidov).
- Miscellaneous changes in BaseDaemon. #68489 (Alexey Milovidov).
- Miscellaneous changes from #66999 (2). #68490 (Alexey Milovidov).
- Miscellaneous. #68504 (Alexey Milovidov).
- Miscellanous changes from #66999. #68507 (Alexey Milovidov).
- Fix test
01017_uniqCombined_memory_usage
. #68508 (Alexey Milovidov). - Fix race condition in MergeTreeRestartingThread. #68513 (Alexey Milovidov).
- Fix test
01079_bad_alters_zookeeper_long
. #68515 (Alexey Milovidov). - Fix bad exception messages. #68520 (Alexey Milovidov).
- CI: Auto Releases in prod. #68525 (Max K.).
- Fix build with
-DENABLE_LIBRARIES=0
. #68527 (Robert Schulze). - Removed the use of the context during function execution, as it was causing errors when the context was expired. #68534 (Yarik Briukhovetskyi).
- CI: Minor release workflow fix. #68536 (Max K.).
- Fix after #68291. #68548 (Nikolai Kochetov).
- Stateless tests: increase hung check timeout. #68552 (Nikita Fomichev).
- CI: Tidy build timeout from 2h to 3h. #68567 (Max K.).
- Reduce memory consumption in ghdata JSON tests. #68571 (Pavel Kruglov).
- Move enabling experimental settings to a separate file. #68577 (Nikolay Degterinsky).
- Minor logging fixes. #68578 (Robert Schulze).
- Fix enumerating dynamic subcolumns. #68582 (Pavel Kruglov).
- Update version_date.tsv and changelog after v23.8.16.16-lts. #68593 (robot-clickhouse).
- Update 02995_index_7.sh. #68594 (Kseniia Sumarokova).
- Fix style in Functions/printf.cpp. #68595 (vdimir).
- Update version_date.tsv and changelog after v24.3.8.13-lts. #68599 (robot-clickhouse).
- Prioritizing of virtual columns in hive partitioning. #68606 (Yarik Briukhovetskyi).
- materialized_view_deduplication performance comparison test. #68607 (max-vostrikov).
- Update README.md. #68610 (Tyler Hannan).
- patch: fix reference to sorting key in primary key docs. #68612 (Leon Kozlowski).
- Do not fuzz 02835 drop user during session. #68620 (Nikolai Kochetov).
- Fix 03221_s3_imds_decent_timeout. #68627 (vdimir).
- Fix test
01079_bad_alters_zookeeper_long
. #68629 (Alexander Tokmakov). - Disable min_bytes_to_use_direct_io in some tests with Dynamic/JSON subcolumns because it's broken. #68632 (Pavel Kruglov).
- Fix false "Killed by signal (output files)" in stress_tests.lib. #68638 (Michael Kolupaev).
- Remove wrong release version. #68646 (Max K.).
- Increase connectTimeoutMs IMDS connection timeout to 50ms to avoid failures in CI. #68653 (Pavel Kruglov).
- CI: Disable SQLLogic job. #68654 (Max K.).
- Update version_date.tsv and changelog after v24.8.1.2684-lts. #68664 (robot-clickhouse).
- Fix flaky test test_distributed_replica_max_ignored_errors. #68665 (Pavel Kruglov).
- Improve
02293_http_header_full_summary_without_progress
logging. #68666 (Konstantin Bogdanov). - Fix flaky check when all tests are skipped. #68673 (Pavel Kruglov).
- Vector similarity index: make
bf16
the default quantization. #68678 (Robert Schulze). - 3h is not enough. #68683 (Max K.).
- Un-flake 01278_random_string_utf8. #68684 (Robert Schulze).
- CI: Integration tests timeout to 3h. #68685 (Max K.).
- Fix Upgrade Check: move some settings to 24.9 section. #68688 (Pavel Kruglov).
- Update README.md. #68690 (Tanya Bragin).
- Use proper ErrorCodes, replace NETWORK_ERROR by HDFS_ERROR. #68696 (flynn).
- Fix for failing test in CI. #68701 (Pedro Ferreira).
- CI: Stress test fix. #68712 (Max K.).
- Speedup test 02150_index_hypothesis_race_long. #68713 (vdimir).
- Turn off fault injection for insert in
01396_inactive_replica_cleanup_nodes_zookeeper
. #68715 (alesapin). - Update README.md - Meetups update. #68723 (Tanya Bragin).
- Fix flaky check. #68725 (Pavel Kruglov).
- fix shutdown for PeriodicLog. #68728 (Sema Checherinda).
- Update version_date.tsv and changelog after v24.5.5.41-stable. #68729 (robot-clickhouse).
- Bump Replxx to support custom descriptors. #68730 (Nikita Mikhaylov).
- Update version_date.tsv and changelog after v24.6.3.38-stable. #68732 (robot-clickhouse).
- Fixes
00080_show_tables_and_system_tables
that was a bit flaky. #68734 (Yarik Briukhovetskyi). - Update version_date.tsv and changelog after v24.7.3.47-stable. #68735 (robot-clickhouse).
- Fix flaky test 00989_parallel_parts_loading. #68737 (alesapin).
- Update README.md. #68738 (Tyler Hannan).
- Update version_date.tsv and changelog after v24.8.2.3-lts. #68740 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.5.5.41-stable. #68745 (robot-clickhouse).
- To make patch release possible from every commit on release branch, package_debug build is required and must not be skipped. #68750 (Max K.).
- Try to disable rerun check if job triggered manually. #68751 (Max K.).
- Fix 2477 timeout. #68752 (Shichao).
- Update README.md. #68764 (Tanya Bragin).
- Update version_date.tsv and changelog after v24.5.6.45-stable. #68766 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.6.4.42-stable. #68767 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.7.4.51-stable. #68768 (robot-clickhouse).
- Split test case and reduce number of random runs to reduce the time necessary to run the test. #68772 (János Benjamin Antal).
- Check setting use_json_alias_for_old_object_type in runtime. #68793 (Pavel Kruglov).
- Make dynamic structure selection more consistent. #68802 (Pavel Kruglov).
- Fix zero copy bug with encrypted disk and UNFREEZE. #68821 (Joe Lynch).
- Speed up functions
lowerUTF8
/upperUTF8
. #68822 (李扬). - Convert integration test
test_incorrect_datetime_format
to stateless. #68823 (Yarik Briukhovetskyi). - Fix Function Typo. #68835 (Shichao).
- Fix test
03228_virtual_column_merge_dist
. #68841 (Anton Popov). - Fix test
03221_mutation_analyzer_skip_part
. #68842 (Anton Popov). - Fix flaky
02932_analyzer_rewrite_sum_column_and_constant
. #68843 (Nikita Mikhaylov). - Update README.md - Add Austin meetup. #68845 (Tanya Bragin).
- Fix ssl handshake error processing. #68866 (Yakov Olkhovskiy).
- Speedup test 00653_verification_monotonic_data_load. #68878 (vdimir).
- ci: add IPv6 support to
NetworkManager
. #68900 (Konstantin Bogdanov). - ci: add checks for
iptables-nft
in integration test runner. #68902 (Konstantin Bogdanov). - Fix
02378_part_log_profile_events
flakiness. #68917 (Julia Kartseva). - Make long_parquet* tests less long. #68918 (Michael Kolupaev).
- Fix 01114_database_atomic flakiness. #68930 (Raúl Marín).
- CI: Fix job rerun check. #68931 (Max K.).
- perf tests set cgroups_memory_usage_observer_wait_time to zero. #68957 (vdimir).
- Revert un-quoting
auto
settings values. Fixes #68748. #68979 (Konstantin Bogdanov). - Actually fix false "Killed by signal (output files)" in stress_tests.lib. #68983 (Michael Kolupaev).
- no-tsan in 02735_parquet_encoder. #68984 (Michael Kolupaev).
- Update CHANGELOG.md. #68994 (Alexey Milovidov).
- Fix test_role & test_keeper_s3_snapshot integration tests. #69013 (Shankar).
- fix false leak detect in libfiu. #69018 (Han Fei).
- Update README.md. #69041 (Tyler Hannan).
- Update parametric-functions.md. #69055 (Samuel Warfield).
- Disallow
ALTER TABLE ADD VECTOR SIMILARITY INDEX
if corresponding setting is not enabled. #69065 (flynn). - Remove stale moving parts without zookeeper. #69075 (Kirill).
- Update README.md. #69077 (Tyler Hannan).
- Fix typo in ActionsDag. #69086 (LiuNeng).
- Follow-up for https://github.com/ClickHouse/ClickHouse/pull/68332. #69099 (Nikita Mikhaylov).
- Fix subnet in docker_compose_net.yml. #69121 (Ilya Golshtein).
- Fix: expression description in plan after lift up union optimization. #69124 (Igor Nikonov).
- Fix locking in UserDefinedSQLObjectsZooKeeperStorage.cpp. #69132 (Sergei Trifonov).
- Do not use docker pause for Kerberos KDC container in integration tests. #69136 (Ilya Golshtein).
- Don't create Object type if use_json_alias_for_old_object_type=1 but allow_experimental_object_type=0. #69150 (Pavel Kruglov).
- Use QueryPlan for merge. #69167 (Alexander Gololobov).
- Update version_date.tsv and changelog after v24.3.10.33-lts. #69177 (robot-clickhouse).
- CI: Rerun check: do not check if manual rerun. #69181 (Max K.).
- Update version_date.tsv and changelog after v24.5.7.31-stable. #69182 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.6.5.30-stable. #69184 (robot-clickhouse).
- Fix flaky 02915_move_partition_inactive_replica. #69190 (alesapin).
- Update version_date.tsv and changelog after v24.8.3.59-lts. #69191 (robot-clickhouse).
- Disable memory test with sanitizer. #69193 (alesapin).
- Disable perf-like test with sanitizers. #69194 (alesapin).
- Fix jepsen for aarch64. #69199 (Antonio Andelic).
- Test for parallel replicas, reverse in order reading mode. #69202 (Igor Nikonov).
- Collect sanitizer report from client to
client_log
. #69204 (vdimir). - Fix flaky 00157_cache_dictionary. #69205 (Igor Nikonov).
- Hide Settings implementation. #69213 (Raúl Marín).
- Fix progress bar when reading from Memory tables. #69234 (Michael Kolupaev).
- CMake: Update ICU build description. #69240 (Robert Schulze).
- Add testcase for ANN index usage with subquery. #69241 (Robert Schulze).
- Backports should read tags from its repo only. #69252 (Raúl Marín).
- 01114_database_atomic: Increase time frames to reduce flakiness. #69255 (Raúl Marín).
- Fix dropping of file cache in CHECK query in case of enabled transactions. #69256 (Anton Popov).
- CI: Merge stress and func runners type. #69257 (Max K.).
- Make infrastructure related scripts private. #69260 (Mikhail f. Shiryaev).
- Fix: Not-ready Set with parallel replicas. #69264 (Igor Nikonov).
- Revert "CREATE TABLE AS copy PRIMARY KEY, ORDER BY, and similar clauses.". #69268 (János Benjamin Antal).
- Use CANNOT_DECOMPRESS error code for the compressed external data in CompressedReadBufferBase for checksum validation and decompression. #69269 (Alexey Katsman).
- Fix the error in loading client suggestions and select from
system.function
whenallow_experimental_nlp_functions
setting is enabled and no lemmatizers are specified in the configuration. #69271 (Nikolay Degterinsky). - Improve logical error trace for TryResult. #69297 (Pablo Marcos).
- Just a small refactoring. #69298 (Nikolai Kochetov).
- Don't run 01287_max_execution_speed with slow builds. #69299 (Alexander Tokmakov).
- Add checks against segfault in DeltaLakeMetadata. #69305 (Kseniia Sumarokova).
- Update StackTrace.cpp. #69306 (Alexander Tokmakov).
- Bump libarchive from v3.7.0 to v3.7.1. #69318 (Robert Schulze).
- Bump curl to v8.9.1. #69321 (Robert Schulze).
- These tests take long time to execute with slow builds (either 700 or 400 seconds). #69322 (Nikita Mikhaylov).
- Bump grpc to v1.59.5. #69323 (Robert Schulze).
- Speed up some Kafka tests with multiprocessing. #69324 (Antonio Andelic).
- Minor follow-up for #66933. #69326 (Robert Schulze).
- Randomize integration tests settings. #69328 (vdimir).
- Update version_date.tsv and changelog after v24.3.11.7-lts. #69330 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.5.8.10-stable. #69333 (robot-clickhouse).
- Add CITATION.cff. #69334 (Robert Schulze).
- Update version_date.tsv and changelog after v24.6.6.6-stable. #69335 (robot-clickhouse).
- Fix a leak of a few bytes in function
lower/upperUTF8
. #69336 (Robert Schulze). - Update version_date.tsv and changelog after v24.7.6.8-stable. #69337 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.8.4.13-lts. #69340 (robot-clickhouse).
- Bump OpenSSL to v3.2.3. #69341 (Robert Schulze).
- Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. V. #69344 (Robert Schulze). - Bump libuv to v1.48.0. #69345 (Robert Schulze).
- Update README.md - Meetups. #69359 (Tanya Bragin).
- Remove obsolete
--multiquery
parameter (follow-up to #63898), pt. VI. #69361 (Robert Schulze). - Bump libarchive to v3.7.4. #69367 (Robert Schulze).
- Rename
count_min
statistics tocountmin
. #69377 (JackyWoo). - Refactor temporary file usage in MergeTask with QueryPlan. #69383 (Alexander Gololobov).
- Fix type mismatch. #69385 (Nikita Taranov).
- Fix 24.8 setting compatibility
rows_before_aggregation
. #69394 (Nikita Fomichev). - Support more oss endpoints. #69400 (Nikita Taranov).
- Fix 01603_read_with_backoff_bug. #69413 (Nikita Taranov).
- See if it would address strange issue with
Resource temporarily unavailable
. #69416 (Mikhail f. Shiryaev). - Added a server setting
database_replicated_allow_detach_permanently
that disallowsDETACH TABLE PERMANENTLY
queries in Replicated databases (allowed by default). #69422 (Alexander Tokmakov). - See https://s3.amazonaws.com/clickhouse-test-reports/69093/d9b1fb7a7cb0813dd12d8f73423662e02a458056/fast_test.html. #69430 (Alexander Tokmakov).
- Fix test_disks_app_func. #69449 (Nikita Taranov).
- Fixed consistency for
groupConcat
witharrayStringConcat
in case ofFixedString
as argument. #69455 (Yarik Briukhovetskyi). - The fix from #69416 was incomplete. Reduce the size of printed statement to 4k max. #69483 (Mikhail f. Shiryaev).
- CMake: Add comment about ICU data files. #69511 (Robert Schulze).
- S3Queue: small refactoring. #69513 (Kseniia Sumarokova).
- Backport MergeTask changes from private to minimize merge conflicts. #69525 (Alexander Gololobov).
- Fix getting the latest synced commit for long PRs. #69538 (Mikhail f. Shiryaev).
- When trying to ATTACH ReplicatedMergeTree tables that were previously PERMANENTLY DETACHED, we will fail due to explicitly specified parameters. Reference https://github.com/ClickHouse/ClickHouse/pull/66104. #69539 (Nikolay Degterinsky).
- Fix ci upgrade check failure for
join_to_sort
settings in pr https://github.com/ClickHouse/ClickHouse/pull/60341. #69554 (kevinyhzou). - Add function
kill_ci_runner
. Kill runner when pre-pull failed. #69557 (Mikhail f. Shiryaev). - Replace libpq code dump by postgresql fork + bump to v14.3. #69564 (Robert Schulze).
- Mask azure connection string sensitive info. #69570 (Alexey Katsman).
- Don't leave an empty znode when replicated table is dropped. #69574 (Michael Kolupaev).
- Enable removerecursive in ci. #69578 (Mikhail Artemenko).
- Bump libpqxx to v7.7.5. #69580 (Robert Schulze).
- Remove superfluous
--multiquery/-n
, pt. V. #69581 (Robert Schulze). - A better fix for #67476. #69583 (Robert Schulze).
- Add more contexts to the debug action and use it broadly. #69599 (Mikhail f. Shiryaev).
- Try to fix data race in
WriteBufferFromHTTPServerResponse
. #69601 (Konstantin Bogdanov). - Remove explicit announce from local replica in ReadFromMergeTree. #69602 (Nikita Taranov).
- Unification of FunctionSecretArgumentsFinder. #69615 (Yakov Olkhovskiy).
- Reorder some tests (follow-up to #69514). #69626 (Yarik Briukhovetskyi).
- Fix invalid
clickhouse-format
invocation in tests. #69642 (Konstantin Bogdanov). - Try fix
02447_drop_database_replica
. #69655 (Konstantin Bogdanov). - S3Queue small refactoring. #69672 (Kseniia Sumarokova).
- Update assert. #69673 (Kseniia Sumarokova).
- Bump libpq from v14.3 to v15.8. #69674 (Robert Schulze).
- Try fix asserts failure in
HashJoin
. #69682 (Konstantin Bogdanov). - Prohibit
ALTER TABLE ... ADD INDEX ... TYPE
inverted if setting = 0. #69684 (Robert Schulze). - Fixed 2 bugs in Remove Recursive implementation:. #69690 (Mikhail Artemenko).
- New profile events for parallel replicas. #69706 (Nikita Taranov).
- Update README.md - Meetups. #69714 (Tanya Bragin).
- added some edge cases for printf tests. #69737 (max-vostrikov).
- Bump libpq to v16.4. #69741 (Robert Schulze).
- Fix parallel replicas protocol after #68424. #69744 (Nikita Taranov).
- CI: Include PostgreSQL in sparse checkout script. #69746 (Robert Schulze).
- Get rid of the import cycle when env_helper can't be imported from
ci_*.py
files. Now,env_helper
won't import download helper or basically anything from theci
module. The report is the best place for the GH job info since we use it mostly there. #69750 (Mikhail f. Shiryaev). - Save CREATE QUERY with the KeeperMap engine with evaluated parameters. It is important because, for example, we can use the
currentDatabase()
function in the KeeperMapzk_root_path
argument, and we want it to point to the same path even if we move this table to another database. #69751 (Nikolay Degterinsky). - Fix native macOS build. #69767 (Robert Schulze).
- Backported in #69974: S3Queue: support having deprecated settings to not fail server startup. #69769 (Kseniia Sumarokova).
- Limit fast tests to 30 seconds. #69781 (Raúl Marín).
- Backported in #69824: Allow cyrillic characters in generated contributor names. #69820 (Raúl Marín).
- Backported in #69912: Add Proj Obsolete Setting. #69883 (Shichao).
- Backported in #69899: Revert "Merge pull request #69032 from alexon1234/include_real_time_execution_in_http_header". #69885 (Alexey Milovidov).
- Backported in #69931: 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).
- Backported in #70034: Revert "Add RIPEMD160 function". #70005 (Robert Schulze).