mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 21:42:39 +00:00
84 KiB
84 KiB
sidebar_position | sidebar_label |
---|---|
1 | 2024 |
2024 Changelog
ClickHouse release v24.11.1.2557-stable (1574e794bf
) FIXME as compared to v24.11.1.1-new (c82cf25b3e
)
Backward Incompatible Change
- Remove system tables
generate_series
andgenerateSeries
. They were added by mistake here: #59390. #71091 (Alexey Milovidov). - Remove
StorageExternalDistributed
. Closes #70600. ### Documentation entry for user-facing changes. #71176 (flynn). - Fix possible error
No such file or directory
due 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
SOURCES
hierarchy. 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_cache
tofilesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit
. #71578 (Kseniia Sumarokova). - Remove support for
Enum
as well asUInt128
andUInt256
arguments indeltaSumTimestamp
. Remove support forInt8
,UInt8
,Int16
, andUInt16
of the second ("timestamp") argument ofdeltaSumTimestamp
. #71790 (Alexey Milovidov).
New Feature
- A new data type,
BFloat16
, represents 16-bit floating point numbers with 8-bit exponent, sign, and 7-bit mantissa. This closes #44206. This closes #49937. #64712 (Alexey Milovidov). Added an option to select the side of the join that will act as the inner table in the query plan. This is controlled byResubmitted https://github.com/ClickHouse/ClickHouse/pull/71577. #68682 (Vladimir Cherkasov).query_plan_join_inner_table_selection
, which can be set toauto
. In this mode, ClickHouse will try to choose the table with the smallest number of rows.- Add
CHECK GRANT
query 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).
- Added server setting
async_load_system_database
that allows the server to start with not fully loaded system database. This helps to start ClickHouse faster if there are many system tables. #69847 (Sergei Trifonov). - 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). - 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_id
from 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
, andanyValueRespectNulls
for aggregation functionany
. Also added aliasesanyLastRespectNulls
andlastValueRespectNulls
for aggregation functionanyLast
. This allows using more natural camel-case-only syntax rather than mixed camel-case/underscore syntax, for example:SELECT anyLastRespectNullsStateIf
instead ofanyLast_respect_nullsStateIf
. #71403 (Peter Nguyen). - Added the configuration
date_time_utc
parameter, enabling JSON log formatting to support UTC date-time in RFC 3339/ISO8601 format. #71560 (Ali). - 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_tier
as a global switch to disable all experimental / beta features. #71841 (Raúl Marín). - Add
iceberg[S3;HDFS;Azure]Cluster
,deltaLakeCluster
,hudiCluster
table functions. #72045 (Mikhail Artemenko).
Performance Improvement
- Add 2 new settings
short_circuit_function_evaluation_for_nulls
andshort_circuit_function_evaluation_for_nulls_threshold
that allow to execute functions overNullable
columns 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 (李扬). - Now we won't copy input blocks columns for
join_algorithm='parallel_hash'
when distribute them between threads for parallel processing. #67782 (Nikita Taranov). - Optimized
Replacing
merge 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).
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
Atomic
database 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 (tuanpach).
-
- Refactor
DDLQueryStatusSource
: * RenameDDLQueryStatusSource
toDistributedQueryStatusSource
, and make it a base class * Create two subclassesDDLOnClusterQueryStatusSource
andReplicatedDatabaseQueryStatusSource
derived fromDDLQueryStatusSource
to query the status of DDL tasks fromDDL On Cluster and Replicated databases respectively. 2. Support stop waiting for offline hosts in
DDLOnClusterQueryStatusSource`. #69660 (tuanpach).
- Refactor
- 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).
- Better error-handling and cancellation of
ON CLUSTER
backups and restores: - If a backup or restore fails on one host then it'll be cancelled on other hosts automatically - No weird errors must be produced because some hosts failed while other hosts continued their work - If a backup or restore is cancelled on one host then it'll be cancelled on other hosts automatically - Fix issues withtest_disallow_concurrency
- now disabling of concurrency must work better - Backups and restores now are much more resistant to ZooKeeper disconnects. #70027 (Vitaly Baranov). - Enable
parallel_replicas_local_plan
by 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). - Fix the metadata_version record in ZooKeeper in restarting thread rather than in attach thread. #70297 (Miсhael Stetsyuk).
- 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
--threads
parameter toclickhouse-compressor
, which allows to compress data in parallel. #70860 (Alexey Milovidov). - Fix the issue where ClickHouse in Docker containers printed "get_mempolicy: Operation not permitted" into stderr due to restricted syscalls. #70900 (filimonov).
- Added the ability to reload client certificates in the same way as the procedure for reloading server certificates. #70997 (Roman Antonov).
- Refactored internal structure of files which work with DataLake Storages. #71012 (Daniil Ivanik).
- Make the Replxx client history size configurable. #71014 (Jiří Kozlovský).
- Added a setting
prewarm_mark_cache
which 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_log
is 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).
- Use
AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE
instead ofAWS_CONTAINER_AUTHORIZATION_TOKEN
if former is available. Fixes #71074. #71269 (Konstantin Bogdanov). - Remove the metadata_version ZooKeeper node creation from RMT restarting thread. The only scenario where we need to create this node is when the user updated from a version earlier than 20.4 straight to one later than 24.10. ClickHouse does not support upgrades that span more than a year, so we should throw an exception and ask the user to update gradually, instead of creating the node. #71385 (Miсhael Stetsyuk).
- Add per host dashboards
Overview (host)
andCloud overview (host)
to advanced dashboard. #71422 (alesapin). - The methods
removeObject
andremoveObjects
are not idempotent. When retries happen due to network errors, the result could beobject not found
because it has been deleted at previous attempts. #71529 (Sema Checherinda). - Added new functions
parseDateTime64
,parseDateTime64OrNull
andparseDateTime64OrZero
. Compared to the existing functionparseDateTime
(and variants), they return a value of typeDateTime64
instead ofDateTime
. #71581 (kevinyhzou). - Allow using clickhouse with a file argument as --queries-file. #71589 (Raúl Marín).
- Shrink to fit index_granularity array in memory to reduce memory footprint for MergeTree table engines family. #71595 (alesapin).
clickhouse-local
uses implicit SELECT by default, which allows to use it as a calculator. Improve the syntax highlighting for the implicit SELECT mode. #71620 (Alexey Milovidov).- 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
disks
application returned zero on errors, and other applications returned zero for errors 256 (PARTITION_ALREADY_EXISTS
) and 512 (SET_NON_GRANTED_ROLE
). #71623 (Alexey Milovidov). - When user/group is given as ID, the
clickhouse su
fails. This patch fixes it to acceptUID:GID
as well. ### Documentation entry for user-facing changes. #71626 (Mikhail f. Shiryaev). - The
Vertical
format (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_size
for background download max file segment size in filesystem cache. #71648 (Kseniia Sumarokova). - Forbid Dynamic/Variant types in min/max functions to avoid confusion. #71761 (Pavel Kruglov).
- Changes the default value of
enable_http_compression
from 0 to 1. Closes #71591. #71774 (Peter Nguyen). - 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_metrics
would 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_ms
to 10 minutes,polling_backoff_ms
to 30 seconds. #71817 (Kseniia Sumarokova). - Turn-off filesystem cache setting
boundary_alignment
for non-disk read. #71827 (Kseniia Sumarokova). - Update
HostResolver
3 times in ahistory
period. #71863 (Sema Checherinda). - 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_ARGUMENT
counter in thesystem.errors
table when thebitmapTransform
function 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
SELECT
permission ordictGet
permission 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.dashboards
table. #72081 (Sergei Trifonov). - Backported in #72471: Move JSON/Dynamic/Variant types from experimental features to beta. #72294 (Pavel Kruglov).
Bug Fix (user-visible misbehavior in an official stable release)
- The parts deduplicated during
ATTACH PART
query don't get stuck with theattaching_
prefix anymore. #65636 (Kirill). - Fix for the bug when dateTime64 losing precision for the
IN
function. #67230 (Yarik Briukhovetskyi). - Fix possible logical error when using functions with
IGNORE/RESPECT NULLS
inORDER BY ... WITH FILL
, close #57609. #68234 (Vladimir Cherkasov). - Fixed rare logical errors in asynchronous inserts with format
Native
in 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).
- Allow system drop replica zkpath when there is another replica with the same zk path. #70642 (MikhailBurdukov).
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).- 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). - Fixed named sessions not being closed and hanging on forever under certain circumstances. #70998 (Márcio Martins).
- Fix the bug that didn't consider _row_exists column in rebuild option of projection lightweight delete. #71089 (Shichao Jin).
- Fix
AT_* is out of range
problem when running on Oracle Linux UEK 6.10. #71109 (Örjan Fors). - Fix wrong value in system.query_metric_log due to unexpected race condition. #71124 (Pablo Marcos).
- Check if default DB is present after authorization. Fixes #71097. #71140 (Konstantin Bogdanov).
- 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).
- Checks that read 7z file is on a local machine. #71184 (Daniil Ivanik).
- 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 = 1
are 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).
- Add try/catch to data parts destructors to avoid terminate. #71364 (alesapin).
- Check suspicious and experimental types in JSON type hints. #71369 (Pavel Kruglov).
- Start memory worker thread on non-Linux OS too (fixes #71051). #71384 (Alexandre Snarskii).
- Fix error Invalid number of rows in Chunk with Variant column. #71388 (Pavel Kruglov).
- Fix error column "attgenerated" does not exist for old PostgreSQL, fix #60651. #71396 (0xMihalich).
- To avoid spamming the server logs, failing authentication attempts are now logged at level
DEBUG
instead ofERROR
. #71405 (Robert Schulze). - Fix crash in
mongodb
table 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).
- Backported in #72336: 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_write
in 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 inWHERE
condition. 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).
- Avoid crash when using a UDF in a constraint. #71541 (Raúl Marín).
- 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 docker init script fail log message for more clean understanding. #71734 (Андрей).
- 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). - Acquiring zero-copy shared lock before moving a part to zero-copy disk to prevent possible data loss if Keeper is unavailable. #71845 (Aleksei Filatov).
- 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).
- Fix possible crash when using
allow_experimental_join_condition
, close #71693. #71857 (Vladimir Cherkasov). - Fixed select statements that use
WITH TIES
clause 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-benchmark
reported 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).
- Use auxiliary keepers for cluster autodiscovery. #71911 (Anton Ivashkin).
- Backported in #72275: 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
/s3Cluster
functions 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_files
changed fromfalse
totrue
by default. #71947 (Nikita Taranov). - Fix a crash in clickhouse-client syntax highlighting. Closes #71864. #71949 (Nikolay Degterinsky).
- Fix
Illegal type
error forMergeTree
tables with binary monotonic function inORDER BY
when 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_parentheses
is enabled. #72080 (János Benjamin Antal). - Backported in #72300: Fix bugs when using UDF in join on expression with the old analyzer. #72179 (Raúl Marín).
- Backported in #72378: Fixed a crash in
SimpleSquashingChunksTransform
that occurred in rare cases when processing sparse columns. #72226 (Vladimir Cherkasov). - Backported in #72329: Fixed data race in
GraceHashJoin
as the result of which some rows might be missing in the join output. #72233 (Nikita Taranov). - Backported in #72372: Fixed
ALTER DELETE
queries with materialized_block_number
column (if settingenable_block_number_column
is enabled). #72261 (Anton Popov). - Backported in #72318: Fixed data race when
ColumnDynamic::dumpStructure()
is called concurrently e.g. inConcurrentHashJoin
constructor. #72278 (Nikita Taranov). - Backported in #72420: Use
AWS_CONTAINER_AUTHORIZATION_TOKEN_FILE
instead ofAWS_CONTAINER_AUTHORIZATION_TOKEN_PATH
. Fixes #71074. #72397 (Konstantin Bogdanov).
Build/Testing/Packaging Improvement
- Add the script to update sources of docker official library. #57203 (Mikhail f. Shiryaev).
- The build system will prevent libraries with unexpected licenses. #70988 (Alexey Milovidov).
- Upgrade docker base image for clickhouse-server and keeper to
ubuntu:22.04
. Breaking change: the minimal supported docker version is20.10.10
. #71505 (Mikhail f. Shiryaev). - Improve clickhouse-server Dockerfile.ubuntu. Deprecate
CLICKHOUSE_UID/CLICKHOUSE_GID
envs. RemoveCLICKHOUSE_DOCKER_RESTART_ON_EXIT
processing to complien requirements. Consistentclickhouse/clickhouse-server/clickhouse-keeper
execution to not have it plain in one place and/usr/bin/clickhouse*
in another. #71573 (Mikhail f. Shiryaev).
NO CL CATEGORY
- Backported in #72289:. #72258 (Raúl Marín).
- Backported in #72343:. #72319 (Raúl Marín).
NO CL ENTRY
- NO CL ENTRY: 'Revert "Miscellaneous"'. #71083 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "Revert "Miscellaneous""'. #71084 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "[RFC] Fix optimize_functions_to_subcolumns optimization"'. #71220 (Raúl Marín).
- NO CL ENTRY: 'Revert "SQL syntax for workload and resource management"'. #71251 (Alexander Tokmakov).
- NO CL ENTRY: 'Revert "Revert "SQL syntax for workload and resource management""'. #71266 (Sergei Trifonov).
- NO CL ENTRY: 'Revert "Selection of hash join inner table"'. #71527 (Raúl Marín).
- NO CL ENTRY: 'Revert "Enable enable_job_stack_trace by default"'. #71619 (Raúl Marín).
- NO CL ENTRY: 'Revert "Remove ridiculous code bloat"'. #71914 (Raúl Marín).
- NO CL ENTRY: 'Revert "Revert "Remove ridiculous code bloat""'. #71945 (Alexey Milovidov).
- NO CL ENTRY: 'Revert "CI: Functional Tests with praktika"'. #71974 (Max Kainov).
- NO CL ENTRY: 'CI: Functional Tests with praktika'. #71976 (Max Kainov).
NOT FOR CHANGELOG / INSIGNIFICANT
- Refactor TempDataOnDisk. #66606 (Vladimir Cherkasov).
- Bump krb5 from v1.21.2 to v1.21.3. #69360 (Robert Schulze).
- USearch: Enable SimSIMD backend + enable dynamic dispatch. #69387 (Robert Schulze).
- CI: enable libfuzzer. #70112 (Yakov Olkhovskiy).
- Use
clang-19
. #70414 (Konstantin Bogdanov). - Randomize Keeper feature flags in integration tests. #70523 (Antonio Andelic).
- All the patches for arrow were re-applied in this PR: https://github.com/ClickHouse/arrow/pull/68. #70691 (Nikita Mikhaylov).
- Check number of arguments for function with Dynamic argument. #70749 (Nikita Taranov).
- Add a settings
filesystem_cache_enable_background_download_for_metadata_files
for filesystem cache to allow to disable background download of filesystem cache for metadata files. This feature is needed for private code feature, for public version it does not make much sense. #70806 (Kseniia Sumarokova). - Make ParquetMetadata say whether bloom filter is present. #70947 (Michael Kolupaev).
- test for reproducing that ReplacingMergeTree depends on the order of part attachment. #71010 (Konstantin Morozov).
- CI: Build Job with praktika. #71015 (Max Kainov).
- Fix bad test
01524_do_not_merge_across_partitions_select_final.sql
. #71035 (Alexey Milovidov). - Enable enable_job_stack_trace by default. #71039 (Yakov Olkhovskiy).
- Fix two logical errors when reading from stdin in clickhouse local. #71046 (Michael Kolupaev).
- Sync changes to
ProtocolServerAdapter
. #71058 (Antonio Andelic). - Fix a typo. #71067 (Alexey Milovidov).
- Miscellaneous. #71070 (Alexey Milovidov).
- Remove bad test
test_system_replicated_fetches
. #71071 (Alexey Milovidov). - Update version after release. #71076 (robot-clickhouse).
- CI: Functional tests for ARM + ASAN binary. #71079 (Max Kainov).
- CI: Functional Tests with praktika. #71081 (Max Kainov).
- Fixup of TrivialMergeSelector. #71082 (Alexey Milovidov).
- Sync integration test with private. #71096 (Alexey Milovidov).
- Unique symbols in the
system.coverage_log
. #71099 (Alexey Milovidov). - Better log messages. #71102 (Alexey Milovidov).
- Fix error in Replicated database. #71103 (Alexey Milovidov).
- Update error message for JSONAsObject format. #71123 (Pavel Kruglov).
- Initial changelog for 24.10. #71127 (Raúl Marín).
- Followup #70520. #71129 (Vladimir Cherkasov).
- Update compatibility setting for
hnsw_candidate_list_size_for_search
. #71133 (Robert Schulze). - Try fix rabbitmq. #71143 (Kseniia Sumarokova).
- Randomize setting
enable_vertical_final
. #71144 (Anton Popov). - Fix bad test
02561_sorting_constants_and_distinct_crash
. #71147 (Alexey Milovidov). - Add test to verify #62308 works. #71149 (János Benjamin Antal).
- Fix 02932_refreshable_materialized_views_1 flakiness. #71160 (Michael Kolupaev).
- Use
_minmax_count_projection
instead ofOptimized trivial count
forReadFromPreparedSource
node in trivial count optimized query plans, providing a more descriptive representation of the projection-based trivial count optimization. This addresses #70939. #71166 (Amos Bird). - Close #8687. #71169 (Alexey Milovidov).
- Fixes for interactive metrics. #71173 (Julia Kartseva).
- Maybe not GWPAsan by default. #71174 (Antonio Andelic).
- Print compression method in
clickhouse-compressor --stat
. Useful for inspecting random data files. #71192 (Amos Bird). - Updating the events into the recent category and adding the new york event. #71194 (Zoe Steinkamp).
- Improve error and log messages around memory usage. #71195 (Raúl Marín).
- Minor test adjustments. #71199 (Raúl Marín).
- Fix LOGICAL_ERROR on wrong scalar subquery argument to table functions. #71216 (Raúl Marín).
- Disable enable_named_columns_in_function_tuple for 24.10. #71219 (Raúl Marín).
- Update README.md - Update meetups. #71223 (Tanya Bragin).
- Fix
WITH TOTALS
in subquery with parallel replicas. #71224 (Nikita Taranov). - Ignore
No such key
exceptions in some cases. #71236 (Antonio Andelic). - Make cloud sync title shorter. #71255 (Raúl Marín).
- Update README.md - Meetups update. #71271 (Tanya Bragin).
- Improve system.query_metric_log to remove flakiness. #71295 (Pablo Marcos).
- Fix debug log timestamp. #71311 (Pablo Marcos).
- Expose one more simple merge selector setting. #71313 (alesapin).
- Better style for some sever-level settings. #71319 (alesapin).
- Sync some changes. #71321 (Kseniia Sumarokova).
- Add library to requirements for style-check and fix warning. #71322 (Ilya Yatsishin).
- Fix test
test_workload_entity_keeper_storage
: add more retries. #71325 (Sergei Trifonov). - Debug build faild with clang-18 after https://github.com/ClickHouse/ClickHouse/pull/71266, don't know why it's ok in release build, simply changing _ to _1 is ok for both release and debug build. #71335 (Chang chen).
- Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query. #71336 (Michael Kolupaev).
- CI: Remove deprecated release script. #71341 (Max Kainov).
- Update version_date.tsv and changelog after v24.10.1.2812-stable. #71343 (robot-clickhouse).
- Bump USearch to 2.16.0 and add more tests. #71344 (Robert Schulze).
- check-doc-aspell: Print full path to script in CI report. #71345 (Vladimir Cherkasov).
- CI: Fix fedora version in create release workflow. #71347 (Max Kainov).
- fs cache: add assertions. #71348 (Kseniia Sumarokova).
- More info in TOO_SLOW exception. #71365 (Vladimir Cherkasov).
- Fix unused variables around WorkloadEntityStorageBase. #71367 (Nikita Mikhaylov).
- Allow to prewarm mark cache by system command without enabled setting. #71368 (Anton Popov).
- Fix after https://github.com/ClickHouse/ClickHouse/pull/64847. #71380 (Nikita Mikhaylov).
- Rename
compare8
tocompare16
for consistency. #71416 (Nikita Taranov). - Fix missing cluster startup for test_quorum_inserts::test_insert_quorum_with_keeper_fail. #71418 (Azat Khuzhin).
- [Experiment] Analyzer: Check what happens after if-condition removal. #71425 (Dmitry Novik).
- Update version_date.tsv and changelog after v24.8.6.70-lts. #71428 (robot-clickhouse).
- Fix upgrade check (24.11). #71438 (Raúl Marín).
- Fix assert during insert into vector similarity index in presence of other skipping indexes. #71457 (Robert Schulze).
- Avoid
seconds left [-3]
in cache await. #71468 (Mikhail f. Shiryaev). - Added tests for corner cases for 24.10. #71469 (Max Vostrikov).
- Expose base setting for merge selector. #71497 (alesapin).
- Fixed incorrect settings order
max_parser_depth
andmax_parser_backtracks
. #71498 (Nikita Mikhaylov). - Support the endpoint of oss accelerator. #71502 (Kai Zhu).
- Fix flaky test_drop_complex_columns. #71504 (Ilya Golshtein).
- Move bitShift function changelog entries to backward incompatible. #71510 (Pablo Marcos).
- Fix copy/paste error. #71513 (Denny Crane).
- Allow specifying cmdline flags in integration test. It's needed by #71452 to validate a bugfix. #71523 (Amos Bird).
- Add ProfileEvents for merge selector timings. #71524 (alesapin).
- Minor: Remove "experimental" mention of analyzer. #71525 (Robert Schulze).
- Our builds, jobs, and hosts are called aarch64, so we make the code consistent with the content. #71530 (Mikhail f. Shiryaev).
- Add test to check that accessing system.functions does not populate query_log used_functions. #71535 (Raúl Marín).
- Improve
query_plan_merge_filters
optimization. Fixes #71408. #71539 (Nikolai Kochetov). - Fix compatibility with refreshable materialized views created by old clickhouse servers. #71556 (Michael Kolupaev).
- Vector similarity index: Re-introduce support for legacy index creation syntax. #71572 (Robert Schulze).
- Avoid port clash in CoordinationTest/0.TestSummingRaft1. #71584 (Raúl Marín).
- Fix for
00180_no_seek_avoiding_when_reading_from_cache
. #71596 (Kseniia Sumarokova). - Fix for
test_storage_s3_queue::test_shards_distributed[ordered-2]
. #71597 (Kseniia Sumarokova). - Enable enable_job_stack_trace by default, second attempt. #71625 (Yakov Olkhovskiy).
- Update version_date.tsv and changelog after v24.3.13.40-lts. #71627 (robot-clickhouse).
- Fixes the bug regarding max rows/bytes to read. #71634 (Yarik Briukhovetskyi).
- SimSIMD: Improve suppression for msan false positive. #71635 (Robert Schulze).
- Add
min_parts_to_merge_at_once
merge tree setting which introduces lower limit of amount of data parts to merge at once. The main motivation for this setting is Trifonov's theorem which states that it's not effective to merge less thane
(2.71...) data parts at once because it increases both write amplification and parts number. #71637 (alesapin). - Fix broken 03247_ghdata_string_to_json_alter. #71638 (Pavel Kruglov).
- Update test. #71654 (Kseniia Sumarokova).
- CI: fix mysql containers using improper log directory. #71655 (Yakov Olkhovskiy).
- Update README.md - Update meetups. #71657 (Tanya Bragin).
- Add index granularity size column to system.parts. #71658 (alesapin).
- Update PULL_REQUEST_TEMPLATE.md. #71687 (Alexey Milovidov).
- Historically we have a strange cherry-pick branches naming, e.g.
cherrypick/24.3/5849aeb8c3ca5402f7d8e16e780598c88774371e
. Thecherrypick/24.3/62297
looks nicer and more straightforward. #71698 (Mikhail f. Shiryaev). - Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027. #71715 (Vitaly Baranov).
- Fix flaky test test_prometheus_protocols. #71772 (Vitaly Baranov).
- Fix issues we face on orphane backport branches and closed release PRs, when fake-master events are sent to the check DB. #71782 (Mikhail f. Shiryaev).
- Fix processors profile is not collected for subqueries in some cases. #71787 (Nikita Taranov).
- Lint some stuff. #71795 (Konstantin Bogdanov).
- Backported in #72280: After https://github.com/ClickHouse/ClickHouse/pull/70442 (not included in any release yet) we use new Native serialization for JSON and Dynamic, but it breaks new client - old server communication. This PR fixes it. Also add a setting
merge_tree_use_v1_object_and_dynamic_serialization
that allows to use V1 serialization in newer versions (it can be used during upgrades to new version to be able to rollback). #71816 (Pavel Kruglov). - Closes #71780. #71818 (Kseniia Sumarokova).
- relax memory limit for 00755_avg_value_size_hint_passing.sql. #71820 (Sema Checherinda).
- The change has already been applied to https://github.com/docker-library/official-images/pull/17876. Backport it to every branch to have a proper
Dockerfile.ubuntu
there. #71825 (Mikhail f. Shiryaev). - tests for parseDateTime64InJodaSyntax. #71829 (Max Vostrikov).
- Enable build profiling in pull requests. #71847 (Alexey Milovidov).
- Add check and assertion. #71856 (Kseniia Sumarokova).
- Some healthcheck is better than nothing. #71865 (Nikita Mikhaylov).
- More accurate
calculateCacheKey
implementation. #71868 (Nikita Taranov). - add test 03248_max_parts_to_move. #71869 (Vladimir Cherkasov).
- Update README.md - Update meetups. #71872 (Tanya Bragin).
- Prevents listing files from s3 while inserting. #71889 (Yarik Briukhovetskyi).
- Keep query_plan_merge_filters disabled by default. #71890 (Raúl Marín).
- Remove useless code. #71900 (Alexey Milovidov).
- TreeRewriter: fix typo:
parititon
=>partition
. #71907 (yun). - Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715. #71912 (Vitaly Baranov).
- Fix test_storage_mongodb/test.py::test_secure_connection_uri. #71924 (Vladimir Cherkasov).
- Adapt some test to run in non-CI configurations. #71928 (Raúl Marín).
- Fix build after #71179. Clang-19 checks for unused variables inside of if-conditions. #71929 (Dmitry Novik).
- Fix flaky test 03262_column_sizes_with_dynamic_structure. #71931 (Pavel Kruglov).
- Don't randomise settings in 02354_distributed_with_external_aggregation_memory_usage. #71944 (Nikita Taranov).
- Enabling
query_plan_merge_filters
again after #71890. #71964 (Nikolai Kochetov). - Add comment about bf16 to CMake docs. #71973 (Robert Schulze).
- Fix 00098_primary_key_memory_allocated. #71977 (Alexander Gololobov).
- Add a test for #71908. #71986 (Alexey Milovidov).
- Miscellaneous. #71987 (Alexey Milovidov).
- Minor follow-up to #71581. #71993 (Robert Schulze).
- CI: Stress test with praktika. #71995 (Max Kainov).
- Fix prewarm of mark cache after adding a new column. #71996 (Anton Popov).
- Follow-up to https://github.com/ClickHouse/ClickHouse/pull/70027 and https://github.com/ClickHouse/ClickHouse/pull/71715 and https://github.com/ClickHouse/ClickHouse/pull/71912. #72018 (Vitaly Baranov).
- Fix comments. #72023 (Alexey Milovidov).
- Add test for 33604. #72026 (Nikita Taranov).
- CI: Remove unsafe secret_envs input from yml workflows. #72028 (Max Kainov).
- Update version_date.tsv and changelog after v24.10.2.80-stable. #72029 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.8.7.41-lts. #72037 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.9.3.128-stable. #72041 (robot-clickhouse).
- Update version_date.tsv and changelog after v24.3.14.35-lts. #72042 (robot-clickhouse).
- Save several minutes of build time. #72046 (Raúl Marín).
- Update README.md - Update meetups. #72048 (Tanya Bragin).
- Fix client syntax highlighting that was broken in https://github.com/ClickHouse/ClickHouse/pull/71949. #72049 (Nikolay Degterinsky).
- Fix test_disk_over_web_server/. #72075 (Kseniia Sumarokova).
- Minor improvement for system.query_metric_log stateless test. #72076 (Pablo Marcos).
- A follow-up for #72057 and https://github.com/ClickHouse/ClickHouse/pull/71505. #72079 (Mikhail f. Shiryaev).
- Add google-cloud-cpp submodule. #72092 (Pablo Marcos).
- CI: Enable fuzzer job in Nightly workflow. #72101 (Max Kainov).
- Get rid of code duplication after adding
CHECK GRANT
in https://github.com/ClickHouse/ClickHouse/pull/68885. #72103 (Vitaly Baranov). - Add jwt-cpp submodule. #72104 (Pablo Marcos).
- Fix list-licenses.sh with OSX. #72107 (Raúl Marín).
- fix cancelation for PartitionedSink. #72126 (Sema Checherinda).
- FIx 02374_analyzer_join_using. #72145 (Vladimir Cherkasov).
- Fixed a test which was flaky-flaky. #72147 (Nikita Mikhaylov).
- Backported in #72236: Revert "CI: Stress test with praktika". #72231 (Max Kainov).
- Backported in #72345: Apply colors correctly to terminal output. #72283 (Antonio Andelic).
- Backported in #72361: disable a cloud setting. #72292 (Han Fei).
- Backported in #72469: Remove flaky test test_move_shared_lock_fail_keeper_unavailable and extend the stable one. #72357 (Aleksei Filatov).
- Backported in #72463: Fix data race in Squashing with LowCardinality. #72392 (Vladimir Cherkasov).
- Backported in #72453: make operations_to_execute as shared ptr. #72400 (Sema Checherinda).