v24.12 Changelog for Cloud
Relevant changes for ClickHouse Cloud services based on the v24.12 release.
Backward Incompatible Changes
- Functions greatestandleastnow ignore NULL input values, whereas they previously returned NULL if one of the arguments was NULL. For example,SELECT greatest(1, 2, NULL)now returns 2. This makes the behavior compatible with PostgreSQL. #65519 (kevinyhzou).
- Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default because it may lead to unexpected results. #69731 (Pavel Kruglov).
- Remove system tables generate_seriesandgenerateSeries. They were added by mistake here: #59390. #71091 (Alexey Milovidov).
- Remove StorageExternalDistributed. Closes #70600. #71176 (flynn).
- Settings from server config (users.xml) now apply on the client too. Useful for format settings, e.g. date_time_output_format. #71178 (Michael Kolupaev).
- Fix possible error No such file or directorydue to unescaped special symbols in files for JSON subcolumns. #71182 (Pavel Kruglov).
- The table engines Kafka, NATS and RabbitMQ are now covered by their own grants in the SOURCEShierarchy. Add grants to any non-default database users that create tables with these engine types. #71250 (Christoph Wurm).
- Check the full mutation query before executing it (including subqueries). This prevents accidentally running an invalid query and building up dead mutations that block valid mutations. #71300 (Christoph Wurm).
- Rename filesystem cache setting skip_download_if_exceeds_query_cachetofilesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit. #71578 (Kseniia Sumarokova).
- Forbid Dynamic/Variant types in min/max functions to avoid confusion. #71761 (Pavel Kruglov).
- Remove support for Enumas well asUInt128andUInt256arguments indeltaSumTimestamp. Remove support forInt8,UInt8,Int16, andUInt16of the second ("timestamp") argument ofdeltaSumTimestamp. #71790 (Alexey Milovidov).
- Added source query validation when ClickHouse is used as a source for a dictionary. #72548 (Alexey Katsman).
New Features
- Implement SYSTEM LOAD PRIMARY KEY command to load primary indexes for all parts of a specified table or for all tables if no table is specified. This will be useful for benchmarks and to prevent extra latency during query execution. #66252 (ZAWA_ll).
- Added statement SYSTEM LOAD PRIMARY KEYfor loading the primary indexes of all parts in a specified table or for all tables if no table is specified. This can be useful for benchmarking and to prevent extra latency during query execution. #67733 (ZAWA_ll).
- Add CHECK GRANTquery to check whether the current user/role has been granted the specific privilege and whether the corresponding table/column exists in the memory. #68885 (Unalian).
- Added SQL syntax to describe workload and resource management. https://clickhouse.com/docs/en/operations/workload-scheduling. #69187 (Sergei Trifonov).
- The Iceberg data storage format provides the user with extensive options for modifying the schema of their table. In this pull request, reading a table in Iceberg format has been implemented, where the order of columns, column names, and simple type extensions have been changed. #69445 (Daniil Ivanik).
- Allow each authentication method to have its own expiration date, remove from user entity. #70090 (Arthur Passos).
- Push external user roles from query originator to other nodes in cluster. Helpful when only originator has access to the external authenticator (like LDAP). #70332 (Andrey Zvonov).
- Support alter from String to JSON. This PR also changes the serialization of JSON and Dynamic types to new version V2. Old version V1 can be still used by enabling setting merge_tree_use_v1_object_and_dynamic_serialization(can be used during upgrade to be able to rollback the version without issues). #70442 (Pavel Kruglov).
- Add function toUnixTimestamp64Secondwhich converts aDateTime64to aInt64value with fixed second precision, so we can support return negative value if date is before 00:00:00 UTC on Thursday, 1 January 1970. #70597 (zhanglistar).
- Add new setting enforce_index_structure_match_on_partition_manipulationto allow attach when source table's projections and secondary indices is a subset of those in the target table. Close #70602. #70603 (zwy991114).
- The output of function castdiffers with Apache Spark which cause difference in gluten project, see https://github.com/apache/incubator-gluten/issues/7602 This PR adds Spark text output format support feature, default closed. #70957 (zhanglistar).
- Added a new header type for S3 endpoints for user authentication (access_header). This allows to get some access header with the lowest priority, which will be overwritten withaccess_key_idfrom any other source (for example, a table schema or a named collection). #71011 (MikhailBurdukov).
- Initial implementation of settings tiers. #71145 (Raúl Marín).
- Add support for staleness clause in order by with fill operator. #71151 (Mikhail Artemenko).
- Implement simple CAST from Map/Tuple/Object to new JSON through serialization/deserialization from JSON string. #71320 (Pavel Kruglov).
- Added aliases anyRespectNulls,firstValueRespectNulls, andanyValueRespectNullsfor aggregation functionany. Also added aliasesanyLastRespectNullsandlastValueRespectNullsfor aggregation functionanyLast. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example:SELECT anyLastRespectNullsStateIfinstead ofanyLast_respect_nullsStateIf. #71403 (Peter Nguyen).
- Added the configuration date_time_utcparameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. #71560 (Ali).
- Added an option to select the side of the join that will act as the inner (build) table in the query plan. This is controlled by query_plan_join_swap_table, which can be set toauto. In this mode, ClickHouse will try to choose the table with the smallest number of rows. #71577 (Vladimir Cherkasov).
- Optimized memory usage for values of index granularity if granularity is constant for part. Added an ability to always select constant granularity for part (setting use_const_adaptive_granularity), which helps to ensure that it is always optimized in memory. It helps in large workloads (trillions of rows in shared storage) to avoid constantly growing memory usage by metadata (values of index granularity) of data parts. #71786 (Anton Popov).
- Implement allowed_feature_tieras a global switch to disable all experimental / beta features. #71841 (Raúl Marín).
- Add iceberg[S3;HDFS;Azure]Cluster,deltaLakeCluster,hudiClustertable functions. #72045 (Mikhail Artemenko).
- Add syntax ALTER USER {ADD|MODIFY|DROP SETTING},ALTER USER {ADD|DROP PROFILE}, the same forALTER ROLEandALTER PROFILE. #72050 (pufit).
- Added arrayPrAUCfunction, which calculates the AUC (Area Under the Curve) for the Precision Recall curve. #72073 (Emmanuel).
- Added cache for primary index of MergeTreetables (can be enabled by table settinguse_primary_key_cache). If lazy load and cache are enabled for primary index, it will be loaded to cache on demand (similar to mark cache) instead of keeping it in memory forever. Added prewarm of primary index on inserts/mergs/fetches of data parts and on restarts of table (can be enabled by settingprewarm_primary_key_cache). #72102 (Anton Popov).
- Add indexOfAssumeSorted function for array types. Optimizes the search in the case of a sorted in non-decreasing order array. #72517 (Eric Kurbanov).
- Allows to use a delimiter as a optional second argument for aggregate function groupConcat. #72540 (Yarik Briukhovetskyi).
- A new setting, http_response_headerswhich allows you to customize the HTTP response headers. For example, you can tell the browser to render a picture that is stored in the database. This closes #59620. #72656 (Alexey Milovidov).
- Add function fromUnixTimestamp64Secondwhich converts a Int64 Unix timestamp value to a DateTime64. #73146 (Robert Schulze).
Performance Improvements
- Add 2 new settings short_circuit_function_evaluation_for_nullsandshort_circuit_function_evaluation_for_nulls_thresholdthat allow to execute functions overNullablecolumns in short-circuit manner when the ratio of NULL values in the block of data exceeds the specified threshold. It means that the function will be executed only on rows with non-null values. It applies only to functions that return NULL value for rows where at least one argument is NULL. #60129 (李扬).
- Memory usage of clickhouse disks remove --recursiveis reduced for object storage disks. #67323 (Kirill).
- Now we won't copy input blocks columns for join_algorithm='parallel_hash'when distribute them between threads for parallel processing. #67782 (Nikita Taranov).
- Enable JIT compilation for more expressions: abs/bitCount/sign/modulo/pmod/isNull/isNotNull/assumeNotNull/to(U)Int*/toFloat*, comparison functions(=,<,>,>=,<=), logical functions(and,or). #70598 (李扬).
- Now parallel_hashalgorithm will be used (if applicable) whenjoin_algorithmsetting is set todefault. Two previous alternatives (directandhash) are still considered whenparallel_hashcannot be used. #70788 (Nikita Taranov).
- Optimized Replacingmerge algorithm for non intersecting parts. #70977 (Anton Popov).
- Do not list detached parts from readonly and write-once disks for metrics and system.detached_parts. #71086 (Alexey Milovidov).
- Do not calculate heavy asynchronous metrics by default. The feature was introduced in #40332, but it isn't good to have a heavy background job that is needed for only a single customer. #71087 (Alexey Milovidov).
- Improve the performance and accuracy of system.query_metric_log collection interval by reducing the critical region. #71473 (Pablo Marcos).
- Add option to extract common expressions from WHEREandONexpressions in order to reduce the number of hash tables used during joins. Can be enabled byoptimize_extract_common_expressions = 1. #71537 (János Benjamin Antal).
- Allows to use indexes on SELECTwithLowCardinality(String). #71598 (Yarik Briukhovetskyi).
- During query execution with parallel replicas and enabled local plan, skip index analysis on workers. The coordinator will choose ranges to read for workers based on index analysis on its side (on query initiator). #72109 (Igor Nikonov).
- Bring back optimization for reading subcolumns of single column in Compact parts from https://github.com/ClickHouse/ClickHouse/pull/57631. It was deleted accidentally. #72285 (Pavel Kruglov).
- Speedup sorting of LowCardinality(String)columns by de-virtualizing calls in comparator. #72337 (Alexander Gololobov).
- Optimize function argMin/Max for some simple data types. #72350 (alesapin).
- Optimize locking with shared locks in the memory tracker to reduce lock contention. #72375 (Jiebin Sun).
- Add a new setting, use_async_executor_for_materialized_views. Use async and potentially multithreaded execution of materialized view query, can speedup views processing during INSERT, but also consume more memory. #72497 (alesapin).
- Default values for settings max_size_to_preallocate_for_aggregation,max_size_to_preallocate_for_joinswere further increased to10^12, so the optimisation will be applied in more cases. #72555 (Nikita Taranov).
- Improved performance of deserialization of states of aggregate functions (in data type AggregateFunctionand in distributed queries). Slightly improved performance of parsing of formatRowBinary. #72818 (Anton Popov).
Improvement
- Higher-order functions with constant arrays and constant captured arguments will return constants. #58400 (Alexey Milovidov).
- Read-in-order optimization via generating virtual rows, so less data would be read during merge sort especially useful when multiple parts exist. #62125 (Shichao Jin).
- Query plan step names (EXPLAIN PLAN json=1) and pipeline processor names (EXPLAIN PIPELINE compact=0,graph=1) now have a unique id as a suffix. This allows to match processors profiler output and OpenTelemetry traces with explain output. #63518 (qhsong).
- Added option to check object exists after writing to Azure Blob Storage, this is controlled by setting check_objects_after_upload. #64847 (Smita Kulkarni).
- Fix use-after-dtor logic in HashTable destroyElements. #65279 (cangyin).
- Use Atomicdatabase by default inclickhouse-local. Address items 1 and 5 from #50647. Closes #44817. #68024 (Alexey Milovidov).
- Write buffer has to be canceled or finalized explicitly. Exceptions break the HTTP protocol in order to alert the client about error. #68800 (Sema Checherinda).
- Report running DDLWorker hosts by creating replica_dir and mark replicas active in DDLWorker. #69658 (Tuan Pham Anh).
- 
- Refactor DDLQueryStatusSource: - RenameDDLQueryStatusSourcetoDistributedQueryStatusSource, and make it a base class - Create two subclassesDDLOnClusterQueryStatusSourceandReplicatedDatabaseQueryStatusSourcederived fromDDLQueryStatusSourceto query the status of DDL tasks fromDDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts inDDLOnClusterQueryStatusSource`. #69660 (Tuan Pham Anh).
 
- Refactor 
- Adding a new cancellation logic: CancellationCheckerchecks timeouts for every started query and stops them once the timeout has reached. #69880 (Yarik Briukhovetskyi).
- Remove the allow_experimental_join_conditionsetting, allowing non-equi conditions by default. #69910 (Vladimir Cherkasov).
- Enable parallel_replicas_local_planby default. Building a full-fledged local plan on the query initiator improves parallel replicas performance with less resource consumption, provides opportunities to apply more query optimizations. #70171 (Igor Nikonov).
- Add ability to set user/password in http_handlers (for dynamic_query_handler/predefined_query_handler). #70725 (Azat Khuzhin).
- Support ALTER TABLE ... MODIFY/RESET SETTING ...for certain settings in storage S3Queue. #70811 (Kseniia Sumarokova).
- 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).
- Add --threadsparameter toclickhouse-compressor, which allows to compress data in parallel. #70860 (Alexey Milovidov).
- Make the Replxx client history size configurable. #71014 (Jiří Kozlovský).
- Added a setting prewarm_mark_cachewhich enables loading of marks to mark cache on inserts, merges, fetches of parts and on startup of the table. #71053 (Anton Popov).
- Boolean support for parquet native reader. #71055 (Arthur Passos).
- Retry more errors when interacting with S3, such as "Malformed message". #71088 (Alexey Milovidov).
- Lower log level for some messages about S3. #71090 (Alexey Milovidov).
- Support write hdfs files with space. #71105 (exmy).
- system.session_logis quite okay. This closes #51760. #71150 (Alexey Milovidov).
- Fixes RIGHT / FULL joins in queries with parallel replicas. Now, RIGHT joins can be executed with parallel replicas (right table reading is distributed). FULL joins can't be parallelized among nodes, - executed locally. #71162 (Igor Nikonov).
- Added settings limiting the number of replicated tables, dictionaries and views. #71179 (Kirill).
- Fixes #71227. #71286 (Arthur Passos).
- Automatic GROUP BY/ORDER BYto disk based on the server/user memory usage. Controlled withmax_bytes_ratio_before_external_group_by/max_bytes_ratio_before_external_sortquery settings. #71406 (Azat Khuzhin).
- Add per host dashboards Overview (host)andCloud overview (host)to advanced dashboard. #71422 (alesapin).
- Function translatenow supports character deletion if thefromargument contains more characters than thetoargument. Example:SELECT translate('clickhouse', 'clickhouse', 'CLICK')now returnsCLICK. #71441 (shuai.xu).
- Added new functions parseDateTime64,parseDateTime64OrNullandparseDateTime64OrZero. Compared to the existing functionparseDateTime(and variants), they return a value of typeDateTime64instead ofDateTime. #71581 (kevinyhzou).
- Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. #71595 (alesapin).
- The command line applications will highlight syntax even for multi-statements. #71622 (Alexey Milovidov).
- Command-line applications will return non-zero exit codes on errors. In previous versions, the disksapplication returned zero on errors, and other applications returned zero for errors 256 (PARTITION_ALREADY_EXISTS) and 512 (SET_NON_GRANTED_ROLE). #71623 (Alexey Milovidov).
- The Verticalformat (which is also activated when you end your query with\G) gets the features of Pretty formats, such as: - highlighting thousand groups in numbers; - printing a readable number tip. #71630 (Alexey Milovidov).
- Allow to disable memory buffer increase for filesystem cache via setting filesystem_cache_prefer_bigger_buffer_size. #71640 (Kseniia Sumarokova).
- Add a separate setting background_download_max_file_segment_sizefor background download max file segment size in filesystem cache. #71648 (Kseniia Sumarokova).
- Changes the default value of enable_http_compressionfrom 0 to 1. Closes #71591. #71774 (Peter Nguyen).
- Support ALTER from Object to JSON. #71784 (Pavel Kruglov).
- Slightly better JSON type parsing: if current block for the JSON path contains values of several types, try to choose the best type by trying types in special best-effort order. #71785 (Pavel Kruglov).
- Previously reading from system.asynchronous_metricswould wait for concurrent update to finish. This can take long time if system is under heavy load. With this change the previously collected values can always be read. #71798 (Alexander Gololobov).
- Set polling_max_timeout_msto 10 minutes,polling_backoff_msto 30 seconds. #71817 (Kseniia Sumarokova).
- Queries like 'SELECT - FROM t LIMIT 1' used to load part indexes even though they were not used. #71866 (Alexander Gololobov).
- Allow_reorder_prewhere_conditions is on by default with old compatibility settings. #71867 (Raúl Marín).
- Do not increment the ILLEGAL_TYPE_OF_ARGUMENTcounter in thesystem.errorstable when thebitmapTransformfunction is used, and argument types are valid. #71971 (Dmitry Novik).
- When retrieving data directly from a dictionary using Dictionary storage, dictionary table function, or direct SELECT from the dictionary itself, it is now enough to have SELECTpermission ordictGetpermission for the dictionary. This aligns with previous attempts to prevent ACL bypasses: https://github.com/ClickHouse/ClickHouse/pull/57362 and https://github.com/ClickHouse/ClickHouse/pull/65359. It also makes the latter one backward compatible. #72051 (Nikita Mikhaylov).
- On the advanced dashboard HTML page added a dropdown selector for the dashboard from system.dashboardstable. #72081 (Sergei Trifonov).
- Respect prefer_locahost_replicawhen building plan for distributedINSERT ... SELECT. #72190 (filimonov).
- The problem is described here. Azure Iceberg Writer creates Iceberg metadata files (as well as manifest files) that violate specs. In this PR I added an attempt to read v1 iceberg format metadata with v2 reader (cause they write it in a this way), and added error when they didn't create corresponding fields in a manifest file. #72277 (Daniil Ivanik).
- Move JSON/Dynamic/Variant types from experimental features to beta. #72294 (Pavel Kruglov).
- Now it's allowed to CREATE MATERIALIZED VIEWwithUNION [ALL]in query. Behavior is the same as for matview withJOIN: *only first table inSELECTexpression will work as trigger for insert- , all other tables will be ignored. #72347 (alesapin).
- Speed up insertions into merge tree in case of a single value of partition key inside inserted batch. #72348 (alesapin).
- Add the new MergeTreeIndexGranularityInternalArraysTotalSize metric to system.metrics. This metric is needed to find the instances with huge datasets susceptible to the high memory usage issue. #72490 (Miсhael Stetsyuk).
- All spellings of word Nullnow recognised when query usesFormat Null. Previously other forms (e.g.NULL) did not result in exceptions being thrown, but at the same time formatNullwasn't actually used in those cases. #72658 (Nikita Taranov).
- Allow unknown values in set that are not present in Enum. Fix #72662. #72686 (zhanglistar).
- Add total_bytes_with_inactive to system.tables to count the total bytes of inactive parts. #72690 (Kai Zhu).
- Add MergeTreeSettings to system.settings_changes. #72694 (Raúl Marín).
- Support string search operator(eg. like) for Enum data type, fix #72661. #72732 (zhanglistar).
- Support JSON type in notEmpty function. #72741 (Pavel Kruglov).
- Support parsing GCS S3 error AuthenticationRequired. #72753 (Vitaly Baranov).
- Support Dynamic type in functions ifNull and coalesce. #72772 (Pavel Kruglov).
- Added JoinBuildTableRowCount/JoinProbeTableRowCount/JoinResultRowCountprofile events. #72842 (Vladimir Cherkasov).
- Support Dynamic in functions toFloat64/touInt32/etc. #72989 (Pavel Kruglov).
Bug Fix (user-visible misbehavior in an official stable release)
- The parts deduplicated during ATTACH PARTquery don't get stuck with theattaching_prefix anymore. #65636 (Kirill).
- Fix for the bug when dateTime64 losing precision for the INfunction. #67230 (Yarik Briukhovetskyi).
- Fix possible logical error when using functions with IGNORE/RESPECT NULLSinORDER BY ... WITH FILL, close #57609. #68234 (Vladimir Cherkasov).
- Fixed rare logical errors in asynchronous inserts with format Nativein case of reached memory limit. #68965 (Anton Popov).
- Fix COMMENT in CREATE TABLE for EPHEMERAL column. #70458 (Yakov Olkhovskiy).
- Fix logical error in JSONExtract with LowCardinality(Nullable). #70549 (Pavel Kruglov).
- Fixes behaviour when table name is too long. #70810 (Yarik Briukhovetskyi).
- Add ability to override Content-Type by user headers in the URL engine. #70859 (Artem Iurin).
- Fix logical error in StorageS3Queue"Cannot create a persistent node in /processed since it already exists". #70984 (Kseniia Sumarokova).
- Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. #71089 (Shichao Jin).
- Fix wrong value in system.query_metric_log due to unexpected race condition. #71124 (Pablo Marcos).
- Fix mismatched aggreage function name of quantileExactWeightedInterpolated. The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/69619. cc @Algunenano. #71168 (李扬).
- Fix bad_weak_ptr exception with Dynamic in functions comparison. #71183 (Pavel Kruglov).
- Don't delete a blob when there are nodes using it in ReplicatedMergeTree with zero-copy replication. #71186 (Antonio Andelic).
- Fix ignoring format settings in Native format via HTTP and Async Inserts. #71193 (Pavel Kruglov).
- SELECT queries run with setting use_query_cache = 1are no longer rejected if the name of a system table appears as a literal, e.g.SELECT - FROM users WHERE name = 'system.metrics' SETTINGS use_query_cache = true;now works. #71254 (Robert Schulze).
- Fix bug of memory usage increase if enable_filesystem_cache=1, but disk in storage configuration did not have any cache configuration. #71261 (Kseniia Sumarokova).
- Fix possible error "Cannot read all data" erros during deserialization of LowCardinality dictionary from Dynamic column. #71299 (Pavel Kruglov).
- Fix incomplete cleanup of parallel output format in the client. #71304 (Raúl Marín).
- Added missing unescaping in named collections. Without fix clickhouse-server can't start. #71308 (MikhailBurdukov).
- Fix async inserts with empty blocks via native protocol. #71312 (Anton Popov).
- Fix inconsistent AST formatting when granting wrong wildcard grants #71309. #71332 (pufit).
- Check suspicious and experimental types in JSON type hints. #71369 (Pavel Kruglov).
- Fix error Invalid number of rows in Chunk with Variant column. #71388 (Pavel Kruglov).
- Fix crash in mongodbtable function when passing wrong arguments (e.g.NULL). #71426 (Vladimir Cherkasov).
- Fix crash with optimize_rewrite_array_exists_to_has. #71432 (Raúl Marín).
- Fix NoSuchKey error during transaction rollback when creating a directory fails for the palin_rewritable disk. #71439 (Julia Kartseva).
- Fixed the usage of setting max_insert_delayed_streams_for_parallel_writein inserts. Previously it worked incorrectly which could lead to high memory usage in inserts which write data into several partitions. #71474 (Anton Popov).
- Fix possible error Argument for function must be constant(old analyzer) in case when arrayJoin can apparently appear inWHEREcondition. Regression after https://github.com/ClickHouse/ClickHouse/pull/65414. #71476 (Nikolai Kochetov).
- Prevent crash in SortCursor with 0 columns (old analyzer). #71494 (Raúl Marín).
- Fix date32 out of range caused by uninitialized orc data. For more details, refer to https://github.com/apache/incubator-gluten/issues/7823. #71500 (李扬).
- Fix counting column size in wide part for Dynamic and JSON types. #71526 (Pavel Kruglov).
- Analyzer fix when query inside materialized view uses IN with CTE. Closes #65598. #71538 (Maksim Kita).
- Return 0 or default char instead of throwing an error in bitShift functions in case of out of bounds. #71580 (Pablo Marcos).
- Fix server crashes while using materialized view with certain engines. #71593 (Pervakov Grigorii).
- Array join with a nested data structure, which contains an alias to a constant array was leading to a null pointer dereference. This closes #71677. #71678 (Alexey Milovidov).
- Fix LOGICAL_ERROR when doing ALTER with empty tuple. This fixes #71647. #71679 (Amos Bird).
- Don't transform constant set in predicates over partition columns in case of NOT IN operator. #71695 (Eduard Karacharov).
- Fix CAST from LowCardinality(Nullable) to Dynamic. Previously it could lead to error Bad cast from type DB::ColumnVector<int> to DB::ColumnNullable. #71742 (Pavel Kruglov).
- Fix exception for toDayOfWeek on WHERE condition with primary key of DateTime64 type. #71849 (Yakov Olkhovskiy).
- Fixed filling of defaults after parsing into sparse columns. #71854 (Anton Popov).
- Fix GROUPING function error when input is ALIAS on distributed table, close #68602. #71855 (Vladimir Cherkasov).
- Fixed select statements that use WITH TIESclause which might not return enough rows. #71886 (wxybear).
- Fix an exception of TOO_LARGE_ARRAY_SIZE caused when a column of arrayWithConstant evaluation is mistaken to cross the array size limit. #71894 (Udi).
- clickhouse-benchmarkreported wrong metrics for queries taking longer than one second. #71898 (Alexey Milovidov).
- Fix data race between the progress indicator and the progress table in clickhouse-client. This issue is visible when FROM INFILE is used. Intercept keystrokes during INSERT queries to toggle progress table display. #71901 (Julia Kartseva).
- Fix serialization of Dynamic values in Pretty JSON formats. #71923 (Pavel Kruglov).
- Fix rows_processed column in system.s3/azure_queue_log broken in 24.6. Closes #69975. #71946 (Kseniia Sumarokova).
- Fixed case when s3/s3Clusterfunctions could return incomplete result or throw an exception. It involved using glob pattern in s3 uri (likepattern/*) and an empty object should exist with the keypattern/(such objects automatically created by S3 Console). Also default value for settings3_skip_empty_fileschanged fromfalsetotrueby default. #71947 (Nikita Taranov).
- Fix a crash in clickhouse-client syntax highlighting. Closes #71864. #71949 (Nikolay Degterinsky).
- Fix Illegal typeerror forMergeTreetables with binary monotonic function inORDER BYwhen the first argument is constant. Fixes #71941. #71966 (Nikolai Kochetov).
- Allow only SELECT queries in EXPLAIN AST used inside subquery. Other types of queries lead to logical error: 'Bad cast from type DB::ASTCreateQuery to DB::ASTSelectWithUnionQuery' or Inconsistent AST formatting. #71982 (Pavel Kruglov).
- When insert a record by clickhouse-client, client will read column descriptions from server. but there was a bug that we wrote the descritions with a wrong order , it should be [statistics, ttl, settings]. #71991 (Han Fei).
- Fix formatting of MOVE PARTITION ... TO TABLE ...alter commands whenformat_alter_commands_with_parenthesesis enabled. #72080 (János Benjamin Antal).
- Add inferred format name to create query in File/S3/URL/HDFS/Azure engines. Previously the format name was inferred each time the server was restarted, and if the specified data files were removed, it led to errors during server startup. #72108 (Pavel Kruglov).
- Fix a bug where min_age_to_force_merge_on_partition_onlywas getting stuck trying to merge down the same partition repeatedly that was already merged to a single part and not merging partitions that had multiple parts. #72209 (Christoph Wurm).
- Fixed a crash in SimpleSquashingChunksTransformthat occurred in rare cases when processing sparse columns. #72226 (Vladimir Cherkasov).
- Fixed data race in GraceHashJoinas the result of which some rows might be missing in the join output. #72233 (Nikita Taranov).
- Fixed ALTER DELETEqueries with materialized_block_numbercolumn (if settingenable_block_number_columnis enabled). #72261 (Anton Popov).
- Fixed data race when ColumnDynamic::dumpStructure()is called concurrently e.g. inConcurrentHashJoinconstructor. #72278 (Nikita Taranov).
- Fix possible LOGICAL_ERRORwith duplicate columns inORDER BY ... WITH FILL. #72387 (Vladimir Cherkasov).
- Fixed mismatched types in several cases after applying optimize_functions_to_subcolumns. #72394 (Anton Popov).
- Fix failure on parsing BACKUP DATABASE db EXCEPT TABLES db.tablequeries. #72429 (Konstantin Bogdanov).
- Don't allow creating empty Variant. #72454 (Pavel Kruglov).
- Fix invalid formatting of result_part_pathinsystem.merges. #72567 (Konstantin Bogdanov).
- Fix parsing a glob with one element. #72572 (Konstantin Bogdanov).
- Fix query generation for the follower server in case of a distributed query with ARRAY JOIN. Fixes #69276. #72608 (Dmitry Novik).
- Fix a bug when DateTime64 in DateTime64 returns nothing. #72640 (Yarik Briukhovetskyi).
- Fix "No such key" error in S3Queue Unordered mode with tracked_files_limitsetting smaller than s3 files appearance rate. #72738 (Kseniia Sumarokova).
- Dropping mark cache might take noticeable time if it is big. If we hold context mutex during this it block many other activities, even new client connection cannot be established until it is released. And holding this mutex is not actually required for synchronization, it is enough to have a local reference to the cache via shared ptr. #72749 (Alexander Gololobov).
- PK cache was heavily underestimating it's size on one of the test instances. In particular LowCardinality columns were not including dictionary size. The fix is to use column->allocatedBytes() plus some more overhead estimates for cache entry size. #72750 (Alexander Gololobov).
- Fix exception thrown in RemoteQueryExecutor when user does not exist locally. #72759 (Andrey Zvonov).
- Fixed mutations with materialized _block_numbercolumn (if settingenable_block_number_columnis enabled). #72854 (Anton Popov).
- Fix backup/restore with plain rewritable disk in case there are empty files in backup. #72858 (Kseniia Sumarokova).
- Properly cancel inserts in DistributedAsyncInsertDirectoryQueue. #72885 (Antonio Andelic).
- Fixed crash while parsing of incorrect data into sparse columns (can happen with enabled setting enable_parsing_to_custom_serialization). #72891 (Anton Popov).
- Fix potential crash during backup restore. #72947 (Kseniia Sumarokova).
- Fixed bug in parallel_hashJOIN method that might appear when query has complex condition in theONclause with inequality filters. #72993 (Nikita Taranov).
- Use default format settings during JSON parsing to avoid broken deserialization. #73043 (Pavel Kruglov).
- Fix crash in transactions with unsupported storage. #73045 (Raúl Marín).
- Check for duplicate JSON keys during Tuple parsing. Previously it could lead to logical error Invalid number of rows in Chunkduring parsing. #73082 (Pavel Kruglov).
