From 7ac453ab4c2f7162c5dc25f29eaf396b670357a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 23:06:14 +0800 Subject: [PATCH 001/154] Revert "Merge pull request #60436 from nickitat/revert_56864" This reverts commit 8719a601fac863a2c484bcf97339aecdf9e73c5f, reversing changes made to 657857f9828eb46867197c6f7bc8c2444ab1cc3d. --- src/Backups/BackupSettings.cpp | 2 + src/Backups/BackupSettings.h | 6 + src/Common/ErrorCodes.cpp | 1 + src/Interpreters/MutationsInterpreter.cpp | 14 +- src/Interpreters/MutationsInterpreter.h | 1 + .../Optimizations/projectionsCommon.cpp | 2 +- .../MergeTree/DataPartStorageOnDiskBase.cpp | 35 +- .../MergeTree/DataPartStorageOnDiskBase.h | 4 +- src/Storages/MergeTree/IDataPartStorage.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 62 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 17 +- src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 37 +- src/Storages/MergeTree/MergeTreeData.h | 9 +- .../MergeTree/MergeTreeDataPartChecksum.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 14 +- .../ReplicatedMergeTreePartCheckThread.cpp | 38 +- .../ReplicatedMergeTreePartCheckThread.h | 4 +- src/Storages/MergeTree/checkDataPart.cpp | 71 ++- src/Storages/MergeTree/checkDataPart.h | 4 +- src/Storages/StorageMergeTree.cpp | 5 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- .../System/StorageSystemProjectionParts.cpp | 34 +- .../test_broken_projections/__init__.py | 0 .../config.d/backups.xml | 13 + .../test_broken_projections/test.py | 576 ++++++++++++++++++ .../02117_show_create_table_system.reference | 3 + 27 files changed, 907 insertions(+), 59 deletions(-) create mode 100644 tests/integration/test_broken_projections/__init__.py create mode 100644 tests/integration/test_broken_projections/config.d/backups.xml create mode 100644 tests/integration/test_broken_projections/test.py diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 68d825e9468..51d713f03e1 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -32,6 +32,8 @@ namespace ErrorCodes M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, check_parts) \ + M(Bool, check_projection_parts) \ + M(Bool, allow_backup_broken_projections) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, backup_uuid) diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index f26b992b348..ec430905f51 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -62,6 +62,12 @@ struct BackupSettings /// Check checksums of the data parts before writing them to a backup. bool check_parts = true; + /// Check checksums of the projection data parts before writing them to a backup. + bool check_projection_parts = true; + + /// Allow to create backup with broken projections. + bool allow_backup_broken_projections = false; + /// Internal, should not be specified by user. /// Whether this backup is a part of a distributed backup created by BACKUP ON CLUSTER. bool internal = false; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ca00f2fd513..1ce8997e928 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -592,6 +592,7 @@ M(710, FAULT_INJECTED) \ M(711, FILECACHE_ACCESS_DENIED) \ M(712, TOO_MANY_MATERIALIZED_VIEWS) \ + M(713, BROKEN_PROJECTION) \ M(714, UNEXPECTED_CLUSTER) \ M(715, CANNOT_DETECT_FORMAT) \ M(716, CANNOT_FORGET_PARTITION) \ diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index a3d1b84fdc1..502b961ced8 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -342,6 +342,11 @@ bool MutationsInterpreter::Source::hasProjection(const String & name) const return part && part->hasProjection(name); } +bool MutationsInterpreter::Source::hasBrokenProjection(const String & name) const +{ + return part && part->hasBrokenProjection(name); +} + bool MutationsInterpreter::Source::isCompactPart() const { return part && part->getType() == MergeTreeDataPartType::Compact; @@ -807,7 +812,7 @@ void MutationsInterpreter::prepare(bool dry_run) { mutation_kind.set(MutationKind::MUTATE_INDEX_STATISTIC_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); - if (!source.hasProjection(projection.name)) + if (!source.hasProjection(projection.name) || source.hasBrokenProjection(projection.name)) { for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -994,6 +999,13 @@ void MutationsInterpreter::prepare(bool dry_run) if (!source.hasProjection(projection.name)) continue; + /// Always rebuild broken projections. + if (source.hasBrokenProjection(projection.name)) + { + materialized_projections.insert(projection.name); + continue; + } + if (need_rebuild_projections) { materialized_projections.insert(projection.name); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index eda94190185..4c35ec34b58 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -126,6 +126,7 @@ public: bool materializeTTLRecalculateOnly() const; bool hasSecondaryIndex(const String & name) const; bool hasProjection(const String & name) const; + bool hasBrokenProjection(const String & name) const; bool isCompactPart() const; void read( diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index 8333f5e857b..3009460a468 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -223,7 +223,7 @@ bool analyzeProjectionCandidate( { const auto & created_projections = part_with_ranges.data_part->getProjectionParts(); auto it = created_projections.find(candidate.projection->name); - if (it != created_projections.end()) + if (it != created_projections.end() && !it->second->is_broken) { projection_parts.push_back(it->second); } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 5210d14f3d0..e31d991ef09 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -335,7 +335,9 @@ void DataPartStorageOnDiskBase::backup( const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, - TemporaryFilesOnDisks * temp_dirs) const + TemporaryFilesOnDisks * temp_dirs, + bool is_projection_part, + bool allow_backup_broken_projection) const { fs::path part_path_on_disk = fs::path{root_path} / part_dir; fs::path part_path_in_backup = fs::path{path_in_backup} / part_dir; @@ -377,7 +379,7 @@ void DataPartStorageOnDiskBase::backup( bool copy_encrypted = !backup_settings.decrypt_files_from_encrypted_disks; - for (const auto & filepath : files_to_backup) + auto backup_file = [&](const String & filepath) { auto filepath_on_disk = part_path_on_disk / filepath; auto filepath_in_backup = part_path_in_backup / filepath; @@ -385,8 +387,10 @@ void DataPartStorageOnDiskBase::backup( if (files_without_checksums.contains(filepath)) { backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk, read_settings, copy_encrypted)); - continue; + return; } + else if (is_projection_part && allow_backup_broken_projection && !disk->exists(filepath_on_disk)) + return; if (make_temporary_hard_links) { @@ -411,6 +415,31 @@ void DataPartStorageOnDiskBase::backup( backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner); backup_entries.emplace_back(filepath_in_backup, std::move(backup_entry)); + }; + + auto * log = &Poco::Logger::get("DataPartStorageOnDiskBase::backup"); + + for (const auto & filepath : files_to_backup) + { + if (is_projection_part && allow_backup_broken_projection) + { + try + { + backup_file(filepath); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::FILE_DOESNT_EXIST) + throw; + + LOG_ERROR(log, "Cannot backup file {} of projection part {}. Will try to ignore it", filepath, part_dir); + continue; + } + } + else + { + backup_file(filepath); + } } } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 52dc850c7fd..75bf3d6f93c 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -58,7 +58,9 @@ public: const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, - TemporaryFilesOnDisks * temp_dirs) const override; + TemporaryFilesOnDisks * temp_dirs, + bool is_projection_part, + bool allow_backup_broken_projection) const override; MutableDataPartStoragePtr freeze( const std::string & to, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 5899ef58cd5..d06d9791a53 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -223,7 +223,9 @@ public: const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, - TemporaryFilesOnDisks * temp_dirs) const = 0; + TemporaryFilesOnDisks * temp_dirs, + bool is_projection_part, + bool allow_backup_broken_projection) const = 0; /// Creates hardlinks into 'to/dir_path' for every file in data part. /// Callback is called after hardlinks are created, but before 'delete-on-destroy.txt' marker is removed. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index e06ea5e560c..11ede661f78 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -699,13 +699,14 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks calculateColumnsAndSecondaryIndicesSizesOnDisk(); loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`. loadPartitionAndMinMaxIndex(); + bool has_broken_projections = false; if (!parent_part) { loadTTLInfos(); - loadProjections(require_columns_checksums, check_consistency, false /* if_not_loaded */); + loadProjections(require_columns_checksums, check_consistency, has_broken_projections, false /* if_not_loaded */); } - if (check_consistency) + if (check_consistency && !has_broken_projections) checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); @@ -770,7 +771,7 @@ void IMergeTreeDataPart::addProjectionPart( projection_parts[projection_name] = std::move(projection_part); } -void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded) +void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency, bool & has_broken_projection, bool if_not_loaded) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); for (const auto & projection : metadata_snapshot->projections) @@ -787,10 +788,34 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch else { auto part = getProjectionPartBuilder(projection.name).withPartFormatFromDisk().build(); - part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); + + try + { + part->loadColumnsChecksumsIndexes(require_columns_checksums, check_consistency); + } + catch (...) + { + if (isRetryableException(std::current_exception())) + throw; + + auto message = getCurrentExceptionMessage(true); + LOG_ERROR(&Poco::Logger::get("IMergeTreeDataPart"), + "Cannot load projection {}, will consider it broken. Reason: {}", projection.name, message); + + has_broken_projection = true; + part->setBrokenReason(message, getCurrentExceptionCode()); + } + addProjectionPart(projection.name, std::move(part)); } } + else if (checksums.has(path)) + { + auto part = getProjectionPartBuilder(projection.name).withPartFormatFromDisk().build(); + part->setBrokenReason("Projection directory " + path + " does not exist while loading projections", ErrorCodes::NO_FILE_IN_DATA_PART); + addProjectionPart(projection.name, std::move(part)); + has_broken_projection = true; + } } } @@ -1189,7 +1214,8 @@ void IMergeTreeDataPart::loadChecksums(bool require) /// Check the data while we are at it. LOG_WARNING(storage.log, "Checksums for part {} not found. Will calculate them from data on disk.", name); - checksums = checkDataPart(shared_from_this(), false); + bool noop; + checksums = checkDataPart(shared_from_this(), false, noop, /* is_cancelled */[]{ return false; }, /* throw_on_broken_projection */false); writeChecksums(checksums, {}); bytes_on_disk = checksums.getTotalSizeOnDisk(); @@ -2196,6 +2222,32 @@ std::optional IMergeTreeDataPart::getStreamNameForColumn( return getStreamNameOrHash(stream_name, extension, storage_); } +void IMergeTreeDataPart::markProjectionPartAsBroken(const String & projection_name, const String & message, int code) const +{ + auto it = projection_parts.find(projection_name); + if (it == projection_parts.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no projection part '{}'", projection_name); + it->second->setBrokenReason(message, code); +} + +bool IMergeTreeDataPart::hasBrokenProjection(const String & projection_name) const +{ + auto it = projection_parts.find(projection_name); + if (it == projection_parts.end()) + return false; + return it->second->is_broken; +} + +void IMergeTreeDataPart::setBrokenReason(const String & message, int code) const +{ + std::lock_guard lock(broken_reason_mutex); + if (is_broken) + return; + is_broken = true; + exception = message; + exception_code = code; +} + bool isCompactPart(const MergeTreeDataPartPtr & data_part) { return (data_part && data_part->getType() == MergeTreeDataPartType::Compact); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 91c559d30c8..0d7acfab891 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -259,6 +259,12 @@ public: /// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table. mutable std::atomic is_frozen {false}; + /// If it is a projection part, it can be broken sometimes. + mutable std::atomic is_broken {false}; + mutable std::string exception; + mutable int exception_code = 0; + mutable std::mutex broken_reason_mutex; + /// Indicates that the part was marked Outdated by PartCheckThread because the part was not committed to ZooKeeper mutable bool is_unexpected_local_part = false; @@ -418,9 +424,16 @@ public: void addProjectionPart(const String & projection_name, std::shared_ptr && projection_part); + void markProjectionPartAsBroken(const String & projection_name, const String & message, int code) const; + bool hasProjection(const String & projection_name) const { return projection_parts.contains(projection_name); } - void loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); + bool hasBrokenProjection(const String & projection_name) const; + + /// Return true, if all projections were loaded successfully and none was marked as broken. + void loadProjections(bool require_columns_checksums, bool check_consistency, bool & has_broken_projection, bool if_not_loaded = false); + + void setBrokenReason(const String & message, int code) const; /// Return set of metadata file names without checksums. For example, /// columns.txt or checksums.txt itself. @@ -580,7 +593,7 @@ protected: const IMergeTreeDataPart * parent_part; String parent_part_name; - std::map> projection_parts; + mutable std::map> projection_parts; mutable PartMetadataManagerPtr metadata_manager; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index df64ae33713..e6ae63da7e3 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -731,8 +731,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c MergeTreeData::DataPartsVector projection_parts; for (const auto & part : global_ctx->future_part->parts) { - auto it = part->getProjectionParts().find(projection.name); - if (it != part->getProjectionParts().end()) + auto actual_projection_parts = part->getProjectionParts(); + auto it = actual_projection_parts.find(projection.name); + if (it != actual_projection_parts.end() && !it->second->is_broken) projection_parts.push_back(it->second); } if (projection_parts.size() < global_ctx->future_part->parts.size()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6d5e486f6a1..babc593ff62 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5311,7 +5311,7 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( if (hold_table_lock && !table_lock) table_lock = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); - if (backup_settings.check_parts) + if (backup_settings.check_projection_parts) part->checkConsistencyWithProjections(/* require_part_metadata= */ true); BackupEntries backup_entries_from_part; @@ -5323,7 +5323,8 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( read_settings, make_temporary_hard_links, backup_entries_from_part, - &temp_dirs); + &temp_dirs, + false, false); auto projection_parts = part->getProjectionParts(); for (const auto & [projection_name, projection_part] : projection_parts) @@ -5336,7 +5337,9 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( read_settings, make_temporary_hard_links, backup_entries_from_part, - &temp_dirs); + &temp_dirs, + projection_part->is_broken, + backup_settings.allow_backup_broken_projections); } if (hold_storage_and_part_ptrs) @@ -7825,21 +7828,39 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, String & out_reason) { - if (left->getProjectionParts().size() != right->getProjectionParts().size()) + auto remove_broken_parts_from_consideration = [](auto & parts) + { + std::set broken_projection_parts; + for (const auto & [name, part] : parts) + { + if (part->is_broken) + broken_projection_parts.emplace(name); + } + for (const auto & name : broken_projection_parts) + parts.erase(name); + }; + + auto left_projection_parts = left->getProjectionParts(); + auto right_projection_parts = right->getProjectionParts(); + + remove_broken_parts_from_consideration(left_projection_parts); + remove_broken_parts_from_consideration(right_projection_parts); + + if (left_projection_parts.size() != right_projection_parts.size()) { out_reason = fmt::format( "Parts have different number of projections: {} in part '{}' and {} in part '{}'", - left->getProjectionParts().size(), + left_projection_parts.size(), left->name, - right->getProjectionParts().size(), + right_projection_parts.size(), right->name ); return false; } - for (const auto & [name, _] : left->getProjectionParts()) + for (const auto & [name, _] : left_projection_parts) { - if (!right->hasProjection(name)) + if (!right_projection_parts.contains(name)) { out_reason = fmt::format( "The part '{}' doesn't have projection '{}' while part '{}' does", right->name, name, left->name diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ab265715688..c638505604f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -468,8 +468,13 @@ public: struct ProjectionPartsVector { - DataPartsVector projection_parts; DataPartsVector data_parts; + + DataPartsVector projection_parts; + DataPartStateVector projection_parts_states; + + DataPartsVector broken_projection_parts; + DataPartStateVector broken_projection_parts_states; }; /// Returns a copy of the list so that the caller shouldn't worry about locks. @@ -484,7 +489,7 @@ public: const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const; /// Same as above but only returns projection parts ProjectionPartsVector getProjectionPartsVectorForInternalUsage( - const DataPartStates & affordable_states, DataPartStateVector * out_states = nullptr) const; + const DataPartStates & affordable_states, MergeTreeData::DataPartStateVector * out_states) const; /// Returns absolutely all parts (and snapshot of their states) diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h index 837b940e354..d4980a67a43 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h @@ -54,6 +54,8 @@ struct MergeTreeDataPartChecksums bool has(const String & file_name) const { return files.find(file_name) != files.end(); } + bool remove(const String & file_name) { return files.erase(file_name); } + bool empty() const { return files.empty(); } /// Checks that the set of columns and their checksums are the same. If not, throws an exception. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 1c33f018a5d..6bacce9e2c5 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -573,7 +573,9 @@ static std::set getProjectionsToRecalculate( { bool need_recalculate = materialized_projections.contains(projection.name) - || (!is_full_part_storage && source_part->hasProjection(projection.name)); + || (!is_full_part_storage + && source_part->hasProjection(projection.name) + && !source_part->hasBrokenProjection(projection.name)); if (need_recalculate) projections_to_recalc.insert(&projection); @@ -917,7 +919,8 @@ void finalizeMutatedPart( new_data_part->modification_time = time(nullptr); /// Load rest projections which are hardlinked - new_data_part->loadProjections(false, false, true /* if_not_loaded */); + bool noop; + new_data_part->loadProjections(false, false, noop, true /* if_not_loaded */); /// All information about sizes is stored in checksums. /// It doesn't make sense to touch filesystem for sizes. @@ -1500,7 +1503,9 @@ private: bool need_recalculate = ctx->materialized_projections.contains(projection.name) - || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name)); + || (!is_full_part_storage + && ctx->source_part->hasProjection(projection.name) + && !ctx->source_part->hasBrokenProjection(projection.name)); if (need_recalculate) { @@ -1637,8 +1642,9 @@ private: void finalize() { + bool noop; ctx->new_data_part->minmax_idx = std::move(ctx->minmax_idx); - ctx->new_data_part->loadProjections(false, false, true /* if_not_loaded */); + ctx->new_data_part->loadProjections(false, false, noop, true /* if_not_loaded */); ctx->mutating_executor.reset(); ctx->mutating_pipeline.reset(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 156c41563ec..bc0b4f73a31 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -63,7 +63,7 @@ void ReplicatedMergeTreePartCheckThread::enqueuePart(const String & name, time_t if (parts_set.contains(name)) return; - LOG_TRACE(log, "Enqueueing {} for check after after {}s", name, delay_to_check_seconds); + LOG_TRACE(log, "Enqueueing {} for check after {}s", name, delay_to_check_seconds); parts_queue.emplace_back(name, std::chrono::steady_clock::now() + std::chrono::seconds(delay_to_check_seconds)); parts_set.insert(name); task->schedule(); @@ -274,7 +274,7 @@ std::pair ReplicatedMergeTreePartCheckThread::findLo return std::make_pair(exists_in_zookeeper, part); } -ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const String & part_name) +ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const String & part_name, bool throw_on_broken_projection) { ReplicatedCheckResult result; auto [exists_in_zookeeper, part] = findLocalPart(part_name); @@ -341,6 +341,7 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St /// before the ReplicatedMergeTreePartHeader was introduced. String part_path = storage.replica_path + "/parts/" + part_name; String part_znode = zookeeper->get(part_path); + bool is_broken_projection = false; try { @@ -362,8 +363,10 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St checkDataPart( part, - true, - [this] { return need_stop.load(); }); + /* require_checksums */true, + is_broken_projection, + [this] { return need_stop.load(); }, + throw_on_broken_projection); if (need_stop) { @@ -382,14 +385,27 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St if (isRetryableException(std::current_exception())) throw; - tryLogCurrentException(log, __PRETTY_FUNCTION__); + PreformattedMessage message; + if (is_broken_projection) + { + WriteBufferFromOwnString wb; + message = PreformattedMessage::create( + "Part {} has a broken projections. It will be ignored. Broken projections info: {}", + part_name, getCurrentExceptionMessage(false)); + LOG_DEBUG(log, message); + result.action = ReplicatedCheckResult::DoNothing; + } + else + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); - auto message = PreformattedMessage::create("Part {} looks broken. Removing it and will try to fetch.", part_name); - LOG_ERROR(log, message); + message = PreformattedMessage::create("Part {} looks broken. Removing it and will try to fetch.", part_name); + LOG_ERROR(log, message); + result.action = ReplicatedCheckResult::TryFetchMissing; + } /// Part is broken, let's try to find it and fetch. result.status = {part_name, false, message}; - result.action = ReplicatedCheckResult::TryFetchMissing; return result; } @@ -419,12 +435,12 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St } -CheckResult ReplicatedMergeTreePartCheckThread::checkPartAndFix(const String & part_name, std::optional * recheck_after) +CheckResult ReplicatedMergeTreePartCheckThread::checkPartAndFix(const String & part_name, std::optional * recheck_after, bool throw_on_broken_projection) { LOG_INFO(log, "Checking part {}", part_name); ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks); - ReplicatedCheckResult result = checkPartImpl(part_name); + ReplicatedCheckResult result = checkPartImpl(part_name, throw_on_broken_projection); switch (result.action) { case ReplicatedCheckResult::None: UNREACHABLE(); @@ -577,7 +593,7 @@ void ReplicatedMergeTreePartCheckThread::run() } std::optional recheck_after; - checkPartAndFix(selected->name, &recheck_after); + checkPartAndFix(selected->name, &recheck_after, /* throw_on_broken_projection */false); if (need_stop) return; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index f2e26b3d324..9091f698546 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -65,9 +65,9 @@ public: size_t size() const; /// Check part by name - CheckResult checkPartAndFix(const String & part_name, std::optional * recheck_after = nullptr); + CheckResult checkPartAndFix(const String & part_name, std::optional * recheck_after = nullptr, bool throw_on_broken_projection = true); - ReplicatedCheckResult checkPartImpl(const String & part_name); + ReplicatedCheckResult checkPartImpl(const String & part_name, bool throw_on_broken_projection); std::unique_lock pausePartsCheck(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8ae9b54b6e9..0b545beb116 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -43,6 +43,7 @@ namespace ErrorCodes extern const int NO_FILE_IN_DATA_PART; extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; + extern const int BROKEN_PROJECTION; } @@ -117,7 +118,9 @@ static IMergeTreeDataPart::Checksums checkDataPart( const NameSet & files_without_checksums, const ReadSettings & read_settings, bool require_checksums, - std::function is_cancelled) + std::function is_cancelled, + bool & is_broken_projection, + bool throw_on_broken_projection) { /** Responsibility: * - read list of columns from columns.txt; @@ -126,6 +129,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( */ CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedChecks}; + Poco::Logger * log = &Poco::Logger::get("checkDataPart"); NamesAndTypesList columns_txt; @@ -275,17 +279,55 @@ static IMergeTreeDataPart::Checksums checkDataPart( } } + std::string broken_projections_message; for (const auto & [name, projection] : data_part->getProjectionParts()) { if (is_cancelled()) return {}; auto projection_file = name + ".proj"; - auto projection_checksums = checkDataPart( - projection, *data_part_storage.getProjection(projection_file), - projection->getColumns(), projection->getType(), - projection->getFileNamesWithoutChecksums(), - read_settings, require_checksums, is_cancelled); + if (!throw_on_broken_projection && projection->is_broken) + { + projections_on_disk.erase(projection_file); + checksums_txt.remove(projection_file); + } + + IMergeTreeDataPart::Checksums projection_checksums; + try + { + bool noop; + projection_checksums = checkDataPart( + projection, *data_part_storage.getProjection(projection_file), + projection->getColumns(), projection->getType(), + projection->getFileNamesWithoutChecksums(), + read_settings, require_checksums, is_cancelled, noop, /* throw_on_broken_projection */false); + } + catch (...) + { + if (isRetryableException(std::current_exception())) + throw; + + if (!projection->is_broken) + { + LOG_TEST(log, "Marking projection {} as broken ({})", name, projection_file); + projection->setBrokenReason(getCurrentExceptionMessage(false), getCurrentExceptionCode()); + } + + is_broken_projection = true; + if (throw_on_broken_projection) + { + if (!broken_projections_message.empty()) + broken_projections_message += "\n"; + + broken_projections_message += fmt::format( + "Part {} has a broken projection {} (error: {})", + data_part->name, name, getCurrentExceptionMessage(false)); + continue; + } + + projections_on_disk.erase(projection_file); + checksums_txt.remove(projection_file); + } checksums_data.files[projection_file] = IMergeTreeDataPart::Checksums::Checksum( projection_checksums.getTotalSizeOnDisk(), @@ -294,6 +336,11 @@ static IMergeTreeDataPart::Checksums checkDataPart( projections_on_disk.erase(projection_file); } + if (throw_on_broken_projection && !broken_projections_message.empty()) + { + throw Exception(ErrorCodes::BROKEN_PROJECTION, "{}", broken_projections_message); + } + if (require_checksums && !projections_on_disk.empty()) { throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, @@ -321,7 +368,9 @@ IMergeTreeDataPart::Checksums checkDataPartInMemory(const DataPartInMemoryPtr & IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, bool require_checksums, - std::function is_cancelled) + bool & is_broken_projection, + std::function is_cancelled, + bool throw_on_broken_projection) { if (auto part_in_memory = asInMemoryPart(data_part)) return checkDataPartInMemory(part_in_memory); @@ -363,7 +412,9 @@ IMergeTreeDataPart::Checksums checkDataPart( data_part->getFileNamesWithoutChecksums(), read_settings, require_checksums, - is_cancelled); + is_cancelled, + is_broken_projection, + throw_on_broken_projection); }; try @@ -377,7 +428,9 @@ IMergeTreeDataPart::Checksums checkDataPart( data_part->getFileNamesWithoutChecksums(), read_settings, require_checksums, - is_cancelled); + is_cancelled, + is_broken_projection, + throw_on_broken_projection); } catch (...) { diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index d0e48b6f80a..a01978f4efe 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -10,7 +10,9 @@ namespace DB IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, bool require_checksums, - std::function is_cancelled = []{ return false; }); + bool & is_broken_projection, + std::function is_cancelled = []{ return false; }, + bool throw_on_broken_projection = false); bool isNotEnoughMemoryErrorCode(int code); bool isRetryableException(const std::exception_ptr exception_ptr); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e15b308f084..d94a4ff14c0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2315,11 +2315,12 @@ std::optional StorageMergeTree::checkDataNext(DataValidationTasksPt { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; + bool noop; if (part->isStoredOnDisk() && !part->getDataPartStorage().exists(checksums_path)) { try { - auto calculated_checksums = checkDataPart(part, false); + auto calculated_checksums = checkDataPart(part, false, noop, /* is_cancelled */[]{ return false; }, /* throw_on_broken_projection */true); calculated_checksums.checkEqual(part->checksums, true); auto & part_mutable = const_cast(*part); @@ -2340,7 +2341,7 @@ std::optional StorageMergeTree::checkDataNext(DataValidationTasksPt { try { - checkDataPart(part, true); + checkDataPart(part, true, noop, /* is_cancelled */[]{ return false; }, /* throw_on_broken_projection */true); return CheckResult(part->name, true, ""); } catch (...) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1702b52fa35..ee15a26f244 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8891,12 +8891,11 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList( std::optional StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list) { - if (auto part = assert_cast(check_task_list.get())->next()) { try { - return CheckResult(part_check_thread.checkPartAndFix(part->name)); + return part_check_thread.checkPartAndFix(part->name, /* recheck_after */nullptr, /* throw_on_broken_projection */true); } catch (const Exception & ex) { diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 016705f4e66..b1494f2ba98 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -83,7 +83,11 @@ StorageSystemProjectionParts::StorageSystemProjectionParts(const StorageID & tab {"rows_where_ttl_info.expression", std::make_shared(std::make_shared())}, {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, - {"rows_where_ttl_info.max", std::make_shared(std::make_shared())} + {"rows_where_ttl_info.max", std::make_shared(std::make_shared())}, + + {"is_broken", std::make_shared()}, + {"exception_code", std::make_shared()}, + {"exception", std::make_shared()}, } ) { @@ -272,12 +276,38 @@ void StorageSystemProjectionParts::processNextStorage( add_ttl_info_map(part->ttl_infos.moves_ttl); if (columns_mask[src_index++]) - columns[res_index++]->insert(queryToString(part->default_codec->getCodecDesc())); + { + if (part->default_codec) + columns[res_index++]->insert(queryToString(part->default_codec->getCodecDesc())); + else + columns[res_index++]->insertDefault(); + } add_ttl_info_map(part->ttl_infos.recompression_ttl); add_ttl_info_map(part->ttl_infos.group_by_ttl); add_ttl_info_map(part->ttl_infos.rows_where_ttl); + { + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->is_broken.load(std::memory_order_relaxed)); + + if (part->is_broken) + { + std::lock_guard lock(part->broken_reason_mutex); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->exception_code); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->exception); + } + else + { + if (columns_mask[src_index++]) + columns[res_index++]->insertDefault(); + if (columns_mask[src_index++]) + columns[res_index++]->insertDefault(); + } + } + /// _state column should be the latest. /// Do not use part->getState*, it can be changed from different thread if (has_state_column) diff --git a/tests/integration/test_broken_projections/__init__.py b/tests/integration/test_broken_projections/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_broken_projections/config.d/backups.xml b/tests/integration/test_broken_projections/config.d/backups.xml new file mode 100644 index 00000000000..4da8edffd67 --- /dev/null +++ b/tests/integration/test_broken_projections/config.d/backups.xml @@ -0,0 +1,13 @@ + + + + + local + /var/lib/clickhouse/disks/backups/ + + + + + backups + + diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py new file mode 100644 index 00000000000..4a4690a5d0a --- /dev/null +++ b/tests/integration/test_broken_projections/test.py @@ -0,0 +1,576 @@ +import time +import pytest +import logging +import string +import random +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=["config.d/backups.xml"], + stay_alive=True, + with_zookeeper=True, + ) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def create_table(node, table, replica, data_prefix="", aggressive_merge=True): + if data_prefix == "": + data_prefix = table + + if aggressive_merge: + vertical_merge_algorithm_min_rows_to_activate = 1 + vertical_merge_algorithm_min_columns_to_activate = 1 + max_parts_to_merge_at_once = 3 + else: + vertical_merge_algorithm_min_rows_to_activate = 100000 + vertical_merge_algorithm_min_columns_to_activate = 100 + max_parts_to_merge_at_once = 3 + + node.query( + f""" + DROP TABLE IF EXISTS {table} SYNC; + CREATE TABLE {table} + ( + a String, + b String, + c Int64, + d Int64, + e Int64, + PROJECTION proj1 + ( + SELECT c ORDER BY d + ), + PROJECTION proj2 + ( + SELECT d ORDER BY c + ) + ) + ENGINE = ReplicatedMergeTree('/test_broken_projection_{data_prefix}/data/', '{replica}') ORDER BY a + SETTINGS min_bytes_for_wide_part = 0, + max_parts_to_merge_at_once={max_parts_to_merge_at_once}, + enable_vertical_merge_algorithm=0, + vertical_merge_algorithm_min_rows_to_activate = {vertical_merge_algorithm_min_rows_to_activate}, + vertical_merge_algorithm_min_columns_to_activate = {vertical_merge_algorithm_min_columns_to_activate}, + compress_primary_key=0; + """ + ) + + +def insert(node, table, offset, size): + node.query( + f""" + INSERT INTO {table} + SELECT number, number, number, number, number%2 FROM numbers({offset}, {size}) + SETTINGS insert_keeper_fault_injection_probability=0.0; + """ + ) + + +def get_parts(node, table): + return ( + node.query( + f""" + SELECT name + FROM system.parts + WHERE table='{table}' AND database=currentDatabase() AND active = 1 + ORDER BY name;" + """ + ) + .strip() + .split("\n") + ) + + +def bash(node, command): + node.exec_in_container(["bash", "-c", command], privileged=True, user="root") + + +def break_projection(node, table, part, parent_part, break_type): + part_path = node.query( + f""" + SELECT path + FROM system.projection_parts + WHERE table='{table}' + AND database=currentDatabase() + AND active=1 + AND part_name='{part}' + AND parent_name='{parent_part}' + ORDER BY modification_time DESC + LIMIT 1; + """ + ).strip() + + node.query( + f"select throwIf(substring('{part_path}', 1, 1) != '/', 'Path is relative: {part_path}')" + ) + + if break_type == "data": + bash(node, f"rm '{part_path}/d.bin'") + bash(node, f"rm '{part_path}/c.bin'") + elif break_type == "metadata": + bash(node, f"rm '{part_path}/columns.txt'") + elif break_type == "part": + bash(node, f"rm -r '{part_path}'") + + +def break_part(node, table, part): + part_path = node.query( + f""" + SELECT path + FROM system.parts + WHERE table='{table}' + AND database=currentDatabase() + AND active=1 + AND part_name='{part}' + ORDER BY modification_time DESC + LIMIT 1; + """ + ).strip() + + node.query( + f"select throwIf(substring('{part_path}', 1, 1) != '/', 'Path is relative: {part_path}')" + ) + bash(node, f"rm '{part_path}/columns.txt'") + + +def get_broken_projections_info(node, table): + return node.query( + f""" + SELECT parent_name, name, errors.name FROM + ( + SELECT parent_name, name, exception_code + FROM system.projection_parts + WHERE table='{table}' + AND database=currentDatabase() + AND is_broken = 1 + ) AS parts_info + INNER JOIN system.errors AS errors + ON parts_info.exception_code = errors.code + ORDER BY parent_name, name + """ + ).strip() + + +def get_projections_info(node, table): + return node.query( + f""" + SELECT parent_name, name, is_broken + FROM system.projection_parts + WHERE table='{table}' + AND active = 1 + AND database=currentDatabase() + ORDER BY parent_name, name + """ + ).strip() + + +def optimize(node, table, final, no_wait): + query = f"OPTIMIZE TABLE {table}" + if final: + query += " FINAL" + if no_wait: + query += " SETTINGS alter_sync=0" + node.query(query) + + +def reattach(node, table): + node.query( + f""" + DETACH TABLE {table}; + ATTACH TABLE {table}; + """ + ) + + +def materialize_projection(node, table, proj): + node.query( + f"ALTER TABLE {table} MATERIALIZE PROJECTION {proj} SETTINGS mutations_sync=2" + ) + + +def check_table_full(node, table): + return node.query( + f"CHECK TABLE {table} SETTINGS check_query_single_value_result = 0;" + ).strip() + + +def random_str(length=6): + alphabet = string.ascii_lowercase + string.digits + return "".join(random.SystemRandom().choice(alphabet) for _ in range(length)) + + +def check(node, table, check_result, expect_broken_part="", expected_error=""): + if expect_broken_part == "proj1": + assert expected_error in node.query_and_get_error( + f"SELECT c FROM '{table}' WHERE d == 12 ORDER BY c" + ) + else: + query_id = node.query( + f"SELECT queryID() FROM (SELECT c FROM '{table}' WHERE d == 12 ORDER BY c)" + ).strip() + node.query("SYSTEM FLUSH LOGS") + res = node.query( + f""" + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE query_id='{query_id}' AND type='QueryFinish' + """ + ) + if res == "": + res = node.query( + """ + SELECT query_id, query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log ORDER BY query_start_time_microseconds DESC + """ + ) + print(f"LOG: {res}") + assert False + assert "proj1" in res + + if expect_broken_part == "proj2": + assert expected_error in node.query_and_get_error( + f"SELECT d FROM '{table}' WHERE c == 12 ORDER BY d" + ) + else: + query_id = node.query( + f"SELECT queryID() FROM (SELECT d FROM '{table}' WHERE c == 12 ORDER BY d)" + ).strip() + node.query("SYSTEM FLUSH LOGS") + res = node.query( + f""" + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE query_id='{query_id}' AND type='QueryFinish' + """ + ) + if res == "": + res = node.query( + """ + SELECT query_id, query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log ORDER BY query_start_time_microseconds DESC + """ + ) + print(f"LOG: {res}") + assert False + assert "proj2" in res + + assert check_result == int(node.query(f"CHECK TABLE {table}")) + + +def test_broken_ignored(cluster): + node = cluster.instances["node"] + + table_name = "test1" + create_table(node, table_name, 1) + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + # Break metadata (columns.txt) file of projection 'proj1' + break_projection(node, table_name, "proj1", "all_2_2_0", "metadata") + + # Do select and after "check table" query. + # Select works because it does not read columns.txt. + # But expect check table result as 0. + check(node, table_name, 0) + + # Projection 'proj1' from part all_2_2_0 will now appear in broken parts info + # because it was marked broken during "check table" query. + assert "all_2_2_0\tproj1\tFILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name + ) + + # Check table query will also show a list of parts which have broken projections. + assert "all_2_2_0" in check_table_full(node, table_name) + + # Break data file of projection 'proj2' for part all_2_2_0 + break_projection(node, table_name, "proj2", "all_2_2_0", "data") + + # It will not yet appear in broken projections info. + assert "proj2" not in get_broken_projections_info(node, table_name) + + # Select now fails with error "File doesn't exist" + check(node, table_name, 0, "proj2", "FILE_DOESNT_EXIST") + + # Projection 'proj2' from part all_2_2_0 will now appear in broken parts info. + assert "all_2_2_0\tproj2\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name + ) + + # Second select works, because projection is now marked as broken. + check(node, table_name, 0) + + # Break data file of projection 'proj2' for part all_3_3_0 + break_projection(node, table_name, "proj2", "all_3_3_0", "data") + + # It will not yet appear in broken projections info. + assert "all_3_3_0" not in get_broken_projections_info(node, table_name) + + insert(node, table_name, 20, 5) + insert(node, table_name, 25, 5) + + # Part all_3_3_0 has 'proj' and 'proj2' projections, but 'proj2' is broken and server does NOT know it yet. + # Parts all_4_4_0 and all_5_5_0 have both non-broken projections. + # So a merge will be create for future part all_3_5_1. + # During merge it will fail to read from 'proj2' of part all_3_3_0 and proj2 will be marked broken. + # Merge will be retried and on second attempt it will succeed. + # The result part all_3_5_1 will have only 1 projection - 'proj', because + # it will skip 'proj2' as it will see that one part does not have it anymore in the set of valid projections. + optimize(node, table_name, 0, 1) + time.sleep(5) + + # table_uuid=node.query(f"SELECT uuid FROM system.tables WHERE table='{table_name}' and database=currentDatabase()").strip() + # assert 0 < int( + # node.query( + # f""" + # SYSTEM FLUSH LOGS; + # SELECT count() FROM system.text_log + # WHERE level='Error' + # AND logger_name='MergeTreeBackgroundExecutor' + # AND message like 'Exception while executing background task %{table_uuid}:all_3_5_1%%Cannot open file%proj2.proj/c.bin%' + # """) + # ) + + assert "all_3_3_0" in get_broken_projections_info(node, table_name) + check(node, table_name, 0) + + +def test_materialize_broken_projection(cluster): + node = cluster.instances["node"] + + table_name = "test2" + create_table(node, table_name, 1) + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + break_projection(node, table_name, "proj1", "all_1_1_0", "metadata") + reattach(node, table_name) + + assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name + ) + assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( + node, table_name + ) + + break_projection(node, table_name, "proj2", "all_1_1_0", "data") + reattach(node, table_name) + + assert "all_1_1_0\tproj2\tFILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name + ) + assert "Part all_1_1_0 has a broken projection proj2" in check_table_full( + node, table_name + ) + + materialize_projection(node, table_name, "proj1") + + assert "has a broken projection" not in check_table_full(node, table_name) + + +def test_broken_ignored_replicated(cluster): + node = cluster.instances["node"] + + table_name = "test3" + table_name2 = "test3_replica" + create_table(node, table_name, 1) + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + check(node, table_name, 1) + + create_table(node, table_name2, 2, table_name) + check(node, table_name2, 1) + + break_projection(node, table_name, "proj1", "all_0_0_0", "data") + assert "Part all_0_0_0 has a broken projection proj1" in check_table_full( + node, table_name + ) + + break_part(node, table_name, "all_0_0_0") + node.query(f"SYSTEM SYNC REPLICA {table_name}") + assert "has a broken projection" not in check_table_full(node, table_name) + + +def get_random_string(string_length=8): + alphabet = string.ascii_letters + string.digits + return "".join((random.choice(alphabet) for _ in range(string_length))) + + +def test_broken_projections_in_backups_1(cluster): + node = cluster.instances["node"] + + table_name = "test4" + create_table(node, table_name, 1, aggressive_merge=False, data_prefix=table_name) + + node.query("SYSTEM STOP MERGES") + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + check(node, table_name, 1) + + break_projection(node, table_name, "proj1", "all_2_2_0", "data") + check(node, table_name, 0, "proj1", "FILE_DOESNT_EXIST") + + assert "all_2_2_0\tproj1\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name + ) + + backup_name = f"b1-{get_random_string()}" + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false; + """ + ) + + assert "RESTORED" in node.query( + f""" + drop table {table_name} sync; + set backup_restore_keeper_fault_injection_probability=0.0; + restore table {table_name} from Disk('backups', '{backup_name}'); + """ + ) + + node.query("SYSTEM STOP MERGES") + + check(node, table_name, 1) + assert "" == get_broken_projections_info(node, table_name) + + +def test_broken_projections_in_backups_2(cluster): + node = cluster.instances["node"] + + table_name = "test5" + create_table(node, table_name, 1, aggressive_merge=False, data_prefix=table_name) + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + check(node, table_name, 1) + break_projection(node, table_name, "proj2", "all_2_2_0", "part") + check(node, table_name, 0, "proj2", "ErrnoException") + + assert "all_2_2_0\tproj2\tFILE_DOESNT_EXIST" == get_broken_projections_info( + node, table_name + ) + + assert "FILE_DOESNT_EXIST" in node.query_and_get_error( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', 'b2') + """ + ) + + materialize_projection(node, table_name, "proj2") + check(node, table_name, 1) + + backup_name = f"b3-{get_random_string()}" + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false; + """ + ) + + assert "RESTORED" in node.query( + f""" + drop table {table_name} sync; + set backup_restore_keeper_fault_injection_probability=0.0; + restore table {table_name} from Disk('backups', '{backup_name}'); + """ + ) + check(node, table_name, 1) + + +def test_broken_projections_in_backups_3(cluster): + node = cluster.instances["node"] + + table_name = "test6" + create_table(node, table_name, 1, aggressive_merge=False, data_prefix=table_name) + + node.query("SYSTEM STOP MERGES") + + insert(node, table_name, 0, 5) + insert(node, table_name, 5, 5) + insert(node, table_name, 10, 5) + insert(node, table_name, 15, 5) + + assert ["all_0_0_0", "all_1_1_0", "all_2_2_0", "all_3_3_0"] == get_parts( + node, table_name + ) + + check(node, table_name, 1) + + break_projection(node, table_name, "proj1", "all_1_1_0", "part") + assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( + node, table_name + ) + assert "all_1_1_0\tproj1\tFILE_DOESNT_EXIST" == get_broken_projections_info( + node, table_name + ) + + backup_name = f"b4-{get_random_string()}" + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false, allow_backup_broken_projections=true; + """ + ) + + assert "RESTORED" in node.query( + f""" + drop table {table_name} sync; + set backup_restore_keeper_fault_injection_probability=0.0; + restore table {table_name} from Disk('backups', '{backup_name}'); + """ + ) + + check(node, table_name, 0) + assert "all_1_1_0\tproj1\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( + node, table_name + ) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 7382b24afbc..e60fb844de8 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -686,6 +686,9 @@ CREATE TABLE system.projection_parts `rows_where_ttl_info.expression` Array(String), `rows_where_ttl_info.min` Array(DateTime), `rows_where_ttl_info.max` Array(DateTime), + `is_broken` UInt8, + `exception_code` Int32, + `exception` String, `bytes` UInt64 ALIAS bytes_on_disk, `marks_size` UInt64 ALIAS marks_bytes, `part_name` String ALIAS name From d529389522311e7bca11a3beebc07e0439efcfb4 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:20:20 -0500 Subject: [PATCH 002/154] Add support for 'START TRANSACTION' syntax --- src/Parsers/ParserTransactionControl.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Parsers/ParserTransactionControl.cpp b/src/Parsers/ParserTransactionControl.cpp index da593170002..fc3077bb0b6 100644 --- a/src/Parsers/ParserTransactionControl.cpp +++ b/src/Parsers/ParserTransactionControl.cpp @@ -14,6 +14,8 @@ bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (ParserKeyword("BEGIN TRANSACTION").ignore(pos, expected)) action = ASTTransactionControl::BEGIN; + else if (ParserKeyword("START TRANSACTION").ignore(pos, expected)) + action = ASTTransactionControl::BEGIN; else if (ParserKeyword("COMMIT").ignore(pos, expected)) action = ASTTransactionControl::COMMIT; else if (ParserKeyword("ROLLBACK").ignore(pos, expected)) From d596de73847c712d766e0f8598ce43ed078ec968 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:25:48 -0500 Subject: [PATCH 003/154] update documentation --- docs/en/sql-reference/transactions.md | 42 +++++++++++++++++++++------ 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index cb89a091d68..b9c9afc20f9 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -1,26 +1,29 @@ --- slug: /en/guides/developer/transactional --- + # Transactional (ACID) support -## Case 1: INSERT into one partition, of one table, of the MergeTree* family +## Case 1: INSERT into one partition, of one table, of the MergeTree\* family This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): + - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen - Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). - INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). -## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree\* family Same as Case 1 above, with this detail: + - If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own - -## Case 3: INSERT into one distributed table of the MergeTree* family +## Case 3: INSERT into one distributed table of the MergeTree\* family Same as Case 1 above, with this detail: + - INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional ## Case 4: Using a Buffer table @@ -30,9 +33,11 @@ Same as Case 1 above, with this detail: ## Case 5: Using async_insert Same as Case 1 above, with this detail: + - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes + - rows inserted from the client in some data format are packed into a single block when: - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data @@ -61,8 +66,9 @@ In addition to the functionality described at the top of this document, ClickHou ``` ### Notes + - This is an experimental feature, and changes should be expected. -- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. +- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. - Nested transactions are not supported; finish the current transaction and start a new one instead ### Configuration @@ -80,7 +86,7 @@ These examples are with a single node ClickHouse server with ClickHouse Keeper e #### Basic configuration for a single ClickHouse server node with ClickHouse Keeper enabled :::note -See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. +See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. ::: ```xml title=/etc/clickhouse-server/config.d/config.xml @@ -127,17 +133,19 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` :::tip If you see the following error, then check your configuration file to make sure that `allow_experimental_transactions` is set to `1` (or any value other than `0` or `false`). + ``` Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Transactions are not supported. @@ -145,15 +153,18 @@ DB::Exception: Transactions are not supported. ``` You can also check ClickHouse Keeper by issuing + ``` echo ruok | nc localhost 9181 ``` + ClickHouse Keeper should respond with `imok`. ::: ```sql ROLLBACK ``` + ```response Ok. ``` @@ -161,7 +172,7 @@ Ok. #### Create a table for testing :::tip -Creation of tables is not transactional. Run this DDL query outside of a transaction. +Creation of tables is not transactional. Run this DDL query outside of a transaction. ::: ```sql @@ -172,6 +183,7 @@ CREATE TABLE mergetree_table ENGINE = MergeTree ORDER BY n ``` + ```response Ok. ``` @@ -181,6 +193,7 @@ Ok. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` @@ -188,6 +201,7 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (10) ``` + ```response Ok. ``` @@ -196,11 +210,13 @@ Ok. SELECT * FROM mergetree_table ``` + ```response ┌──n─┐ │ 10 │ └────┘ ``` + :::note You can query the table from within a transaction and see that the row was inserted even though it has not yet been committed. ::: @@ -208,16 +224,20 @@ You can query the table from within a transaction and see that the row was inser #### Rollback the transaction, and query the table again Verify that the transaction is rolled back: + ```sql ROLLBACK ``` + ```response Ok. ``` + ```sql SELECT * FROM mergetree_table ``` + ```response Ok. @@ -229,6 +249,7 @@ Ok. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` @@ -236,6 +257,7 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (42) ``` + ```response Ok. ``` @@ -243,6 +265,7 @@ Ok. ```sql COMMIT ``` + ```response Ok. Elapsed: 0.002 sec. ``` @@ -251,6 +274,7 @@ Ok. Elapsed: 0.002 sec. SELECT * FROM mergetree_table ``` + ```response ┌──n─┐ │ 42 │ @@ -267,6 +291,7 @@ SELECT * FROM system.transactions FORMAT Vertical ``` + ```response Row 1: ────── @@ -280,4 +305,3 @@ state: RUNNING ## More Details See this [meta issue](https://github.com/ClickHouse/ClickHouse/issues/48794) to find much more extensive tests and to keep up to date with the progress. - From 34d327a08cc2de20510eb429c2da30b4d135a3f0 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:26:57 -0500 Subject: [PATCH 004/154] Revert "update documentation" This reverts commit d596de73847c712d766e0f8598ce43ed078ec968. --- docs/en/sql-reference/transactions.md | 42 ++++++--------------------- 1 file changed, 9 insertions(+), 33 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index b9c9afc20f9..cb89a091d68 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -1,29 +1,26 @@ --- slug: /en/guides/developer/transactional --- - # Transactional (ACID) support -## Case 1: INSERT into one partition, of one table, of the MergeTree\* family +## Case 1: INSERT into one partition, of one table, of the MergeTree* family This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): - - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen - Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). - INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). -## Case 2: INSERT into multiple partitions, of one table, of the MergeTree\* family +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family Same as Case 1 above, with this detail: - - If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own -## Case 3: INSERT into one distributed table of the MergeTree\* family + +## Case 3: INSERT into one distributed table of the MergeTree* family Same as Case 1 above, with this detail: - - INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional ## Case 4: Using a Buffer table @@ -33,11 +30,9 @@ Same as Case 1 above, with this detail: ## Case 5: Using async_insert Same as Case 1 above, with this detail: - - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes - - rows inserted from the client in some data format are packed into a single block when: - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data @@ -66,9 +61,8 @@ In addition to the functionality described at the top of this document, ClickHou ``` ### Notes - - This is an experimental feature, and changes should be expected. -- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. +- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. - Nested transactions are not supported; finish the current transaction and start a new one instead ### Configuration @@ -86,7 +80,7 @@ These examples are with a single node ClickHouse server with ClickHouse Keeper e #### Basic configuration for a single ClickHouse server node with ClickHouse Keeper enabled :::note -See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. +See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. ::: ```xml title=/etc/clickhouse-server/config.d/config.xml @@ -133,19 +127,17 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` :::tip If you see the following error, then check your configuration file to make sure that `allow_experimental_transactions` is set to `1` (or any value other than `0` or `false`). - ``` Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Transactions are not supported. @@ -153,18 +145,15 @@ DB::Exception: Transactions are not supported. ``` You can also check ClickHouse Keeper by issuing - ``` echo ruok | nc localhost 9181 ``` - ClickHouse Keeper should respond with `imok`. ::: ```sql ROLLBACK ``` - ```response Ok. ``` @@ -172,7 +161,7 @@ Ok. #### Create a table for testing :::tip -Creation of tables is not transactional. Run this DDL query outside of a transaction. +Creation of tables is not transactional. Run this DDL query outside of a transaction. ::: ```sql @@ -183,7 +172,6 @@ CREATE TABLE mergetree_table ENGINE = MergeTree ORDER BY n ``` - ```response Ok. ``` @@ -193,7 +181,6 @@ Ok. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` @@ -201,7 +188,6 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (10) ``` - ```response Ok. ``` @@ -210,13 +196,11 @@ Ok. SELECT * FROM mergetree_table ``` - ```response ┌──n─┐ │ 10 │ └────┘ ``` - :::note You can query the table from within a transaction and see that the row was inserted even though it has not yet been committed. ::: @@ -224,20 +208,16 @@ You can query the table from within a transaction and see that the row was inser #### Rollback the transaction, and query the table again Verify that the transaction is rolled back: - ```sql ROLLBACK ``` - ```response Ok. ``` - ```sql SELECT * FROM mergetree_table ``` - ```response Ok. @@ -249,7 +229,6 @@ Ok. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` @@ -257,7 +236,6 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (42) ``` - ```response Ok. ``` @@ -265,7 +243,6 @@ Ok. ```sql COMMIT ``` - ```response Ok. Elapsed: 0.002 sec. ``` @@ -274,7 +251,6 @@ Ok. Elapsed: 0.002 sec. SELECT * FROM mergetree_table ``` - ```response ┌──n─┐ │ 42 │ @@ -291,7 +267,6 @@ SELECT * FROM system.transactions FORMAT Vertical ``` - ```response Row 1: ────── @@ -305,3 +280,4 @@ state: RUNNING ## More Details See this [meta issue](https://github.com/ClickHouse/ClickHouse/issues/48794) to find much more extensive tests and to keep up to date with the progress. + From f5cf61c32e9f511933b56048f22aaf43fad67a6c Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:28:48 -0500 Subject: [PATCH 005/154] update documentation without autoformat --- docs/en/sql-reference/transactions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index cb89a091d68..09cdc192b03 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -127,7 +127,7 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION From b074477ffb841587c19313063c249bc4c35ef301 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:45:09 -0500 Subject: [PATCH 006/154] use new syntax in a test --- tests/integration/test_transactions/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 46660581223..584e59ba71b 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -67,8 +67,8 @@ def test_rollback_unfinished_on_restart1(start_cluster): tx(1, "insert into mt values (5, 50)") tx(1, "alter table mt update m = m+n in partition id '1' where 1") - # check that uncommitted insert will be rolled back on restart - tx(3, "begin transaction") + # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION syntax`) + tx(3, "start transaction") tid5 = tx(3, "select transactionID()").strip() tx(3, "insert into mt values (6, 70)") From 7ed1be6f3c51db3bd4abb57c429c77e09b0cd8a8 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:46:39 -0500 Subject: [PATCH 007/154] fix test comment --- tests/integration/test_transactions/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 584e59ba71b..d63b7b6f545 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -67,7 +67,7 @@ def test_rollback_unfinished_on_restart1(start_cluster): tx(1, "insert into mt values (5, 50)") tx(1, "alter table mt update m = m+n in partition id '1' where 1") - # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION syntax`) + # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION` syntax) tx(3, "start transaction") tid5 = tx(3, "select transactionID()").strip() tx(3, "insert into mt values (6, 70)") From bf85f7f368c09a11946ccba02d11b55ef71ffdf4 Mon Sep 17 00:00:00 2001 From: Zhuo Qiu Date: Mon, 18 Mar 2024 11:02:02 +0800 Subject: [PATCH 008/154] add docs & minor fix for lwd-aware merge settings --- .../settings/merge-tree-settings.md | 28 +++++++++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index c7e461d15ae..9ce705ff805 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -867,3 +867,31 @@ Default value: `Never` Persists virtual column `_block_number` on merges. Default value: false. + +## exclude_deleted_rows_for_part_size_in_merge {#exclude_deleted_rows_for_part_size_in_merge} + +If enabled, estimated actual size of data parts (i.e., excluding those rows that have been deleted through `DELETE FROM`) will be used when selecting parts to merge. Note that this behavior is only triggered for data parts affected by `DELETE FROM` executed after this setting is enabled. + +Possible values: + +- true, false + +Default value: false + +**See Also** + +- [load_existing_rows_count_for_old_parts](#load_existing_rows_count_for_old_parts) setting + +## load_existing_rows_count_for_old_parts {#load_existing_rows_count_for_old_parts} + +If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. + +Possible values: + +- true, false + +Default value: false + +**See Also** + +- [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 023202019e4..3ae3b678f37 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1349,8 +1349,9 @@ void IMergeTreeDataPart::loadExistingRowsCount() if (existing_rows_count.has_value()) return; - if (!rows_count || !storage.getSettings()->load_existing_rows_count_for_old_parts || !supportLightweightDeleteMutate() - || !hasLightweightDelete()) + if (!rows_count || !supportLightweightDeleteMutate() || !hasLightweightDelete() + || !storage.getSettings()->exclude_deleted_rows_for_part_size_in_merge + || !storage.getSettings()->load_existing_rows_count_for_old_parts) existing_rows_count = rows_count; else existing_rows_count = readExistingRowsCount(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 8bd32e777bc..075a5c45710 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -674,7 +674,8 @@ private: /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); - /// Load existing rows count from _row_exists column if load_existing_rows_count_for_old_parts is true. + /// Load existing rows count from _row_exists column + /// if load_existing_rows_count_for_old_parts and exclude_deleted_rows_for_part_size_in_merge are both enabled. void loadExistingRowsCount(); static void appendFilesOfRowsCount(Strings & files); From 6b954fcfd42746e7c9f09329a89c0e31f329d41c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 14 Mar 2024 11:31:53 +0000 Subject: [PATCH 009/154] Fix crash in ObjectJson parsing array with nulls --- src/DataTypes/ObjectUtils.cpp | 5 +++++ src/DataTypes/ObjectUtils.h | 2 +- .../queries/0_stateless/02287_type_object_convert.reference | 1 + tests/queries/0_stateless/02287_type_object_convert.sql | 1 + 4 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index c565930f49a..1d697159b56 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -1001,6 +1001,11 @@ Field FieldVisitorFoldDimension::operator()(const Array & x) const return res; } +Field FieldVisitorFoldDimension::operator()(const Null &) const +{ + return Array(); +} + void setAllObjectsToDummyTupleType(NamesAndTypesList & columns) { for (auto & column : columns) diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 2bfcaae09ca..2b729dddc28 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -139,7 +139,7 @@ public: Field operator()(const Array & x) const; - Field operator()(const Null & x) const { return x; } + Field operator()(const Null & x) const; template Field operator()(const T & x) const diff --git a/tests/queries/0_stateless/02287_type_object_convert.reference b/tests/queries/0_stateless/02287_type_object_convert.reference index 501536f1f3e..af4407ee052 100644 --- a/tests/queries/0_stateless/02287_type_object_convert.reference +++ b/tests/queries/0_stateless/02287_type_object_convert.reference @@ -16,3 +16,4 @@ {"x":[[],[1,2]]} {"x":[[],[[1],[2]]]} {"x":[[],[[],[2]]]} +{"a.a":[[1],[]]} diff --git a/tests/queries/0_stateless/02287_type_object_convert.sql b/tests/queries/0_stateless/02287_type_object_convert.sql index 2bc08cf7fe1..21c1aba63d3 100644 --- a/tests/queries/0_stateless/02287_type_object_convert.sql +++ b/tests/queries/0_stateless/02287_type_object_convert.sql @@ -32,3 +32,4 @@ SELECT CAST('{"x" : [ 1 , [ 1 , 2] ]}', 'Object(\'json\')'); SELECT CAST('{"x" : [ {} , [ 1 , 2] ]}', 'Object(\'json\')'); SELECT CAST('{"x" : [ {} , [ 1 , [2]] ]}', 'Object(\'json\')'); SELECT CAST('{"x" : [ {} , [ {} , [2]] ]}', 'Object(\'json\')'); +SELECT CAST(' {"a": { "a": [ [1], null ] } }', 'Object(Nullable(\'json\'))'); From 9c9a4db2de3d04a1fce453c521a583264c4b4d06 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Mar 2024 15:07:29 +0000 Subject: [PATCH 010/154] fix --- src/DataTypes/ObjectUtils.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 1d697159b56..1a64b29e82f 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -1001,8 +1001,11 @@ Field FieldVisitorFoldDimension::operator()(const Array & x) const return res; } -Field FieldVisitorFoldDimension::operator()(const Null &) const +Field FieldVisitorFoldDimension::operator()(const Null & x) const { + if (num_dimensions_to_fold == 0) + return x; + return Array(); } From 567a550149a37cfccc1db7b5b7a52f978a4c2e5b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Mar 2024 15:15:36 +0000 Subject: [PATCH 011/154] Safer Subcolumn::insert --- src/Columns/ColumnObject.cpp | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 25a62440629..0a4f90c2262 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -20,12 +20,12 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; + extern const int ARGUMENT_OUT_OF_BOUND; extern const int DUPLICATE_COLUMN; + extern const int EXPERIMENTAL_FEATURE_ERROR; + extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; - extern const int ARGUMENT_OUT_OF_BOUND; - extern const int EXPERIMENTAL_FEATURE_ERROR; } namespace @@ -334,7 +334,18 @@ void ColumnObject::Subcolumn::insert(Field field, FieldInfo info) if (type_changed || info.need_convert) field = convertFieldToTypeOrThrow(field, *least_common_type.get()); - data.back()->insert(field); + if (!data.back()->tryInsert(field)) + { + /** Normalization of the field above is pretty complicated (it uses several FieldVisitors), + * so in the case of a bug, we may get mismatched types. + * The `IColumn::insert` method does not check the type of the inserted field, and it can lead to a segmentation fault. + * Therefore, we use the safer `tryInsert` method to get an exception instead of a segmentation fault. + */ + throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, + "Cannot insert field {} to column {}", + field.dump(), data.back()->dumpStructure()); + } + ++num_rows; } From 62c91fdddc8c0da90b8ef14ecd5a0f8c442073d3 Mon Sep 17 00:00:00 2001 From: Zhuo Qiu Date: Mon, 18 Mar 2024 22:33:07 +0800 Subject: [PATCH 012/154] revise docs Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/operations/settings/merge-tree-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 9ce705ff805..3e411a51ff4 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -884,7 +884,7 @@ Default value: false ## load_existing_rows_count_for_old_parts {#load_existing_rows_count_for_old_parts} -If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. +If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. Note that it may slow down start up table loading. Possible values: From f0b955888b304b952d4ab32941712dc492860b4e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 18 Mar 2024 18:46:17 +0000 Subject: [PATCH 013/154] Fix tryInsert for ColumnUInt8 and Bool --- src/Columns/ColumnVector.cpp | 22 ++++++++++++++++++++++ src/Columns/ColumnVector.h | 10 ++-------- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 015a114105d..4e3b9963107 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -460,6 +460,28 @@ Float32 ColumnVector::getFloat32(size_t n [[maybe_unused]]) const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get the value of {} as Float32", TypeName); } +template +bool ColumnVector::tryInsert(const DB::Field & x) +{ + NearestFieldType value; + if (!x.tryGet>(value)) + { + if constexpr (std::is_same_v) + { + /// It's also possible to insert boolean values into UInt8 column. + bool boolean_value; + if (x.tryGet(boolean_value)) + { + data.push_back(static_cast(boolean_value)); + return true; + } + } + return false; + } + data.push_back(static_cast(value)); + return true; +} + template void ColumnVector::insertRangeFrom(const IColumn & src, size_t start, size_t length) { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index b03d476a8f1..39ee1d931bd 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -224,14 +224,8 @@ public: data.push_back(static_cast(x.get())); } - bool tryInsert(const DB::Field & x) override - { - NearestFieldType value; - if (!x.tryGet>(value)) - return false; - data.push_back(static_cast(value)); - return true; - } + bool tryInsert(const DB::Field & x) override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override; From 200b500ecfdb5468f9682a024e8a7903f39bb8bb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 20 Mar 2024 13:12:58 +0100 Subject: [PATCH 014/154] Enable experimental analyzer by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5c25368fb00..db31ace310f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -372,7 +372,7 @@ class IColumn; M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \ M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \ - M(Bool, allow_experimental_analyzer, false, "Allow experimental analyzer", 0) \ + M(Bool, allow_experimental_analyzer, true, "Allow experimental analyzer.", 0) \ M(Bool, analyzer_compatibility_join_using_top_level_identifier, false, "Force to resolve identifier in JOIN USING from projection (for example, in `SELECT a + 1 AS b FROM t1 JOIN t2 USING (b)` join will be performed by `t1.a + 1 = t2.b`, rather then `t1.b = t2.b`).", 0) \ M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \ M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 0bce814170b..5c430517fa7 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -102,6 +102,7 @@ static std::map sett {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, + {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From f0e7b78f8be1d4b0f8c851418fd56fcccd970983 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 20 Mar 2024 13:35:30 +0100 Subject: [PATCH 015/154] Inverse analyzer in tests --- .github/workflows/master.yml | 4 ++-- .github/workflows/release_branches.yml | 2 +- tests/ci/functional_test_check.py | 4 ++-- tests/ci/integration_test_check.py | 2 +- tests/ci/libfuzzer_test_check.py | 2 +- tests/config/install.sh | 2 +- tests/config/users.d/analyzer.xml | 2 +- tests/integration/ci-runner.py | 4 ++-- tests/integration/helpers/0_common_enable_analyzer.xml | 2 +- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/runner | 2 +- 11 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2853adff48a..816bdfd4f31 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -374,7 +374,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Stateless tests (release, analyzer, s3, DatabaseReplicated) + test_name: Stateless tests (release, old analyzer, s3, DatabaseReplicated) runner_type: func-tester data: ${{ needs.RunConfig.outputs.data }} FunctionalStatelessTestS3Debug: @@ -632,7 +632,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Integration tests (asan, analyzer) + test_name: Integration tests (asan, old analyzer) runner_type: stress-tester data: ${{ needs.RunConfig.outputs.data }} IntegrationTestsTsan: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 9e95b3d3d8f..4d45c8d8d4b 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -436,7 +436,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Integration tests (asan, analyzer) + test_name: Integration tests (asan, old analyzer) runner_type: stress-tester data: ${{ needs.RunConfig.outputs.data }} IntegrationTestsTsan: diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 4416b7863a6..a0c977ea25f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -39,7 +39,7 @@ def get_additional_envs( result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") result.append("RANDOMIZE_OBJECT_KEY_TYPE=1") if "analyzer" in check_name: - result.append("USE_NEW_ANALYZER=1") + result.append("USE_OLD_ANALYZER=1") if run_by_hash_total != 0: result.append(f"RUN_BY_HASH_NUM={run_by_hash_num}") @@ -94,7 +94,7 @@ def get_run_command( env_str = " ".join(envs) volume_with_broken_test = ( f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " - if "analyzer" in check_name + if "analyzer" not in check_name else "" ) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 751abf617fa..a8701a6ec72 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -73,7 +73,7 @@ def get_env_for_runner( my_env["CLICKHOUSE_TESTS_RUNNER_RESTART_DOCKER"] = "0" if "analyzer" in check_name.lower(): - my_env["CLICKHOUSE_USE_NEW_ANALYZER"] = "1" + my_env["CLICKHOUSE_USE_OLD_ANALYZER"] = "1" return my_env diff --git a/tests/ci/libfuzzer_test_check.py b/tests/ci/libfuzzer_test_check.py index 62dadc6fea9..4bb39010978 100644 --- a/tests/ci/libfuzzer_test_check.py +++ b/tests/ci/libfuzzer_test_check.py @@ -33,7 +33,7 @@ def get_additional_envs(check_name, run_by_hash_num, run_by_hash_total): result.append("USE_S3_STORAGE_FOR_MERGE_TREE=1") result.append("RANDOMIZE_OBJECT_KEY_TYPE=1") if "analyzer" in check_name: - result.append("USE_NEW_ANALYZER=1") + result.append("USE_OLD_ANALYZER=1") if run_by_hash_total != 0: result.append(f"RUN_BY_HASH_NUM={run_by_hash_num}") diff --git a/tests/config/install.sh b/tests/config/install.sh index 467636cfa40..652d25a0a35 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -92,7 +92,7 @@ ln -sf $SRC_PATH/users.d/nonconst_timezone.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/allow_introspection_functions.yaml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/replicated_ddl_entry.xml $DEST_SERVER_PATH/users.d/ -if [[ -n "$USE_NEW_ANALYZER" ]] && [[ "$USE_NEW_ANALYZER" -eq 1 ]]; then +if [[ -n "$USE_OLD_ANALYZER" ]] && [[ "$USE_OLD_ANALYZER" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/analyzer.xml $DEST_SERVER_PATH/users.d/ fi diff --git a/tests/config/users.d/analyzer.xml b/tests/config/users.d/analyzer.xml index aa374364ef0..4b9764526fa 100644 --- a/tests/config/users.d/analyzer.xml +++ b/tests/config/users.d/analyzer.xml @@ -1,7 +1,7 @@ - 1 + 0 diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index e7f691d2237..3c017db7964 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -263,7 +263,7 @@ class ClickhouseIntegrationTestsRunner: self.start_time = time.time() self.soft_deadline_time = self.start_time + (TASK_TIMEOUT - MAX_TIME_IN_SANDBOX) - self.use_analyzer = os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None + self.use_analyzer = os.environ.get("CLICKHOUSE_USE_OLD_ANALYZER") is not None if "run_by_hash_total" in self.params: self.run_by_hash_total = self.params["run_by_hash_total"] @@ -916,7 +916,7 @@ class ClickhouseIntegrationTestsRunner: random.shuffle(items_to_run) broken_tests = [] - if self.use_analyzer: + if not self.use_analyzer: with open( f"{repo_path}/tests/analyzer_integration_broken_tests.txt", "r", diff --git a/tests/integration/helpers/0_common_enable_analyzer.xml b/tests/integration/helpers/0_common_enable_analyzer.xml index aa374364ef0..4b9764526fa 100644 --- a/tests/integration/helpers/0_common_enable_analyzer.xml +++ b/tests/integration/helpers/0_common_enable_analyzer.xml @@ -1,7 +1,7 @@ - 1 + 0 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index b695b493db7..ea023bb5e97 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4408,8 +4408,8 @@ class ClickHouseInstance: write_embedded_config("0_common_instance_users.xml", users_d_dir) if ( - os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None - and self.allow_analyzer + os.environ.get("CLICKHOUSE_USE_OLD_ANALYZER") is not None + or self.allow_analyzer ): write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) diff --git a/tests/integration/runner b/tests/integration/runner index f1d5198f545..b88f277cdbc 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -387,7 +387,7 @@ if __name__ == "__main__": use_analyzer = "" if args.analyzer: - use_analyzer = "-e CLICKHOUSE_USE_NEW_ANALYZER=1" + use_analyzer = "-e CLICKHOUSE_USE_OLD_ANALYZER=1" # NOTE: since pytest options is in the argument value already we need to additionally escape '"' pytest_opts = " ".join( From b74a0359e5567ae7c18ae49b55ae804298234eb1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 20 Mar 2024 13:43:18 +0100 Subject: [PATCH 016/154] Update integration tests --- tests/integration/helpers/cluster.py | 10 +++++----- tests/integration/test_backward_compatibility/test.py | 4 ++-- .../test_aggregate_fixed_key.py | 6 +++--- .../test_aggregate_function_state.py | 8 ++++---- .../test_insert_profile_events.py | 4 ++-- .../test_ip_types_binary_compatibility.py | 2 +- .../test_memory_bound_aggregation.py | 6 +++--- .../test_normalized_count_comparison.py | 4 ++-- .../test_select_aggregate_alias_column.py | 4 ++-- .../test_short_strings_aggregation.py | 6 +++--- .../test_vertical_merges_from_compact_parts.py | 4 ++-- tests/integration/test_disk_over_web_server/test.py | 8 ++++---- .../test.py | 2 +- .../test_groupBitmapAnd_on_distributed/test.py | 4 ++-- .../test_groupBitmapAndState_on_distributed_table.py | 4 ++-- tests/integration/test_old_versions/test.py | 2 +- tests/integration/test_polymorphic_parts/test.py | 2 +- .../test_replicated_merge_tree_compatibility/test.py | 4 ++-- tests/integration/test_storage_s3_queue/test.py | 2 +- tests/integration/test_ttl_replicated/test.py | 6 +++--- tests/integration/test_version_update/test.py | 2 +- .../test_version_update_after_mutation/test.py | 6 +++--- 22 files changed, 50 insertions(+), 50 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index ea023bb5e97..28143cf735e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1606,7 +1606,7 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, - allow_analyzer=True, + use_old_analyzer=False, hostname=None, env_variables=None, instance_env_variables=False, @@ -1705,7 +1705,7 @@ class ClickHouseCluster: with_coredns=with_coredns, with_cassandra=with_cassandra, with_ldap=with_ldap, - allow_analyzer=allow_analyzer, + use_old_analyzer=use_old_analyzer, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -3267,7 +3267,7 @@ class ClickHouseInstance: with_coredns, with_cassandra, with_ldap, - allow_analyzer, + use_old_analyzer, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, @@ -3361,7 +3361,7 @@ class ClickHouseInstance: self.with_hive = with_hive self.with_coredns = with_coredns self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) - self.allow_analyzer = allow_analyzer + self.use_old_analyzer = use_old_analyzer self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -4409,7 +4409,7 @@ class ClickHouseInstance: write_embedded_config("0_common_instance_users.xml", users_d_dir) if ( os.environ.get("CLICKHOUSE_USE_OLD_ANALYZER") is not None - or self.allow_analyzer + or self.use_old_analyzer ): write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index 098fc8c1025..00973640998 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,13 +10,13 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", main_configs=["configs/wide_parts_only.xml", "configs/no_compress_marks.xml"], with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index b0c0f5d17c7..263f2d0af55 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -9,10 +9,10 @@ node1 = cluster.add_instance( image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) -node2 = cluster.add_instance("node2", with_zookeeper=True, allow_analyzer=False) -node3 = cluster.add_instance("node3", with_zookeeper=True, allow_analyzer=False) +node2 = cluster.add_instance("node2", with_zookeeper=True, use_old_analyzer=True) +node3 = cluster.add_instance("node3", with_zookeeper=True, use_old_analyzer=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py index 5972f57b928..ad2b20a07ad 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,10 +19,10 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) -node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) -node4 = cluster.add_instance("node4", with_zookeeper=False, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) +node4 = cluster.add_instance("node4", with_zookeeper=False, use_old_analyzer=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 30958127ee5..7caf15335d0 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -7,13 +7,13 @@ import pytest from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) -upstream_node = cluster.add_instance("upstream_node", allow_analyzer=False) +upstream_node = cluster.add_instance("upstream_node", use_old_analyzer=True) old_node = cluster.add_instance( "old_node", image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py index 02c81ddbd52..f91f04da823 100644 --- a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py +++ b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py @@ -10,7 +10,7 @@ node = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index 337a967e309..2869b91f984 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,9 +19,9 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) -node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index 7f6c3fc92e1..830dec8c517 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, use_old_analyzer=True) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,7 +11,7 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index be161df0640..8ddcbb71cab 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, use_old_analyzer=True) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,7 +11,7 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 5d0e5e24af5..715d295ac9e 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,9 +19,9 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) -node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index d3730357989..9740a8828b2 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -11,7 +11,7 @@ node_old = cluster.add_instance( stay_alive=True, with_installed_binary=True, with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) node_new = cluster.add_instance( "node2", @@ -21,7 +21,7 @@ node_new = cluster.add_instance( ], with_zookeeper=True, stay_alive=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index dbcd7cc3c21..115161dcaad 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -13,7 +13,7 @@ def cluster(): "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True, - allow_analyzer=False, + use_old_analyzer=True, ) cluster.add_instance( "node2", @@ -21,14 +21,14 @@ def cluster(): with_nginx=True, stay_alive=True, with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) cluster.add_instance( "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True, with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) cluster.add_instance( @@ -39,7 +39,7 @@ def cluster(): with_installed_binary=True, image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, - allow_analyzer=False, + use_old_analyzer=True, ) cluster.start() diff --git a/tests/integration/test_distributed_insert_backward_compatibility/test.py b/tests/integration/test_distributed_insert_backward_compatibility/test.py index 839e1008df1..72cc74e627a 100644 --- a/tests/integration/test_distributed_insert_backward_compatibility/test.py +++ b/tests/integration/test_distributed_insert_backward_compatibility/test.py @@ -14,7 +14,7 @@ node_dist = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index 5119a4e0e36..118cd78105e 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -8,13 +8,13 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py b/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py index 237acf6b9e0..20ba02e7a80 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py @@ -8,13 +8,13 @@ node1 = cluster.add_instance( "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index 43f91b7d265..e8b4999af46 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -10,7 +10,7 @@ node_oldest = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], - allow_analyzer=False, + use_old_analyzer=True, ) old_nodes = [node_oldest] new_node = cluster.add_instance("node_new") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index b91a72c5534..714a89352bf 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -369,7 +369,7 @@ node7 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node8 = cluster.add_instance( "node8", diff --git a/tests/integration/test_replicated_merge_tree_compatibility/test.py b/tests/integration/test_replicated_merge_tree_compatibility/test.py index 25cf3caa50c..c7e9933e1de 100644 --- a/tests/integration/test_replicated_merge_tree_compatibility/test.py +++ b/tests/integration/test_replicated_merge_tree_compatibility/test.py @@ -9,7 +9,7 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -18,7 +18,7 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index a7abd840834..c7893c3e643 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -108,7 +108,7 @@ def started_cluster(): tag="23.12", stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) logging.info("Starting cluster...") diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index b20b761ef47..c62e5c23438 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -23,7 +23,7 @@ node4 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) node5 = cluster.add_instance( @@ -36,7 +36,7 @@ node5 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) node6 = cluster.add_instance( "node6", @@ -48,7 +48,7 @@ node6 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index ab3eb1860f3..704b1bd2b21 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -14,7 +14,7 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, stay_alive=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 90f8d283a6a..e9cc34a8f27 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -16,7 +16,7 @@ node1 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -28,7 +28,7 @@ node2 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) node3 = cluster.add_instance( "node3", @@ -40,7 +40,7 @@ node3 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - allow_analyzer=False, + use_old_analyzer=True, ) From 1524d5a4512d558888cd76b28fad685d23c63ac3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 20 Mar 2024 14:33:14 +0100 Subject: [PATCH 017/154] Pass broken tests list to FastTest --- tests/ci/fast_test_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index a624ee06e85..383f5b340c7 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -47,6 +47,7 @@ def get_fasttest_cmd( f"-e SCCACHE_BUCKET={S3_BUILDS_BUCKET} -e SCCACHE_S3_KEY_PREFIX=ccache/sccache " "-e stage=clone_submodules " f"--volume={workspace}:/fasttest-workspace --volume={repo_path}:/ClickHouse " + f"--volume={repo_path}/tests/analyzer_tech_debt.txt:/analyzer_tech_debt.txt " f"--volume={output_path}:/test_output {image}" ) From a3718451b595c577658a349159d3ab71308e9b93 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 20 Mar 2024 16:36:21 +0000 Subject: [PATCH 018/154] stash --- src/Databases/DatabaseOnDisk.cpp | 45 +++++++++++++++++++------------- 1 file changed, 27 insertions(+), 18 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index dcfc1916450..53c94cff22d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -1,12 +1,17 @@ #include +#include +#include +#include +#include +#include #include #include #include #include +#include #include #include -#include #include #include #include @@ -15,14 +20,11 @@ #include #include #include -#include -#include -#include #include #include -#include -#include -#include +#include +#include +#include namespace fs = std::filesystem; @@ -612,7 +614,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat }; /// Metadata files to load: name and flag for .tmp_drop files - std::set> metadata_files; + std::vector> metadata_files; fs::directory_iterator dir_end; for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) @@ -633,7 +635,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat if (endsWith(file_name, ".sql.tmp_drop")) { /// There are files that we tried to delete previously - metadata_files.emplace(file_name, false); + metadata_files.emplace_back(file_name, false); } else if (endsWith(file_name, ".sql.tmp")) { @@ -644,23 +646,30 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat else if (endsWith(file_name, ".sql")) { /// The required files have names like `table_name.sql` - metadata_files.emplace(file_name, true); + metadata_files.emplace_back(file_name, true); } else throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Incorrect file extension: {} in metadata directory {}", file_name, getMetadataPath()); } + std::sort(metadata_files.begin(), metadata_files.end()); + metadata_files.erase(std::unique(metadata_files.begin(), metadata_files.end()), metadata_files.end()); + /// Read and parse metadata in parallel ThreadPool pool(CurrentMetrics::DatabaseOnDiskThreads, CurrentMetrics::DatabaseOnDiskThreadsActive, CurrentMetrics::DatabaseOnDiskThreadsScheduled); - for (const auto & file : metadata_files) + const auto batch_size = metadata_files.size() / pool.getMaxThreads() + 1; + for (auto it = metadata_files.begin(); it < metadata_files.end(); std::advance(it, batch_size)) { - pool.scheduleOrThrowOnError([&]() - { - if (file.second) - process_metadata_file(file.first); - else - process_tmp_drop_metadata_file(file.first); - }); + std::span batch{it, std::min(std::next(it, batch_size), metadata_files.end())}; + pool.scheduleOrThrowOnError( + [batch, &process_metadata_file, &process_tmp_drop_metadata_file]() mutable + { + for (const auto & file : batch) + if (file.second) + process_metadata_file(file.first); + else + process_tmp_drop_metadata_file(file.first); + }); } pool.wait(); } From 0af1bb2eaa45e08c79e2687fa8d8fe40680b78de Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Mar 2024 16:58:58 +0000 Subject: [PATCH 019/154] Don't allow the same expression in ORDER BY with and without WITH FILL --- src/Processors/Transforms/FillingTransform.cpp | 13 ++++++++++++- .../03015_with_fill_invalid_expression.reference | 0 .../03015_with_fill_invalid_expression.sql | 2 ++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03015_with_fill_invalid_expression.reference create mode 100644 tests/queries/0_stateless/03015_with_fill_invalid_expression.sql diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c659a2578f3..54812405610 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -237,10 +237,21 @@ FillingTransform::FillingTransform( } logDebug("fill description", dumpSortDescription(fill_description)); - std::set unique_positions; + std::unordered_set ordinary_sort_positions; + for (const auto & desc : sort_description) + { + if (!desc.with_fill) + ordinary_sort_positions.insert(header_.getPositionByName(desc.column_name)); + } + + std::unordered_set unique_positions; for (auto pos : fill_column_positions) + { if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); + if (ordinary_sort_positions.contains(pos)) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "ORDER BY containing the same expression with and without WITH FILL is not supported"); + } if (use_with_fill_by_sorting_prefix) { diff --git a/tests/queries/0_stateless/03015_with_fill_invalid_expression.reference b/tests/queries/0_stateless/03015_with_fill_invalid_expression.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03015_with_fill_invalid_expression.sql b/tests/queries/0_stateless/03015_with_fill_invalid_expression.sql new file mode 100644 index 00000000000..dbb63c029d6 --- /dev/null +++ b/tests/queries/0_stateless/03015_with_fill_invalid_expression.sql @@ -0,0 +1,2 @@ +select number as x, number + 1 as y from numbers(5) where number % 3 == 1 order by y, x with fill from 1 to 4, y with fill from 2 to 5; -- {serverError INVALID_WITH_FILL_EXPRESSION} + From c799bfd016f9f537df2626a178eaba2f4de8a5b2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Jan 2024 16:57:44 +0000 Subject: [PATCH 020/154] Analyzer: fixes for join using columns --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 124 +++++++++++++++++- src/Interpreters/ActionsDAG.cpp | 2 +- src/Planner/CollectTableExpressionData.cpp | 28 +++- src/Planner/PlannerJoinTree.cpp | 44 ++++--- .../0_stateless/00722_inner_join.reference | 6 +- .../02514_analyzer_drop_join_on.reference | 36 ++--- .../02722_matcher_join_use_nulls.reference | 16 +-- .../02835_join_step_explain.reference | 36 ++--- ...5_analyzer_using_functional_args.reference | 79 +++++++++++ .../02955_analyzer_using_functional_args.sql | 53 +++++++- ..._join_use_nulls_column_not_found.reference | 2 + ...alyzer_join_use_nulls_column_not_found.sql | 31 +++++ 12 files changed, 373 insertions(+), 84 deletions(-) create mode 100644 tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference create mode 100644 tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5a557875fec..5058a194239 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1350,6 +1350,7 @@ private: const NamesAndTypes & matched_columns, const IdentifierResolveScope & scope); + void updateMatchedColumnsFromJoinUsing(QueryTreeNodesWithNames & result_matched_column_nodes_with_names, const QueryTreeNodePtr & source_table_expression, IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); @@ -3272,6 +3273,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto & result_column = result_column_node->as(); result_column.setColumnType(using_column_node.getColumnType()); + /// Reset the expression for the column in the projection (or any other section outside of JOIN), it is calculated beforehand for JOIN USING. + result_column.setExpression(nullptr); + resolved_identifier = std::move(result_column_node); } else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false})) @@ -3344,11 +3348,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo { auto & left_resolved_column = left_resolved_identifier->as(); auto using_column_node_it = join_using_column_name_to_column_node.find(left_resolved_column.getColumnName()); - if (using_column_node_it != join_using_column_name_to_column_node.end() && - !using_column_node_it->second->getColumnType()->equals(*left_resolved_column.getColumnType())) + if (using_column_node_it != join_using_column_name_to_column_node.end()) { auto left_resolved_column_clone = std::static_pointer_cast(left_resolved_column.clone()); - left_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + if (!using_column_node_it->second->getColumnType()->equals(*left_resolved_column.getColumnType())) + left_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + + /// Reset the expression for the column in the projection (or any other section outside of JOIN), it is calculated beforehand for JOIN USING. + left_resolved_column_clone->setExpression(nullptr); + resolved_identifier = std::move(left_resolved_column_clone); } } @@ -3366,11 +3374,15 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo { auto & right_resolved_column = right_resolved_identifier->as(); auto using_column_node_it = join_using_column_name_to_column_node.find(right_resolved_column.getColumnName()); - if (using_column_node_it != join_using_column_name_to_column_node.end() && - !using_column_node_it->second->getColumnType()->equals(*right_resolved_column.getColumnType())) + if (using_column_node_it != join_using_column_name_to_column_node.end()) { auto right_resolved_column_clone = std::static_pointer_cast(right_resolved_column.clone()); - right_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + if (!using_column_node_it->second->getColumnType()->equals(*right_resolved_column.getColumnType())) + right_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); + + /// Reset the expression for the column in the projection (or any other section outside of JOIN), it is calculated beforehand for JOIN USING. + right_resolved_column_clone->setExpression(nullptr); + resolved_identifier = std::move(right_resolved_column_clone); } } @@ -3381,7 +3393,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (join_use_nulls) { + auto it = node_to_projection_name.find(resolved_identifier); resolved_identifier = resolved_identifier->clone(); + if (it != node_to_projection_name.end()) + node_to_projection_name.emplace(resolved_identifier, it->second); convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side); } @@ -3955,6 +3970,97 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::getMatchedColumnNodesWithN return matched_column_nodes_with_names; } + +bool hasTableExpressionInJoinTree(const QueryTreeNodePtr & join_tree_node, const QueryTreeNodePtr & table_expression) +{ + QueryTreeNodes nodes_to_process; + nodes_to_process.push_back(join_tree_node); + + while (!nodes_to_process.empty()) + { + auto node_to_process = std::move(nodes_to_process.back()); + nodes_to_process.pop_back(); + if (node_to_process == table_expression) + return true; + + if (node_to_process->getNodeType() == QueryTreeNodeType::JOIN) + { + const auto & join_node = node_to_process->as(); + nodes_to_process.push_back(join_node.getLeftTableExpression()); + nodes_to_process.push_back(join_node.getRightTableExpression()); + } + } + return false; +} + +/// Columns that resolved from matcher can also match columns from JOIN USING. +/// In that case we update type to type of column in USING section. +/// TODO: It's not completely correct for qualified matchers, so t1.* should be resolved to left table column type. +/// But in planner we do not distinguish such cases. +void QueryAnalyzer::updateMatchedColumnsFromJoinUsing( + QueryTreeNodesWithNames & result_matched_column_nodes_with_names, + const QueryTreeNodePtr & source_table_expression, + IdentifierResolveScope & scope) +{ + auto * nearest_query_scope = scope.getNearestQueryScope(); + auto * nearest_query_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; + + /// If there are no parent query scope or query scope does not have join tree + if (!nearest_query_scope_query_node || !nearest_query_scope_query_node->getJoinTree()) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "There are no table sources. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + } + + const auto & join_tree = nearest_query_scope_query_node->getJoinTree(); + + const auto * join_node = join_tree->as(); + if (join_node && join_node->isUsingJoinExpression()) + { + const auto & join_using_list = join_node->getJoinExpression()->as(); + const auto & join_using_nodes = join_using_list.getNodes(); + + for (auto & [matched_column_node, _] : result_matched_column_nodes_with_names) + { + auto & matched_column_node_typed = matched_column_node->as(); + const auto & matched_column_name = matched_column_node_typed.getColumnName(); + + for (const auto & join_using_node : join_using_nodes) + { + auto & join_using_column_node = join_using_node->as(); + const auto & join_using_column_name = join_using_column_node.getColumnName(); + + if (matched_column_name != join_using_column_name) + continue; + + const auto & join_using_column_nodes_list = join_using_column_node.getExpressionOrThrow()->as(); + const auto & join_using_column_nodes = join_using_column_nodes_list.getNodes(); + + auto it = node_to_projection_name.find(matched_column_node); + + if (hasTableExpressionInJoinTree(join_node->getLeftTableExpression(), source_table_expression)) + matched_column_node = join_using_column_nodes.at(0); + else if (hasTableExpressionInJoinTree(join_node->getRightTableExpression(), source_table_expression)) + matched_column_node = join_using_column_nodes.at(1); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot find column {} in JOIN USING section {}", + matched_column_node->dumpTree(), join_node->dumpTree()); + + matched_column_node = matched_column_node->clone(); + if (it != node_to_projection_name.end()) + node_to_projection_name.emplace(matched_column_node, it->second); + + matched_column_node->as().setColumnType(join_using_column_node.getResultType()); + + /// Reset the expression for the column in the projection (or any other section outside of JOIN), it is calculated beforehand for JOIN USING. + matched_column_node->as().setExpression(nullptr); + } + } + } +} + /** Resolve qualified tree matcher. * * First try to match qualified identifier to expression. If qualified identifier matched expression node then @@ -4049,7 +4155,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu if (table_expression_query_node || table_expression_union_node) { matched_columns = table_expression_query_node ? table_expression_query_node->getProjectionColumns() - : table_expression_union_node->computeProjectionColumns(); + : table_expression_union_node->computeProjectionColumns(); } else if (table_expression_table_node || table_expression_table_function_node) { @@ -4072,6 +4178,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu matched_columns, scope); + updateMatchedColumnsFromJoinUsing(result_matched_column_nodes_with_names, table_expression_node, scope); return result_matched_column_nodes_with_names; } @@ -4186,6 +4293,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( matched_column_node = matched_column_node->clone(); matched_column_node->as().setColumnType(join_using_column_node.getResultType()); + /// Reset the expression for the column in the projection (or any other section outside of JOIN), it is calculated beforehand for JOIN USING. + matched_column_node->as().setExpression(nullptr); + table_expression_column_names_to_skip.insert(join_using_column_name); matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), join_using_column_name); } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 1124ba94bc1..4a5eb514bb0 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -679,7 +679,7 @@ Block ActionsDAG::updateHeader(Block header) const catch (Exception & e) { if (e.code() == ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK) - e.addMessage(" in block {}", header.dumpStructure()); + e.addMessage("in block {}", header.dumpStructure()); throw; } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 78a7c7074c3..b87446a3ea2 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -91,6 +91,11 @@ public: return; } + /// JoinNode doesn't have its dedicated identifiers, + /// we apply projection actions from outer scope to columns from JOIN + if (column_source_node_type == QueryTreeNodeType::JOIN) + return; + if (column_source_node_type != QueryTreeNodeType::TABLE && column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && column_source_node_type != QueryTreeNodeType::QUERY && @@ -123,7 +128,7 @@ private: void visitUsingColumn(QueryTreeNodePtr & node) { - auto & column_node = node->as(); + auto & column_node = node->as(); if (column_node.hasExpression()) { auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_node.getColumnSource()); @@ -134,10 +139,10 @@ private: auto column_identifier = planner_context.getGlobalPlannerContext()->createColumnIdentifier(node); table_expression_data.addAliasColumnName(column_node.getColumnName(), column_identifier); - visitImpl(column_node.getExpressionOrThrow()); + visit(column_node.getExpressionOrThrow()); } else - visitImpl(node); + visit(node); } PlannerContext & planner_context; @@ -275,13 +280,28 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr } CollectSourceColumnsVisitor collect_source_columns_visitor(*planner_context); + + /** Visit the join nodes first to ensure that all columns in subtrees are collected before the projection node is visited. + * This is crucial for column nodes that originate from the JOIN USING clause. + * For example: + * SELECT a FROM t1 JOIN t2 USING a ORDER BY a; + * In this case, if `a` is an ALIAS column, the expression for it will be calculated from the USING clause. + * Therefore, the table expression data for t1 and t2 should be prepared before visiting the `a` column node in projection or ORDER BY. + */ + auto table_expression_stack = buildTableExpressionsStack(query_node_typed.getJoinTree()); + for (auto & table_expression_node : table_expression_stack) + { + if (table_expression_node->getNodeType() == QueryTreeNodeType::JOIN) + collect_source_columns_visitor.visit(table_expression_node); + } + for (auto & node : query_node_typed.getChildren()) { if (!node || node == query_node_typed.getPrewhere()) continue; auto node_type = node->getNodeType(); - if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) + if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION || node_type == QueryTreeNodeType::JOIN) continue; collect_source_columns_visitor.visit(node); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index ab25f6d2423..ae20694f14c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -990,40 +990,42 @@ struct UsingAliasKeyActions ) : left_alias_columns_keys(std::make_shared(left_plan_output_columns)) , right_alias_columns_keys(std::make_shared(right_plan_output_columns)) - {} - - void addLeftColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) { - addColumnImpl(left_alias_columns_keys, node, plan_output_columns, planner_context); + for (const auto * input_node : left_alias_columns_keys->getInputs()) + left_alias_columns_keys->addOrReplaceInOutputs(*input_node); + + for (const auto * input_node : right_alias_columns_keys->getInputs()) + right_alias_columns_keys->addOrReplaceInOutputs(*input_node); } - void addRightColumn(QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) + void addLeftColumn(QueryTreeNodePtr & node, const PlannerContextPtr & planner_context) { - addColumnImpl(right_alias_columns_keys, node, plan_output_columns, planner_context); + addColumnImpl(left_alias_columns_keys, node, planner_context); } - ActionsDAGPtr getLeftActions() + void addRightColumn(QueryTreeNodePtr & node, const PlannerContextPtr & planner_context) { - left_alias_columns_keys->projectInput(); - return std::move(left_alias_columns_keys); + addColumnImpl(right_alias_columns_keys, node, planner_context); } - ActionsDAGPtr getRightActions() - { - right_alias_columns_keys->projectInput(); - return std::move(right_alias_columns_keys); - } + ActionsDAGPtr getLeftActions() { return left_alias_columns_keys; } + ActionsDAGPtr getRightActions() { return right_alias_columns_keys; } private: - void addColumnImpl(ActionsDAGPtr & alias_columns_keys, QueryTreeNodePtr & node, const ColumnsWithTypeAndName & plan_output_columns, const PlannerContextPtr & planner_context) + + void addColumnImpl(ActionsDAGPtr & actions_dag, QueryTreeNodePtr & node, const PlannerContextPtr & planner_context) { - auto & column_node = node->as(); + auto & column_node = node->as(); if (column_node.hasExpression()) { - auto dag = buildActionsDAGFromExpressionNode(column_node.getExpressionOrThrow(), plan_output_columns, planner_context); + PlannerActionsVisitor actions_visitor(planner_context); + auto expression_dag_index_nodes = actions_visitor.visit(actions_dag, column_node.getExpressionOrThrow()); + for (const auto * out_node : expression_dag_index_nodes) + actions_dag->addOrReplaceInOutputs(*out_node); + const auto & left_inner_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(node); - dag->addOrReplaceInOutputs(dag->addAlias(*dag->getOutputs().front(), left_inner_column_identifier)); - alias_columns_keys->mergeInplace(std::move(*dag)); + const auto & alias_output = actions_dag->addAlias(*actions_dag->getOutputs().back(), left_inner_column_identifier); + actions_dag->addOrReplaceInOutputs(alias_output); } } @@ -1110,12 +1112,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto & left_inner_column_node = inner_columns_list.getNodes().at(0); auto & left_inner_column = left_inner_column_node->as(); - using_alias_key_actions.addLeftColumn(left_inner_column_node, left_plan_output_columns, planner_context); + using_alias_key_actions.addLeftColumn(left_inner_column_node, planner_context); auto & right_inner_column_node = inner_columns_list.getNodes().at(1); auto & right_inner_column = right_inner_column_node->as(); - using_alias_key_actions.addRightColumn(right_inner_column_node, right_plan_output_columns, planner_context); + using_alias_key_actions.addRightColumn(right_inner_column_node, planner_context); const auto & join_node_using_column_node_type = join_node_using_column_node.getColumnType(); if (!left_inner_column.getColumnType()->equals(*join_node_using_column_node_type)) diff --git a/tests/queries/0_stateless/00722_inner_join.reference b/tests/queries/0_stateless/00722_inner_join.reference index b5e8a77a20d..c7ba26e3e6a 100644 --- a/tests/queries/0_stateless/00722_inner_join.reference +++ b/tests/queries/0_stateless/00722_inner_join.reference @@ -31,9 +31,9 @@ ┌─database─┬─name─┐ │ system │ one │ └──────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ +┌─database─┬─t.name─┐ +│ system │ one │ +└──────────┴────────┘ 2 2 2 diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index a5a71560d00..bbeaae0d31b 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -52,27 +52,27 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String __table4.d2 String Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String Expression ((Actions for left table alias column keys + DROP unused columns after JOIN)) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String Join (JOIN FillRightFirst) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String Expression ((Actions for left table alias column keys + Change column names to column identifiers)) - Header: __table1.a2 String - __table1.k UInt64 + Header: __table1.k UInt64 + __table1.a2 String ReadFromMemoryStorage - Header: a2 String - k UInt64 + Header: k UInt64 + a2 String Expression ((Actions for right table alias column keys + Change column names to column identifiers)) Header: __table2.k UInt64 ReadFromMemoryStorage @@ -82,11 +82,11 @@ Header: a2 String ReadFromMemoryStorage Header: k UInt64 Expression ((Actions for right table alias column keys + Change column names to column identifiers)) - Header: __table4.d2 String - __table4.k UInt64 + Header: __table4.k UInt64 + __table4.d2 String ReadFromMemoryStorage - Header: d2 String - k UInt64 + Header: k UInt64 + d2 String EXPLAIN PLAN header = 1 SELECT b.bx FROM a JOIN (SELECT b1, b2 || 'x' AS bx FROM b ) AS b ON b.b1 = a.a1 diff --git a/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference index 746d02dc381..5617854bfd1 100644 --- a/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference +++ b/tests/queries/0_stateless/02722_matcher_join_use_nulls.reference @@ -72,7 +72,7 @@ FROM ( LEFT JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -1 Int32 +1 Int64 SELECT *, * APPLY toTypeName FROM ( SELECT t2.* @@ -80,7 +80,7 @@ FROM ( LEFT JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -\N Nullable(UInt32) +\N Nullable(Int64) SELECT *, * APPLY toTypeName FROM ( SELECT * @@ -209,7 +209,7 @@ FROM ( RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -2 Nullable(Int32) +2 Nullable(Int64) SELECT *, * APPLY toTypeName FROM ( SELECT t2.* @@ -217,7 +217,7 @@ FROM ( RIGHT JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -2 UInt32 +2 Int64 SELECT *, * APPLY toTypeName FROM ( SELECT * @@ -354,8 +354,8 @@ FROM ( FULL JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -1 Nullable(Int32) -2 Nullable(Int32) +1 Nullable(Int64) +2 Nullable(Int64) SELECT *, * APPLY toTypeName FROM ( SELECT t2.* @@ -363,8 +363,8 @@ FROM ( FULL JOIN (SELECT 2 :: UInt32 as a) t2 USING (a) ) ORDER BY 1; -2 Nullable(UInt32) -\N Nullable(UInt32) +2 Nullable(Int64) +\N Nullable(Int64) SELECT *, * APPLY toTypeName FROM ( SELECT * diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index 06f4a9cfc99..d2042b56f55 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -56,20 +56,20 @@ Header: id UInt64 rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 - INPUT : 1 -> __table1.value_1 String : 1 - INPUT :: 2 -> __table1.value_2 UInt64 : 2 + INPUT :: 1 -> __table1.value_2 UInt64 : 1 + INPUT : 2 -> __table1.value_1 String : 2 INPUT : 3 -> __table2.value_1 String : 3 INPUT :: 4 -> __table2.value_2 UInt64 : 4 INPUT : 5 -> __table2.id UInt64 : 5 ALIAS __table1.id :: 0 -> id UInt64 : 6 - ALIAS __table1.value_1 :: 1 -> value_1 String : 0 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 + ALIAS __table1.value_1 :: 2 -> value_1 String : 0 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 -Positions: 6 0 3 1 +Positions: 6 0 3 2 Join (JOIN FillRightFirst) Header: __table1.id UInt64 - __table1.value_1 String __table1.value_2 UInt64 + __table1.value_1 String __table2.value_1 String __table2.value_2 UInt64 __table2.id UInt64 @@ -80,37 +80,37 @@ Positions: 6 0 3 1 Clauses: [(__table1.id, __table1.value_2) = (__table2.id, __table2.value_2)] Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.id UInt64 - __table1.value_1 String __table1.value_2 UInt64 + __table1.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 - INPUT : 1 -> value_1 String : 1 - INPUT : 2 -> value_2 UInt64 : 2 + INPUT : 1 -> value_2 UInt64 : 1 + INPUT : 2 -> value_1 String : 2 ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value_1 :: 1 -> __table1.value_1 String : 0 - ALIAS value_2 :: 2 -> __table1.value_2 UInt64 : 1 + ALIAS value_2 :: 1 -> __table1.value_2 UInt64 : 0 + ALIAS value_1 :: 2 -> __table1.value_1 String : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_1) Header: id UInt64 - value_1 String value_2 UInt64 + value_1 String ReadType: Default Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table2.id UInt64 - __table2.value_1 String __table2.value_2 UInt64 + __table2.value_1 String Actions: INPUT : 0 -> id UInt64 : 0 - INPUT : 1 -> value_1 String : 1 - INPUT : 2 -> value_2 UInt64 : 2 + INPUT : 1 -> value_2 UInt64 : 1 + INPUT : 2 -> value_1 String : 2 ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value_1 :: 1 -> __table2.value_1 String : 0 - ALIAS value_2 :: 2 -> __table2.value_2 UInt64 : 1 + ALIAS value_2 :: 1 -> __table2.value_2 UInt64 : 0 + ALIAS value_1 :: 2 -> __table2.value_1 String : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_2) Header: id UInt64 - value_1 String value_2 UInt64 + value_1 String ReadType: Default Parts: 1 Granules: 1 diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference index 6ed281c757a..f4d8a77ef7f 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference @@ -1,2 +1,81 @@ 1 1 +369 124 123 b +369 124 123 b +124 +3693 1231 a 1231 +3693 1232 1231 1231 a +a +-- {echoOn } +-- USING alias column contains default in old analyzer (but both queries below should have the same result) +SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +738 ba +7386 aa +13332 a +SELECT y * 2, s || 'a' FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; +738 ba +7386 aa +13332 a +SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +(1,369,123,'b',124) +(1,3693,1231,'a',0) +(1,6666,0,'',48) +SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +(1,369,'b') +(1,3693,'a') +(1,6666,'') +SELECT (1, t1.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; +(1,0,'') +(1,123,'b') +(1,1231,'a') +SELECT (1, t1.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; +(1,'',6666) +(1,'a',3693) +(1,'b',369) +SELECT (1, t1.*, t2.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; +(1,0,'',6666,48) +(1,123,'b',369,124) +(1,1231,'a',0,0) +SELECT (1, t1.*, t2.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; +(1,'',0,6666) +(1,'a',3693,0) +(1,'b',369,369) +SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +1 0 43 +1 48 1 +124 124 1 +1232 0 1232 +SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +126 0 0 42 +369 123 b 124 0 +3693 1231 a 0 1231 +6666 0 48 0 +SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +0 126 0 42 +0 6666 48 0 +123 b 369 124 0 +1231 a 3693 0 1231 +SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS allow_experimental_analyzer = 1; +(1,0,'',126,0,42) +(1,0,'',6666,48,0) +(1,123,'b',369,124,0) +(1,1231,'a',3693,0,1231) +SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +369 +3693 +6666 +SELECT y FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; +369 +3693 +6666 +SELECT s FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; + +a +b +SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + +a +b +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql index 7983b43d7e5..f92598c7c32 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql @@ -1,12 +1,57 @@ -CREATE TABLE t1 (x Int16, y ALIAS x + x * 2) ENGINE=MergeTree() ORDER BY x; -CREATE TABLE t2 (y Int16, z Int16) ENGINE=MergeTree() ORDER BY y; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t1 (x Int16, y Int64 ALIAS x + x * 2, z ALIAS x + 1, s String) ENGINE=MergeTree() ORDER BY x; +CREATE TABLE t2 (y Int128, z Int16) ENGINE=MergeTree() ORDER BY y; + +CREATE TABLE t3 (x Int16, y Int64 ALIAS x + x * 2, z ALIAS x + 1) ENGINE=MergeTree() ORDER BY x; + +INSERT INTO t1 VALUES (1231, 'a'), (123, 'b'); -INSERT INTO t1 VALUES (1231), (123); INSERT INTO t2 VALUES (6666, 48); -INSERT INTO t2 VALUES (369, 50); +INSERT INTO t2 VALUES (369, 124); + +INSERT INTO t3 VALUES (1231), (42); SELECT count() FROM t1 INNER JOIN t2 USING (y); SELECT count() FROM t2 INNER JOIN t1 USING (y); +-- `SELECT *` works differently for ALIAS columns with analyzer +SELECT * FROM t1 INNER JOIN t2 USING (y, z) SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t2 INNER JOIN t1 USING (y, z) SETTINGS allow_experimental_analyzer = 1; +SELECT t2.z FROM t1 INNER JOIN t2 USING (y); + +SELECT * FROM t1 INNER JOIN t3 USING (y) SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t3 INNER JOIN t1 USING (y, z) SETTINGS allow_experimental_analyzer = 1; +SELECT s FROM t1 INNER JOIN t3 USING (y); + +-- {echoOn } +-- USING alias column contains default in old analyzer (but both queries below should have the same result) +SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y * 2, s || 'a' FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + +SELECT (1, *) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT (1, *) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; + +SELECT (1, t1.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; +SELECT (1, t1.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + +SELECT (1, t1.*, t2.*) FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; +SELECT (1, t1.*, t2.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + +SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT t1.*, t2.*, t3.* FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; +SELECT (1, t1.*, t2.*, t3.*) FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1 SETTINGS allow_experimental_analyzer = 1; + +SELECT y FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; +SELECT y FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + +SELECT s FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; +SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; + + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference new file mode 100644 index 00000000000..a5c8806279f --- /dev/null +++ b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference @@ -0,0 +1,2 @@ +3 +3 diff --git a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql new file mode 100644 index 00000000000..7101d5c89ae --- /dev/null +++ b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql @@ -0,0 +1,31 @@ +SET join_use_nulls = 1; + +select c FROM ( + select + d2.c + from ( select 1 as a, 2 as b ) d1 + FULL join ( select 1 as a, 3 as c ) d2 + on (d1.a = d2.a) +) +; + +with d1 as ( + select + 1 as a, + 2 as b +), +d2 as ( + select + 1 as a, + 3 as c +), +joined as ( + select + d1.*, + d2.c + from d1 + inner join d2 + on (d1.a = d2.a) +) +select c +from joined; From dcaa71245164289ff9d1b74fc2516950558170a8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 19 Mar 2024 23:43:01 +0000 Subject: [PATCH 021/154] disallow LowCardinality input type for JSONExtract Workaraund for a memory corruption issue https://github.com/ClickHouse/ClickHouse/issues/61562 It seems that the root cause lies not within the parser itself, but rather either with the Columns/ColumnLowCardinality or Functions/IFunction code paths. --- src/Functions/FunctionsJSON.h | 4 +--- tests/queries/0_stateless/00918_json_functions.reference | 1 + tests/queries/0_stateless/00918_json_functions.sql | 2 ++ 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 2539fa1aeb4..53515985e39 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -348,6 +348,7 @@ public: String getName() const override { return Name::name; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { @@ -469,9 +470,6 @@ public: else return_type = json_return_type; - /// Top-level LowCardinality columns are processed outside JSON parser. - json_return_type = removeLowCardinality(json_return_type); - DataTypes argument_types; argument_types.reserve(arguments.size()); for (const auto & argument : arguments) diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 43b15ded93d..7b725111755 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -69,6 +69,7 @@ hello (3333.6,'test') (3333.6333333333,'test') (3333.6333333333,'test') +\N 123456.1234 Decimal(20, 4) 123456.1234 Decimal(20, 4) 123456789012345.12 Decimal(30, 4) diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index e19dd17670e..61fcb21fcbd 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -81,6 +81,7 @@ SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Dec SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(10,1), b LowCardinality(String))'); SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); +SELECT JSONExtract(materialize('{"string_value":null}'), materialize('string_value'), 'LowCardinality(Nullable(String))'); SELECT JSONExtract('{"a":123456.123456}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456.123456"}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456789012345.12"}', 'a', 'Decimal(30, 4)') as a, toTypeName(a); @@ -326,3 +327,4 @@ SELECT JSONExtract('[]', JSONExtract('0', 'UInt256'), 'UInt256'); -- { serverErr SELECT '--show error: key of map type should be String'; SELECT JSONExtract('{"a": [100.0, 200], "b": [-100, 200.0, 300]}', 'Map(Int64, Array(Float64))'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT JSONExtract(materialize(toLowCardinality('{"string_value":null}')), materialize('string_value'), 'LowCardinality(Nullable(String))'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT } From 8d9a58c93614cfc01bbd93765e3e01dc26da4e76 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Mar 2024 02:17:58 +0000 Subject: [PATCH 022/154] LowCardinalityFixedStringColumn: allow generic types Fix tests and add new ones. --- src/Functions/FunctionsJSON.h | 9 ++------- .../02474_extract_fixedstring_from_json.reference | 7 +++++++ .../0_stateless/02474_extract_fixedstring_from_json.sql | 7 +++++++ 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 53515985e39..8a193785f87 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -865,11 +865,9 @@ struct JSONExtractTree explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } bool insertResultToColumn(IColumn & dest, const Element & element) override { - // If element is an object we delegate the insertion to JSONExtractRawImpl - if (element.isObject()) + // For types other than string, delegate the insertion to JSONExtractRawImpl. + if (!element.isString()) return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); - else if (!element.isString()) - return false; auto str = element.getString(); if (str.size() > fixed_length) @@ -1484,9 +1482,6 @@ public: // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) { - if (element.getObject().size() > fixed_length) - return false; - ColumnFixedString::Chars chars; WriteBufferFromVector buf(chars, AppendModeTag()); traverse(element, buf); diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference index 783d12fcf1a..21ddf5d3512 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference @@ -8,3 +8,10 @@ \0\0\0\0\0 131231 131231 +1234 +1234 +{"b":131231} +\0\0\0\0 +1234567890 +18446744073709551615 +-9223372036854775807 diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql index cfc47e00cba..bbb9f55062b 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql @@ -6,3 +6,10 @@ SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(5)'); SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(6)'); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(5))') FROM numbers(2); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(6))') FROM numbers(2); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": "1234"}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": {"b": 131231} }'), 'a', 'LowCardinality(FixedString(12))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(10))'); +SELECT JSONExtract(materialize('{"a": 18446744073709551615}'), 'a', 'LowCardinality(FixedString(20))'); +SELECT JSONExtract(materialize('{"a": -9223372036854775807}'), 'a', 'LowCardinality(FixedString(20))'); From 42d59a8fd11a8a66dd8392ecb83eb1971b8cfb1f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Mar 2024 17:05:34 +0000 Subject: [PATCH 023/154] Update after refactoring ALIAS columns computation --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 96 +++++++++++++++++++ .../0_stateless/00722_inner_join.reference | 6 +- .../02835_join_step_explain.reference | 36 +++---- ...alyzer_using_functional_args.reference.j2} | 11 ++- ...955_analyzer_using_functional_args.sql.j2} | 15 ++- 5 files changed, 135 insertions(+), 29 deletions(-) rename tests/queries/0_stateless/{02955_analyzer_using_functional_args.reference => 02955_analyzer_using_functional_args.reference.j2} (94%) rename tests/queries/0_stateless/{02955_analyzer_using_functional_args.sql => 02955_analyzer_using_functional_args.sql.j2} (86%) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 0414ea93f94..6a5efa6532f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1392,6 +1392,8 @@ private: const NamesAndTypes & matched_columns, const IdentifierResolveScope & scope); + void updateMatchedColumnsFromJoinUsing(QueryTreeNodesWithNames & result_matched_column_nodes_with_names, const QueryTreeNodePtr & source_table_expression, IdentifierResolveScope & scope); + QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); QueryTreeNodesWithNames resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope); @@ -3543,9 +3545,14 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (scope.join_use_nulls) { + auto it = node_to_projection_name.find(resolved_identifier); auto nullable_resolved_identifier = convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side, scope); if (nullable_resolved_identifier) + { resolved_identifier = nullable_resolved_identifier; + if (it != node_to_projection_name.end()) + node_to_projection_name.emplace(resolved_identifier, it->second); + } } return resolved_identifier; @@ -4204,6 +4211,93 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::getMatchedColumnNodesWithN return matched_column_nodes_with_names; } +bool hasTableExpressionInJoinTree(const QueryTreeNodePtr & join_tree_node, const QueryTreeNodePtr & table_expression) +{ + QueryTreeNodes nodes_to_process; + nodes_to_process.push_back(join_tree_node); + + while (!nodes_to_process.empty()) + { + auto node_to_process = std::move(nodes_to_process.back()); + nodes_to_process.pop_back(); + if (node_to_process == table_expression) + return true; + + if (node_to_process->getNodeType() == QueryTreeNodeType::JOIN) + { + const auto & join_node = node_to_process->as(); + nodes_to_process.push_back(join_node.getLeftTableExpression()); + nodes_to_process.push_back(join_node.getRightTableExpression()); + } + } + return false; +} + +/// Columns that resolved from matcher can also match columns from JOIN USING. +/// In that case we update type to type of column in USING section. +/// TODO: It's not completely correct for qualified matchers, so t1.* should be resolved to left table column type. +/// But in planner we do not distinguish such cases. +void QueryAnalyzer::updateMatchedColumnsFromJoinUsing( + QueryTreeNodesWithNames & result_matched_column_nodes_with_names, + const QueryTreeNodePtr & source_table_expression, + IdentifierResolveScope & scope) +{ + auto * nearest_query_scope = scope.getNearestQueryScope(); + auto * nearest_query_scope_query_node = nearest_query_scope ? nearest_query_scope->scope_node->as() : nullptr; + + /// If there are no parent query scope or query scope does not have join tree + if (!nearest_query_scope_query_node || !nearest_query_scope_query_node->getJoinTree()) + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "There are no table sources. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + } + + const auto & join_tree = nearest_query_scope_query_node->getJoinTree(); + + const auto * join_node = join_tree->as(); + if (join_node && join_node->isUsingJoinExpression()) + { + const auto & join_using_list = join_node->getJoinExpression()->as(); + const auto & join_using_nodes = join_using_list.getNodes(); + + for (auto & [matched_column_node, _] : result_matched_column_nodes_with_names) + { + auto & matched_column_node_typed = matched_column_node->as(); + const auto & matched_column_name = matched_column_node_typed.getColumnName(); + + for (const auto & join_using_node : join_using_nodes) + { + auto & join_using_column_node = join_using_node->as(); + const auto & join_using_column_name = join_using_column_node.getColumnName(); + + if (matched_column_name != join_using_column_name) + continue; + + const auto & join_using_column_nodes_list = join_using_column_node.getExpressionOrThrow()->as(); + const auto & join_using_column_nodes = join_using_column_nodes_list.getNodes(); + + auto it = node_to_projection_name.find(matched_column_node); + + if (hasTableExpressionInJoinTree(join_node->getLeftTableExpression(), source_table_expression)) + matched_column_node = join_using_column_nodes.at(0); + else if (hasTableExpressionInJoinTree(join_node->getRightTableExpression(), source_table_expression)) + matched_column_node = join_using_column_nodes.at(1); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot find column {} in JOIN USING section {}", + matched_column_node->dumpTree(), join_node->dumpTree()); + + matched_column_node = matched_column_node->clone(); + if (it != node_to_projection_name.end()) + node_to_projection_name.emplace(matched_column_node, it->second); + + matched_column_node->as().setColumnType(join_using_column_node.getResultType()); + } + } + } +} + /** Resolve qualified tree matcher. * * First try to match qualified identifier to expression. If qualified identifier matched expression node then @@ -4321,6 +4415,8 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu matched_columns, scope); + updateMatchedColumnsFromJoinUsing(result_matched_column_nodes_with_names, table_expression_node, scope); + return result_matched_column_nodes_with_names; } diff --git a/tests/queries/0_stateless/00722_inner_join.reference b/tests/queries/0_stateless/00722_inner_join.reference index c7ba26e3e6a..b5e8a77a20d 100644 --- a/tests/queries/0_stateless/00722_inner_join.reference +++ b/tests/queries/0_stateless/00722_inner_join.reference @@ -31,9 +31,9 @@ ┌─database─┬─name─┐ │ system │ one │ └──────────┴──────┘ -┌─database─┬─t.name─┐ -│ system │ one │ -└──────────┴────────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ 2 2 2 diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference index d2042b56f55..06f4a9cfc99 100644 --- a/tests/queries/0_stateless/02835_join_step_explain.reference +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -56,20 +56,20 @@ Header: id UInt64 rhs.id UInt64 rhs.value_1 String Actions: INPUT : 0 -> __table1.id UInt64 : 0 - INPUT :: 1 -> __table1.value_2 UInt64 : 1 - INPUT : 2 -> __table1.value_1 String : 2 + INPUT : 1 -> __table1.value_1 String : 1 + INPUT :: 2 -> __table1.value_2 UInt64 : 2 INPUT : 3 -> __table2.value_1 String : 3 INPUT :: 4 -> __table2.value_2 UInt64 : 4 INPUT : 5 -> __table2.id UInt64 : 5 ALIAS __table1.id :: 0 -> id UInt64 : 6 - ALIAS __table1.value_1 :: 2 -> value_1 String : 0 - ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 2 + ALIAS __table1.value_1 :: 1 -> value_1 String : 0 + ALIAS __table2.value_1 :: 3 -> rhs.value_1 String : 1 ALIAS __table2.id :: 5 -> rhs.id UInt64 : 3 -Positions: 6 0 3 2 +Positions: 6 0 3 1 Join (JOIN FillRightFirst) Header: __table1.id UInt64 - __table1.value_2 UInt64 __table1.value_1 String + __table1.value_2 UInt64 __table2.value_1 String __table2.value_2 UInt64 __table2.id UInt64 @@ -80,37 +80,37 @@ Positions: 6 0 3 2 Clauses: [(__table1.id, __table1.value_2) = (__table2.id, __table2.value_2)] Expression ((JOIN actions + Change column names to column identifiers)) Header: __table1.id UInt64 - __table1.value_2 UInt64 __table1.value_1 String + __table1.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 - INPUT : 1 -> value_2 UInt64 : 1 - INPUT : 2 -> value_1 String : 2 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 ALIAS id :: 0 -> __table1.id UInt64 : 3 - ALIAS value_2 :: 1 -> __table1.value_2 UInt64 : 0 - ALIAS value_1 :: 2 -> __table1.value_1 String : 1 + ALIAS value_1 :: 1 -> __table1.value_1 String : 0 + ALIAS value_2 :: 2 -> __table1.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_1) Header: id UInt64 - value_2 UInt64 value_1 String + value_2 UInt64 ReadType: Default Parts: 1 Granules: 1 Expression ((JOIN actions + Change column names to column identifiers)) Header: __table2.id UInt64 - __table2.value_2 UInt64 __table2.value_1 String + __table2.value_2 UInt64 Actions: INPUT : 0 -> id UInt64 : 0 - INPUT : 1 -> value_2 UInt64 : 1 - INPUT : 2 -> value_1 String : 2 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 ALIAS id :: 0 -> __table2.id UInt64 : 3 - ALIAS value_2 :: 1 -> __table2.value_2 UInt64 : 0 - ALIAS value_1 :: 2 -> __table2.value_1 String : 1 + ALIAS value_1 :: 1 -> __table2.value_1 String : 0 + ALIAS value_2 :: 2 -> __table2.value_2 UInt64 : 1 Positions: 3 0 1 ReadFromMergeTree (default.test_table_2) Header: id UInt64 - value_2 UInt64 value_1 String + value_2 UInt64 ReadType: Default Parts: 1 Granules: 1 diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 similarity index 94% rename from tests/queries/0_stateless/02955_analyzer_using_functional_args.reference rename to tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 index f4d8a77ef7f..7d8fcf5bce0 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 @@ -1,3 +1,5 @@ +{% for column_expression_type in ['ALIAS', 'MATERIALIZED'] -%} +{{ column_expression_type }} 1 1 369 124 123 b @@ -6,7 +8,7 @@ 3693 1231 a 1231 3693 1232 1231 1231 a a --- {echoOn } +-- { echoOn } -- USING alias column contains default in old analyzer (but both queries below should have the same result) SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; 738 ba @@ -41,9 +43,9 @@ SELECT (1, t1.*, t2.*) FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2 (1,'a',3693,0) (1,'b',369,369) SELECT t1.z, t2.z, t3.z FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; -1 0 43 -1 48 1 -124 124 1 +0 0 43 +0 48 0 +124 124 0 1232 0 1232 SELECT * FROM t1 FULL JOIN t2 USING (y) FULL JOIN t3 USING (y) ORDER BY 1,2,3 SETTINGS allow_experimental_analyzer = 1; 126 0 0 42 @@ -79,3 +81,4 @@ b DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; +{% endfor -%} diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 similarity index 86% rename from tests/queries/0_stateless/02955_analyzer_using_functional_args.sql rename to tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 index f92598c7c32..6fb88a02e5e 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 @@ -2,10 +2,16 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; -CREATE TABLE t1 (x Int16, y Int64 ALIAS x + x * 2, z ALIAS x + 1, s String) ENGINE=MergeTree() ORDER BY x; +{% for column_expression_type in ['ALIAS', 'MATERIALIZED'] %} + +-- { echoOff } + +SELECT '{{ column_expression_type }}'; + +CREATE TABLE t1 (x Int16, y Int64 {{ column_expression_type }} x + x * 2, z {{ column_expression_type }} x + 1, s String) ENGINE=MergeTree() ORDER BY x; CREATE TABLE t2 (y Int128, z Int16) ENGINE=MergeTree() ORDER BY y; -CREATE TABLE t3 (x Int16, y Int64 ALIAS x + x * 2, z ALIAS x + 1) ENGINE=MergeTree() ORDER BY x; +CREATE TABLE t3 (x Int16, y Int64 {{ column_expression_type }} x + x * 2, z {{ column_expression_type }} x + 1) ENGINE=MergeTree() ORDER BY x; INSERT INTO t1 VALUES (1231, 'a'), (123, 'b'); @@ -26,7 +32,7 @@ SELECT * FROM t1 INNER JOIN t3 USING (y) SETTINGS allow_experimental_analyzer = SELECT * FROM t3 INNER JOIN t1 USING (y, z) SETTINGS allow_experimental_analyzer = 1; SELECT s FROM t1 INNER JOIN t3 USING (y); --- {echoOn } +-- { echoOn } -- USING alias column contains default in old analyzer (but both queries below should have the same result) SELECT y * 2, s || 'a' FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL SETTINGS allow_experimental_analyzer = 1; SELECT y * 2, s || 'a' FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; @@ -51,7 +57,8 @@ SELECT y FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) SELECT s FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; - DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; + +{% endfor %} From 33169e72df181915ad058af7fb7d60c53805d84b Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Mar 2024 10:34:10 +0000 Subject: [PATCH 024/154] Fix AMBIGUOUS_IDENTIFIER with join_use_nulls https://github.com/ClickHouse/ClickHouse/issues/59067#issuecomment-1912500556 --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 17 ++++++- ..._join_use_nulls_column_not_found.reference | 12 +++++ ...alyzer_join_use_nulls_column_not_found.sql | 50 +++++++++++++++++++ 3 files changed, 78 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 6a5efa6532f..c8f3f8b884b 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3302,6 +3302,21 @@ QueryTreeNodePtr checkIsMissedObjectJSONSubcolumn(const QueryTreeNodePtr & left_ return {}; } +/// Compare resolved identifiers considering columns that become nullable after JOIN +static bool resolvedIdenfiersFromJoinAreEquals( + const QueryTreeNodePtr & left_resolved_identifier, + const QueryTreeNodePtr & right_resolved_identifier, + const IdentifierResolveScope & scope) +{ + auto lit = scope.nullable_join_columns.find(left_resolved_identifier); + const auto & left_resolved_to_compare = lit != scope.nullable_join_columns.end() ? lit->second : left_resolved_identifier; + + auto rit = scope.nullable_join_columns.find(right_resolved_identifier); + const auto & right_resolved_to_compare = rit != scope.nullable_join_columns.end() ? rit->second : right_resolved_identifier; + + return left_resolved_to_compare->isEqual(*right_resolved_to_compare, IQueryTreeNode::CompareOptions{.compare_aliases = false}); +} + QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) @@ -3438,7 +3453,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo resolved_identifier = std::move(result_column_node); } - else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false})) + else if (resolvedIdenfiersFromJoinAreEquals(left_resolved_identifier, right_resolved_identifier, scope)) { const auto & identifier_path_part = identifier_lookup.identifier.front(); auto * left_resolved_identifier_column = left_resolved_identifier->as(); diff --git a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference index a5c8806279f..d53d7235814 100644 --- a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference +++ b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.reference @@ -1,2 +1,14 @@ +-- 3 +-- 3 +-- +0 +-- +\N \N +-- + a +a a +-- +a a +\N \N diff --git a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql index 7101d5c89ae..5d1afa2a274 100644 --- a/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql +++ b/tests/queries/0_stateless/02973_analyzer_join_use_nulls_column_not_found.sql @@ -1,5 +1,7 @@ SET join_use_nulls = 1; +SELECT '--'; + select c FROM ( select d2.c @@ -9,6 +11,8 @@ select c FROM ( ) ; +SELECT '--'; + with d1 as ( select 1 as a, @@ -29,3 +33,49 @@ joined as ( ) select c from joined; + +SELECT '--'; + +WITH + a AS ( SELECT 0 AS key, 'a' AS acol ), + b AS ( SELECT 2 AS key ) +SELECT a.key +FROM b +LEFT JOIN a ON 1 +LEFT JOIN a AS a1 ON 1 +; + +SELECT '--'; + +WITH + a AS ( SELECT 0 AS key, 'a' AS acol ), + b AS ( SELECT 2 AS key ) +SELECT a.acol, a1.acol +FROM b +LEFT JOIN a ON a.key = b.key +LEFT JOIN a AS a1 ON a1.key = a.key +; +SELECT '--'; + +WITH + a AS ( SELECT 0 AS key, 'a' AS acol ), + b AS ( SELECT 2 AS key ) +SELECT a.acol, a1.acol +FROM b +FULL JOIN a ON a.key = b.key +FULL JOIN a AS a1 ON a1.key = a.key +ORDER BY 1 +SETTINGS join_use_nulls = 0 +; + +SELECT '--'; + +WITH + a AS ( SELECT 0 AS key, 'a' AS acol ), + b AS ( SELECT 2 AS key ) +SELECT a.acol, a1.acol +FROM b +FULL JOIN a ON a.key = b.key +FULL JOIN a AS a1 ON a1.key = a.key +ORDER BY 1 +; From e002e42805a7832f0a54766bc483e6d1fdcf877e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Mar 2024 10:38:50 +0000 Subject: [PATCH 025/154] remove old nodes from node_to_projection_name when node become nullable --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c8f3f8b884b..0ca9108bea0 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3560,13 +3560,17 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo if (scope.join_use_nulls) { - auto it = node_to_projection_name.find(resolved_identifier); + auto projection_name_it = node_to_projection_name.find(resolved_identifier); auto nullable_resolved_identifier = convertJoinedColumnTypeToNullIfNeeded(resolved_identifier, join_kind, resolved_side, scope); if (nullable_resolved_identifier) { resolved_identifier = nullable_resolved_identifier; - if (it != node_to_projection_name.end()) - node_to_projection_name.emplace(resolved_identifier, it->second); + /// Set the same projection name for new nullable node + if (projection_name_it != node_to_projection_name.end()) + { + node_to_projection_name.erase(projection_name_it); + node_to_projection_name.emplace(resolved_identifier, projection_name_it->second); + } } } @@ -4686,7 +4690,10 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I node = nullable_node; /// Set the same projection name for new nullable node if (projection_name_it != node_to_projection_name.end()) + { + node_to_projection_name.erase(projection_name_it); node_to_projection_name.emplace(node, projection_name_it->second); + } } } } From 6d84ccd9f163427a9d169b6d972f8f5e62db5237 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Mar 2024 11:12:36 +0000 Subject: [PATCH 026/154] fix tests/ci/style_check.py for removed files --- tests/ci/style_check.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 797c0ef12d0..f8e49202c8e 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -100,6 +100,8 @@ def is_python(file: Union[Path, str]) -> bool: """returns if the changed file in the repository is python script""" # WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04, # and `Support os.PathLike values in magic.from_file` is only from 0.4.25 + if not os.path.exists(os.path.join(REPO_COPY, file)): + return False return bool( magic.from_file(os.path.join(REPO_COPY, file), mime=True) == "text/x-script.python" @@ -110,6 +112,8 @@ def is_shell(file: Union[Path, str]) -> bool: """returns if the changed file in the repository is shell script""" # WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04, # and `Support os.PathLike values in magic.from_file` is only from 0.4.25 + if not os.path.exists(os.path.join(REPO_COPY, file)): + return False return bool( magic.from_file(os.path.join(REPO_COPY, file), mime=True) == "text/x-shellscript" From 81fd904b54327578c5f66e4452c77456178621a9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 00:05:21 +0100 Subject: [PATCH 027/154] I hate REPLACE_RANGE and actual_part_name --- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 65 ++++++++++++------- ..._move_partition_inactive_replica.reference | 8 +++ .../02916_move_partition_inactive_replica.sql | 46 +++++++++++++ 4 files changed, 98 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02916_move_partition_inactive_replica.reference create mode 100644 tests/queries/0_stateless/02916_move_partition_inactive_replica.sql diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 054c576cfc5..7693f34cc1e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -93,6 +93,7 @@ struct ReplicatedMergeTreeLogEntryData MergeTreeDataPartFormat new_part_format; String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. + mutable std::unordered_set replace_range_actual_new_part_names; /// Same as above, but for REPLACE_RANGE UUID new_part_uuid = UUIDHelpers::Nil; Strings source_parts; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 42f564f40da..ee4ed87d456 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -342,6 +342,11 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( /// NOTE actual_new_part_name is very confusing and error-prone. This approach must be fixed. removeCoveredPartsFromMutations(entry->actual_new_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } + for (const auto & actual_part : entry->replace_range_actual_new_part_names) + { + LOG_TEST(log, "Entry {} has actual new part name {}, removing it from mutations", entry->znode_name, actual_part); + removeCoveredPartsFromMutations(actual_part, /*remove_part = */ false, /*remove_covered_parts = */ true); + } LOG_TEST(log, "Adding parts [{}] to current parts", fmt::join(entry_virtual_parts, ", ")); @@ -1180,9 +1185,9 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry if (entry_for_same_part_it != future_parts.end()) { const LogEntry & another_entry = *entry_for_same_part_it->second; - constexpr auto fmt_string = "Not executing log entry {} of type {} for part {} " + constexpr auto fmt_string = "Not executing log entry {} of type {} for part {} (actual part {})" "because another log entry {} of type {} for the same part ({}) is being processed."; - LOG_INFO(LogToStr(out_reason, log), fmt_string, entry.znode_name, entry.type, entry.new_part_name, + LOG_INFO(LogToStr(out_reason, log), fmt_string, entry.znode_name, entry.type, entry.new_part_name, new_part_name, another_entry.znode_name, another_entry.type, another_entry.new_part_name); return true; @@ -1198,6 +1203,7 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry auto result_part = MergeTreePartInfo::fromPartName(new_part_name, format_version); /// It can slow down when the size of `future_parts` is large. But it can not be large, since background pool is limited. + /// (well, it can actually, thanks to REPLACE_RANGE, but it's a rare case) for (const auto & future_part_elem : future_parts) { auto future_part = MergeTreePartInfo::fromPartName(future_part_elem.first, format_version); @@ -1608,26 +1614,39 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName( std::unique_lock & state_lock, std::vector & covered_entries_to_wait) { - if (!entry.actual_new_part_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry actual part isn't empty yet. This is a bug."); + if (actual_part_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Actual part name is empty"); - entry.actual_new_part_name = actual_part_name; + if (!entry.actual_new_part_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} actual part isn't empty yet: '{}'. This is a bug.", + entry.znode_name, entry.actual_new_part_name); + + auto actual_part_info = MergeTreePartInfo::fromPartName(actual_part_name, queue.format_version); + for (const auto & other_part_name : entry.replace_range_actual_new_part_names) + if (!MergeTreePartInfo::fromPartName(other_part_name, queue.format_version).isDisjoint(actual_part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} already has actual part {} non-disjoint with {}. This is a bug.", + entry.actual_new_part_name, other_part_name, actual_part_name); /// Check if it is the same (and already added) part. - if (entry.actual_new_part_name == entry.new_part_name) + if (actual_part_name == entry.new_part_name) return; - if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second) + if (!queue.future_parts.emplace(actual_part_name, entry.shared_from_this()).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. " "It happened on attempt to execute {}: {}", - entry.actual_new_part_name, entry.znode_name, entry.toString()); + actual_part_name, entry.znode_name, entry.toString()); + + if (entry.type == LogEntry::REPLACE_RANGE) + entry.replace_range_actual_new_part_names.insert(actual_part_name); + else + entry.actual_new_part_name = actual_part_name; for (LogEntryPtr & covered_entry : covered_entries_to_wait) { if (&entry == covered_entry.get()) continue; - LOG_TRACE(queue.log, "Waiting for {} producing {} to finish before executing {} producing not disjoint part {}", - covered_entry->znode_name, covered_entry->new_part_name, entry.znode_name, entry.new_part_name); + LOG_TRACE(queue.log, "Waiting for {} producing {} to finish before executing {} producing not disjoint part {} (actual part {})", + covered_entry->znode_name, covered_entry->new_part_name, entry.znode_name, entry.new_part_name, actual_part_name); covered_entry->execution_complete.wait(state_lock, [&covered_entry] { return !covered_entry->currently_executing; }); } } @@ -1646,25 +1665,27 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() entry->currently_executing = false; entry->execution_complete.notify_all(); - for (const String & new_part_name : entry->getVirtualPartNames(queue.format_version)) + auto erase_and_check = [this](const String & part_name) { - if (!queue.future_parts.erase(new_part_name)) + if (!queue.future_parts.erase(part_name)) { - LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name); + LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", part_name); assert(false); } - } + }; + + for (const String & new_part_name : entry->getVirtualPartNames(queue.format_version)) + erase_and_check(new_part_name); if (!entry->actual_new_part_name.empty()) - { - if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name)) - { - LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name); - assert(false); - } + erase_and_check(entry->actual_new_part_name); - entry->actual_new_part_name.clear(); - } + entry->actual_new_part_name.clear(); + + for (const auto & actual_part : entry->replace_range_actual_new_part_names) + erase_and_check(actual_part); + + entry->replace_range_actual_new_part_names.clear(); } diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference new file mode 100644 index 00000000000..b64cc6289d5 --- /dev/null +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference @@ -0,0 +1,8 @@ +all_0_0_0 0 +all_0_1_1 1 +all_1_1_0 0 +all_2_2_0 0 +all_2_3_1 1 +all_3_3_0 0 +0 +40 1580 diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql new file mode 100644 index 00000000000..d597c037612 --- /dev/null +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +create database if not exists shard_0; +create database if not exists shard_1; + +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +drop table if exists shard_0.to; +drop table if exists shard_1.to; + +create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; +create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +system stop merges shard_0.from_1; +insert into shard_0.from_1 select number + 20 from numbers(10); +insert into shard_0.from_1 select number + 30 from numbers(10); + +insert into shard_0.from_1 select number + 40 from numbers(10); +insert into shard_0.from_1 select number + 50 from numbers(10); + +system sync replica shard_1.from_1; + +create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; + +create table shard_1.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; + +detach table shard_1.to; + +alter table shard_0.from_1 on cluster test_cluster_two_shards_different_databases move partition tuple() to table shard_0.to format Null settings distributed_ddl_output_mode='never_throw', distributed_ddl_task_timeout = 1; + +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +OPTIMIZE TABLE shard_0.to; +OPTIMIZE TABLE shard_0.to; +select name, active from system.parts where database='shard_0' and table='to' order by name; + +system restart replica shard_0.to; + +select sleep(3); + +attach table shard_1.to; +system sync replica shard_1.to; +select count(), sum(x) from shard_1.to; + +drop table if exists shard_0.to; +drop table if exists shard_1.to; From 7db3d21ab7887a486950cb413358e9650369096e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 21 Mar 2024 14:19:17 +0100 Subject: [PATCH 028/154] Update src/Processors/Transforms/FillingTransform.cpp Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 54812405610..879d61bb241 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -250,7 +250,7 @@ FillingTransform::FillingTransform( if (!unique_positions.insert(pos).second) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Multiple WITH FILL for identical expressions is not supported in ORDER BY"); if (ordinary_sort_positions.contains(pos)) - throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "ORDER BY containing the same expression with and without WITH FILL is not supported"); + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "ORDER BY containing the same expression with and without WITH FILL modifier is not supported"); } if (use_with_fill_by_sorting_prefix) From 188bc3cbaf0eb64ed6bd41e3ff71454ee53b70e6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 21 Mar 2024 15:00:27 +0100 Subject: [PATCH 029/154] Update some tests --- .../test_backward_compatibility/test_convert_ordinary.py | 2 +- .../test_backward_compatibility/test_cte_distributed.py | 4 ++-- .../integration/test_backward_compatibility/test_functions.py | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 8e7d773ad2c..ff296d01c18 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -9,7 +9,7 @@ node = cluster.add_instance( stay_alive=True, with_zookeeper=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index e612bf2989a..592ec4ecbcc 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, use_old_analyzer=True) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,7 +11,7 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 0217c46a660..d95c58085f3 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -9,13 +9,13 @@ from helpers.cluster import ClickHouseCluster, CLICKHOUSE_CI_MIN_TESTED_VERSION from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -upstream = cluster.add_instance("upstream", allow_analyzer=False) +upstream = cluster.add_instance("upstream", use_old_analyzer=True) backward = cluster.add_instance( "backward", image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - allow_analyzer=False, + use_old_analyzer=True, ) From 466cfe29b27906bc458ddb2108e994e17a2534b9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 21 Mar 2024 15:01:08 +0100 Subject: [PATCH 030/154] Fix tests with clickhouse-local #ci_set_analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 0414ea93f94..70990cc87ff 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7134,7 +7134,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, ColumnDescription column = insert_columns.get(*insert_column_name_it); /// Change ephemeral columns to default columns. column.default_desc.kind = ColumnDefaultKind::Default; - structure_hint.add(insert_columns.get(*insert_column_name_it)); + structure_hint.add(std::move(column)); } } diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh index acf0608acc5..eb4a91bd850 100755 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||: -$CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }" 2>&1 | grep -o 'Missing columns' +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 0 -- { serverError 47 }" 2>&1 | grep -o 'Missing columns' From 46ab53ef797023fa90ff1d15a91b02d77222e400 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Mar 2024 14:03:43 +0000 Subject: [PATCH 031/154] fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9aeb9cd6f23..12adb762a9a 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3569,7 +3569,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo /// Set the same projection name for new nullable node if (projection_name_it != node_to_projection_name.end()) { - node_to_projection_name.erase(projection_name_it); node_to_projection_name.emplace(resolved_identifier, projection_name_it->second); } } @@ -4692,7 +4691,6 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I /// Set the same projection name for new nullable node if (projection_name_it != node_to_projection_name.end()) { - node_to_projection_name.erase(projection_name_it); node_to_projection_name.emplace(node, projection_name_it->second); } } From 1e41752c76e3d8a361f36e4de7016c7833f26023 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 21 Mar 2024 15:07:56 +0100 Subject: [PATCH 032/154] Fix the last test with clickhouse local --- tests/queries/0_stateless/01600_detach_permanently.reference | 1 + tests/queries/0_stateless/01600_detach_permanently.sh | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01600_detach_permanently.reference b/tests/queries/0_stateless/01600_detach_permanently.reference index 98ed3b6762d..4fda43df162 100644 --- a/tests/queries/0_stateless/01600_detach_permanently.reference +++ b/tests/queries/0_stateless/01600_detach_permanently.reference @@ -96,6 +96,7 @@ View can be reattached test for MV with inner table MV is working 1 +1 View can be reattached ################## DETACH DATABASE is not implemented (proper error) diff --git a/tests/queries/0_stateless/01600_detach_permanently.sh b/tests/queries/0_stateless/01600_detach_permanently.sh index 036706d2fe8..6721dbf3015 100755 --- a/tests/queries/0_stateless/01600_detach_permanently.sh +++ b/tests/queries/0_stateless/01600_detach_permanently.sh @@ -111,7 +111,9 @@ clickhouse_local "INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" clickhouse_local "SELECT if(count() = 10, 'MV is working', 'MV failed') FROM db_ordinary.src_mv_with_inner" clickhouse_local "DETACH VIEW db_ordinary.src_mv_with_inner PERMANENTLY; INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" --stacktrace -clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner" 2>&1 | grep -c "db_ordinary.src_mv_with_inner does not exist" +clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS allow_experimental_analyzer = 0" 2>&1 | grep -c "db_ordinary.src_mv_with_inner does not exist" +clickhouse_local "SELECT if(count() = 10, 'MV can be detached permanently', 'MV detach failed') FROM db_ordinary.src_mv_with_inner SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -c "Unknown table expression identifier 'db_ordinary.src_mv_with_inner'" + ## Quite silly: ATTACH MATERIALIZED VIEW don't work with short syntax (w/o select), but i can attach it using ATTACH TABLE ... clickhouse_local "ATTACH TABLE db_ordinary.src_mv_with_inner" clickhouse_local "INSERT INTO db_ordinary.src SELECT * FROM numbers(10)" From 20a860cf1912f76bf60f18aa8ccacc4b812077cc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 21 Mar 2024 16:18:49 +0100 Subject: [PATCH 033/154] Update ci_config.py --- tests/ci/ci_config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7c213da27ec..bd64712a162 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -94,7 +94,7 @@ class JobNames(metaclass=WithIter): STATELESS_TEST_MSAN = "Stateless tests (msan)" STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" STATELESS_TEST_ANALYZER_S3_REPLICATED_RELEASE = ( - "Stateless tests (release, analyzer, s3, DatabaseReplicated)" + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)" ) # merged into STATELESS_TEST_ANALYZER_S3_REPLICATED_RELEASE: # STATELESS_TEST_ANALYZER_RELEASE = "Stateless tests (release, analyzer)" @@ -127,7 +127,7 @@ class JobNames(metaclass=WithIter): INTEGRATION_TEST = "Integration tests (release)" INTEGRATION_TEST_ASAN = "Integration tests (asan)" - INTEGRATION_TEST_ASAN_ANALYZER = "Integration tests (asan, analyzer)" + INTEGRATION_TEST_ASAN_ANALYZER = "Integration tests (asan, old analyzer)" INTEGRATION_TEST_TSAN = "Integration tests (tsan)" INTEGRATION_TEST_ARM = "Integration tests (aarch64)" INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" From 6a19b77d9ea4545bf7080896a760022ed8fd9410 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 21 Mar 2024 18:39:05 +0000 Subject: [PATCH 034/154] init --- .../settings.md | 4 +- docs/en/sql-reference/statements/undrop.md | 48 +++++++------------ src/Interpreters/DatabaseCatalog.cpp | 4 +- src/Interpreters/InterpreterUndropQuery.cpp | 6 ++- tests/integration/test_undrop_query/test.py | 42 ++++++++++++---- .../0_stateless/02681_undrop_query.sql | 2 +- 6 files changed, 60 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 07c9a2b88ab..f20dcb9025e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -933,9 +933,9 @@ Hard limit is configured via system tools ## database_atomic_delay_before_drop_table_sec {#database_atomic_delay_before_drop_table_sec} -Sets the delay before remove table data in seconds. If the query has `SYNC` modifier, this setting is ignored. +The delay before a table data is dropped in seconds. If the `DROP TABLE` query has a `SYNC` modifier, this setting is ignored. -Default value: `480` (8 minute). +Default value: `480` (8 minutes). ## database_catalog_unused_dir_hide_timeout_sec {#database_catalog_unused_dir_hide_timeout_sec} diff --git a/docs/en/sql-reference/statements/undrop.md b/docs/en/sql-reference/statements/undrop.md index 40ac1ab4f99..4b138bfe679 100644 --- a/docs/en/sql-reference/statements/undrop.md +++ b/docs/en/sql-reference/statements/undrop.md @@ -13,13 +13,6 @@ a system table called `system.dropped_tables`. If you have a materialized view without a `TO` clause associated with the dropped table, then you will also have to UNDROP the inner table of that view. -:::note -UNDROP TABLE is experimental. To use it add this setting: -```sql -set allow_experimental_undrop_table_query = 1; -``` -::: - :::tip Also see [DROP TABLE](/docs/en/sql-reference/statements/drop.md) ::: @@ -32,60 +25,53 @@ UNDROP TABLE [db.]name [UUID ''] [ON CLUSTER cluster] **Example** -``` sql -set allow_experimental_undrop_table_query = 1; -``` - ```sql -CREATE TABLE undropMe +CREATE TABLE tab ( `id` UInt8 ) ENGINE = MergeTree -ORDER BY id -``` +ORDER BY id; + +DROP TABLE tab; -```sql -DROP TABLE undropMe -``` -```sql SELECT * FROM system.dropped_tables -FORMAT Vertical +FORMAT Vertical; ``` + ```response Row 1: ────── index: 0 database: default -table: undropMe +table: tab uuid: aa696a1a-1d70-4e60-a841-4c80827706cc engine: MergeTree -metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.undropMe.aa696a1a-1d70-4e60-a841-4c80827706cc.sql +metadata_dropped_path: /var/lib/clickhouse/metadata_dropped/default.tab.aa696a1a-1d70-4e60-a841-4c80827706cc.sql table_dropped_time: 2023-04-05 14:12:12 1 row in set. Elapsed: 0.001 sec. ``` + ```sql -UNDROP TABLE undropMe -``` -```response -Ok. -``` -```sql +UNDROP TABLE tab; + SELECT * FROM system.dropped_tables -FORMAT Vertical -``` +FORMAT Vertical; + ```response Ok. 0 rows in set. Elapsed: 0.001 sec. ``` + ```sql -DESCRIBE TABLE undropMe -FORMAT Vertical +DESCRIBE TABLE tab +FORMAT Vertical; ``` + ```response Row 1: ────── diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 3dc700e9f93..ec6c8b5924f 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -1143,7 +1143,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) TableMarkedAsDropped dropped_table; { std::lock_guard lock(tables_marked_dropped_mutex); - time_t latest_drop_time = std::numeric_limits::min(); + auto latest_drop_time = std::numeric_limits::min(); auto it_dropped_table = tables_marked_dropped.end(); for (auto it = tables_marked_dropped.begin(); it != tables_marked_dropped.end(); ++it) { @@ -1168,7 +1168,7 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) } if (it_dropped_table == tables_marked_dropped.end()) throw Exception(ErrorCodes::UNKNOWN_TABLE, - "The drop task of table {} is in progress, has been dropped or the database engine doesn't support it", + "Table {} is being dropped, has been dropped, or the database engine does not support UNDROP", table_id.getNameForLogs()); latest_metadata_dropped_path = it_dropped_table->metadata_path; String table_metadata_path = getPathForMetadata(it_dropped_table->table_id); diff --git a/src/Interpreters/InterpreterUndropQuery.cpp b/src/Interpreters/InterpreterUndropQuery.cpp index 1f2ff4b4461..920df3d6aed 100644 --- a/src/Interpreters/InterpreterUndropQuery.cpp +++ b/src/Interpreters/InterpreterUndropQuery.cpp @@ -18,14 +18,16 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } -InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) +InterpreterUndropQuery::InterpreterUndropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_) + , query_ptr(query_ptr_) { } - BlockIO InterpreterUndropQuery::execute() { getContext()->checkAccess(AccessType::UNDROP_TABLE); + auto & undrop = query_ptr->as(); if (!undrop.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext())) { diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index 590a5690e55..2aa5bdacf9d 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -29,30 +29,56 @@ def test_undrop_drop_and_undrop_loop(started_cluster): logging.info( "random_sec: " + random_sec.__str__() + ", table_uuid: " + table_uuid ) + node.query( - "create table test_undrop_loop" + "CREATE TABLE test_undrop_loop" + count.__str__() + " UUID '" + table_uuid - + "' (id Int32) Engine=MergeTree() order by id;" + + "' (id Int32) ENGINE = MergeTree() ORDER BY id;" ) - node.query("drop table test_undrop_loop" + count.__str__() + ";") + + node.query("DROP TABLE test_undrop_loop" + count.__str__() + ";") + time.sleep(random_sec) + if random_sec >= 5: error = node.query_and_get_error( - "undrop table test_undrop_loop" + "UNDROP TABLE test_undrop_loop" + count.__str__() - + " uuid '" + + " UUID '" + table_uuid + "';" ) assert "UNKNOWN_TABLE" in error - else: + elif random_sec <= 3: + # (*) node.query( - "undrop table test_undrop_loop" + "UNDROP TABLE test_undrop_loop" + count.__str__() - + " uuid '" + + " UUID '" + table_uuid + "';" ) count = count + 1 + else: + try: + node.query( + "UNDROP TABLE test_undrop_loop" + + count.__str__() + + " UUID '" + + table_uuid + + "';" + ) + count = count + 1 + except: + error = node.query_and_get_error( + "UNDROP TABLE test_undrop_loop" + + count.__str__() + + " UUID '" + + table_uuid + + "';" + ) + assert "UNKNOWN_TABLE" in error + # ignore random_sec = 4 result to account for communication delay with the database. + # if we don't do that, then the second case (*) may find the table already dropped and receive an unexpected exception from the database (Bug #55167) diff --git a/tests/queries/0_stateless/02681_undrop_query.sql b/tests/queries/0_stateless/02681_undrop_query.sql index 66447fc6c44..d038a383690 100644 --- a/tests/queries/0_stateless/02681_undrop_query.sql +++ b/tests/queries/0_stateless/02681_undrop_query.sql @@ -85,5 +85,5 @@ drop table 02681_undrop_multiple; select table from system.dropped_tables where table = '02681_undrop_multiple' limit 1; undrop table 02681_undrop_multiple; select * from 02681_undrop_multiple order by id; -undrop table 02681_undrop_multiple; -- { serverError 57 } +undrop table 02681_undrop_multiple; -- { serverError TABLE_ALREADY_EXISTS } drop table 02681_undrop_multiple sync; From 2658cca513b43a910f570f8adf957f2c63f143a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 19:42:46 +0100 Subject: [PATCH 035/154] Update 02916_move_partition_inactive_replica.sql --- .../0_stateless/02916_move_partition_inactive_replica.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql index d597c037612..25dd0365d97 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -32,7 +32,7 @@ drop table if exists shard_0.from_1; drop table if exists shard_1.from_1; OPTIMIZE TABLE shard_0.to; OPTIMIZE TABLE shard_0.to; -select name, active from system.parts where database='shard_0' and table='to' order by name; +select name, active from system.parts where database='shard_0' and table='to' and active order by name; system restart replica shard_0.to; From 3745df45900dd89348baa46b10ec7c55967d548c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 19:43:12 +0100 Subject: [PATCH 036/154] Update 02916_move_partition_inactive_replica.reference --- .../02916_move_partition_inactive_replica.reference | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference index b64cc6289d5..361a0d1e9bb 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference @@ -1,8 +1,4 @@ -all_0_0_0 0 all_0_1_1 1 -all_1_1_0 0 -all_2_2_0 0 all_2_3_1 1 -all_3_3_0 0 0 40 1580 From 9d5cabb26d1eb7f1347c62676cdd6b7eb3433eea Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Thu, 7 Mar 2024 15:38:46 +0800 Subject: [PATCH 037/154] fix csv format not support tuple --- .../Serializations/SerializationTuple.cpp | 34 +++++-------------- src/Formats/EscapingRuleUtils.cpp | 4 +-- .../02977_csv_format_support_tuple.reference | 6 ++++ .../02977_csv_format_support_tuple.sql | 4 +++ 4 files changed, 20 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02977_csv_format_support_tuple.reference create mode 100644 tests/queries/0_stateless/02977_csv_format_support_tuple.sql diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 399ad870d60..5fb4337cdfe 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -526,36 +527,17 @@ void SerializationTuple::serializeTextXML(const IColumn & column, size_t row_num void SerializationTuple::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - for (size_t i = 0; i < elems.size(); ++i) - { - if (i != 0) - writeChar(settings.csv.tuple_delimiter, ostr); - elems[i]->serializeTextCSV(extractElementColumn(column, i), row_num, ostr, settings); - } + WriteBufferFromOwnString wb; + serializeText(column, row_num, wb, settings); + writeCSV(wb.str(), ostr); } void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - addElementSafe(elems.size(), column, [&] - { - const size_t size = elems.size(); - for (size_t i = 0; i < size; ++i) - { - if (i != 0) - { - skipWhitespaceIfAny(istr); - assertChar(settings.csv.tuple_delimiter, istr); - skipWhitespaceIfAny(istr); - } - - auto & element_column = extractElementColumn(column, i); - if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) - SerializationNullable::deserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i]); - else - elems[i]->deserializeTextCSV(element_column, istr, settings); - } - return true; - }); + String s; + readCSV(s, istr, settings.csv); + ReadBufferFromString rb(s); + deserializeText(column, rb, settings, true); } bool SerializationTuple::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 577988871f3..c0ff67dc462 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -303,8 +303,8 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet /// Try to determine the type of value inside quotes auto type = tryInferDataTypeForSingleField(data, format_settings); - /// If we couldn't infer any type or it's tuple in quotes or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. - if (!type || isTuple(type) || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) + /// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string. + if (!type || (isNumber(type) && !format_settings.csv.try_infer_numbers_from_strings)) return std::make_shared(); return type; diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference new file mode 100644 index 00000000000..11902398999 --- /dev/null +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference @@ -0,0 +1,6 @@ +Nullable(String) +Nullable(Int64) +Array(Nullable(String)) +Map(String, Nullable(Int64)) +Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) +20240305 1 ['s','d'] {'a':2} ('222',33,{'abc':5}) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql new file mode 100644 index 00000000000..b4726391925 --- /dev/null +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql @@ -0,0 +1,4 @@ +-- Tags: no-fasttest +insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)); +desc file('02977_1.csv'); +select * from file('02977_1.csv') settings max_threads=1; From 5ca0699859163f95259b0c38aae6aae42fd2ead5 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Fri, 15 Mar 2024 18:10:02 +0800 Subject: [PATCH 038/154] fix tryDeserializeTextCSV --- .../Serializations/SerializationTuple.cpp | 33 +++--------------- tests/queries/0_stateless/00300_csv.reference | 6 ++-- .../0_stateless/00309_formats.reference | Bin 18537 -> 18541 bytes .../01016_input_null_as_default.sh | 4 +-- 4 files changed, 10 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 5fb4337cdfe..5179b52a432 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -542,34 +542,11 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, bool SerializationTuple::tryDeserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - return addElementSafe(elems.size(), column, [&] - { - const size_t size = elems.size(); - for (size_t i = 0; i < size; ++i) - { - if (i != 0) - { - skipWhitespaceIfAny(istr); - if (!checkChar(settings.csv.tuple_delimiter, istr)) - return false; - skipWhitespaceIfAny(istr); - } - - auto & element_column = extractElementColumn(column, i); - if (settings.null_as_default && !isColumnNullableOrLowCardinalityNullable(element_column)) - { - if (!SerializationNullable::tryDeserializeNullAsDefaultOrNestedTextCSV(element_column, istr, settings, elems[i])) - return false; - } - else - { - if (!elems[i]->tryDeserializeTextCSV(element_column, istr, settings)) - return false; - } - } - - return true; - }); + String s; + if (!tryReadCSV(s, istr, settings.csv)) + return false; + ReadBufferFromString rb(s); + return tryDeserializeText(column, rb, settings, true); } void SerializationTuple::enumerateStreams( diff --git a/tests/queries/0_stateless/00300_csv.reference b/tests/queries/0_stateless/00300_csv.reference index 42cd22078c4..210dcf51cc7 100644 --- a/tests/queries/0_stateless/00300_csv.reference +++ b/tests/queries/0_stateless/00300_csv.reference @@ -1,11 +1,11 @@ -"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline +"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline here" "x","y","z","a","b" -"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline +"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline here" "x","y","z","a","b" "String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String" -"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline +"Hello, ""World""",123,"[1,2,3]","(456,['abc','def']","Newline here" 0,"0","[]","2000-01-01","2000-01-01 00:00:00" 1,"1","[0]","2000-01-02","2000-01-01 00:00:01" diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index e637ee0363a7b35152a155ae3fa73a4f451d5148..e4913d028746c6eb79ca72c1ac85d26f966b4247 100644 GIT binary patch delta 33 pcmaDkf${AG#tjE#g_JZ54Rq9v3=9l(4GeV+4b?R_pOc-f1pvJ>3Mc>o delta 29 lcmaDmf$`-8#tjE#`3wzol#C1v40H_)bqx(SUy_}z1pu6+35fsz diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index 24d93b2703c..078eab4d3f6 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -11,8 +11,8 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE default_by_other_column (a Float32 DEFA echo 'CSV' echo '\N, 1, \N, "2019-07-22", "[10, 20, 30]", \N -1, world, 3, "2019-07-23", \N, tuple, 3.14 -2, \N, 123, \N, "[]", test, 2.71828 +1, world, 3, "2019-07-23", \N, "(tuple, 3.14)" +2, \N, 123, \N, "[]", "(test, 2.71828)" 3, \N, \N, \N, \N, \N' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; From 1a65aebf81f4df7271bb575babdbdda80f3d0597 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Mon, 18 Mar 2024 16:04:11 +0800 Subject: [PATCH 039/154] fix test failure --- tests/queries/0_stateless/00300_csv.reference | 2 +- .../0_stateless/00309_formats.reference | Bin 18541 -> 18545 bytes .../01016_input_null_as_default.sh | 4 ++-- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00300_csv.reference b/tests/queries/0_stateless/00300_csv.reference index 210dcf51cc7..e7966a9e8d9 100644 --- a/tests/queries/0_stateless/00300_csv.reference +++ b/tests/queries/0_stateless/00300_csv.reference @@ -5,7 +5,7 @@ here" here" "x","y","z","a","b" "String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String" -"Hello, ""World""",123,"[1,2,3]","(456,['abc','def']","Newline +"Hello, ""World""",123,"[1,2,3]","(456,['abc','def'])","Newline here" 0,"0","[]","2000-01-01","2000-01-01 00:00:00" 1,"1","[0]","2000-01-02","2000-01-01 00:00:01" diff --git a/tests/queries/0_stateless/00309_formats.reference b/tests/queries/0_stateless/00309_formats.reference index e4913d028746c6eb79ca72c1ac85d26f966b4247..5c0ae4d2c3b4c65d9f9767ba08e902ad85e3e0e8 100644 GIT binary patch delta 34 qcmaDmf$`%6#tn5!!b%!Ore-?oMg|53x(1fIhQ{ien|qZWr~&}N&I&#N delta 30 mcmex3f${AG#tn5!{6?l`I!Zp diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index 078eab4d3f6..8cf89e73d92 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -11,8 +11,8 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE default_by_other_column (a Float32 DEFA echo 'CSV' echo '\N, 1, \N, "2019-07-22", "[10, 20, 30]", \N -1, world, 3, "2019-07-23", \N, "(tuple, 3.14)" -2, \N, 123, \N, "[]", "(test, 2.71828)" +1, world, 3, "2019-07-23", \N, "('tuple', 3.14)" +2, \N, 123, \N, "[]", "('test', 2.71828)" 3, \N, \N, \N, \N, \N' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; From 6117d421f991df121a15d67b1cb7384388cef740 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Mon, 18 Mar 2024 17:02:45 +0800 Subject: [PATCH 040/154] fix test failure --- tests/queries/0_stateless/01016_input_null_as_default.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01016_input_null_as_default.sh b/tests/queries/0_stateless/01016_input_null_as_default.sh index 8cf89e73d92..8d6a9a07435 100755 --- a/tests/queries/0_stateless/01016_input_null_as_default.sh +++ b/tests/queries/0_stateless/01016_input_null_as_default.sh @@ -11,8 +11,8 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE default_by_other_column (a Float32 DEFA echo 'CSV' echo '\N, 1, \N, "2019-07-22", "[10, 20, 30]", \N -1, world, 3, "2019-07-23", \N, "('tuple', 3.14)" -2, \N, 123, \N, "[]", "('test', 2.71828)" +1, world, 3, "2019-07-23", \N, "('\''tuple'\'', 3.14)" +2, \N, 123, \N, "[]", "('\''test'\'', 2.71828)" 3, \N, \N, \N, \N, \N' | $CLICKHOUSE_CLIENT --input_format_null_as_default=1 --query="INSERT INTO null_as_default FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM null_as_default ORDER BY i"; $CLICKHOUSE_CLIENT --query="TRUNCATE TABLE null_as_default"; From 550a304cdccb83fb7ecf322b1d4b0b390fe777b4 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Tue, 19 Mar 2024 10:54:58 +0800 Subject: [PATCH 041/154] fix tests failure --- ...2246_tsv_csv_best_effort_schema_inference.reference | 4 ++-- .../0_stateless/02969_auto_format_detection.reference | 3 +-- .../02977_csv_format_support_tuple.reference | 10 +++++----- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference index 1c60e40942c..8ad0a566c62 100644 --- a/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference +++ b/tests/queries/0_stateless/02246_tsv_csv_best_effort_schema_inference.reference @@ -97,8 +97,8 @@ c1 Array(Nullable(Bool)) [] [NULL] [false] -c1 Nullable(String) -(1, 2, 3) +c1 Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64)) +(1,2,3) c1 Nullable(String) 123.123 c1 Array(Tuple(Nullable(Int64), Nullable(Int64), Nullable(Int64))) diff --git a/tests/queries/0_stateless/02969_auto_format_detection.reference b/tests/queries/0_stateless/02969_auto_format_detection.reference index 4b86be04996..865db11defc 100644 --- a/tests/queries/0_stateless/02969_auto_format_detection.reference +++ b/tests/queries/0_stateless/02969_auto_format_detection.reference @@ -82,8 +82,7 @@ CSV c1 Nullable(UInt64) c2 Nullable(String) c3 Array(Nullable(UInt64)) -c4 Nullable(UInt64) -c5 Nullable(String) +c4 Tuple(Nullable(UInt64), Nullable(String)) a Nullable(String) b Nullable(String) c Array(Nullable(String)) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference index 11902398999..d71200e6204 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference @@ -1,6 +1,6 @@ -Nullable(String) -Nullable(Int64) -Array(Nullable(String)) -Map(String, Nullable(Int64)) -Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) +c1 Nullable(String) +c2 Nullable(Int64) +c3 Array(Nullable(String)) +c4 Map(String, Nullable(Int64)) +c5 Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) 20240305 1 ['s','d'] {'a':2} ('222',33,{'abc':5}) From d716969284b9b97c10d58b46b2b9e25cb307ceff Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Tue, 19 Mar 2024 16:23:10 +0800 Subject: [PATCH 042/154] fix test missing tabs --- .../02977_csv_format_support_tuple.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference index d71200e6204..a1c6c3880ed 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference @@ -1,6 +1,6 @@ -c1 Nullable(String) -c2 Nullable(Int64) -c3 Array(Nullable(String)) -c4 Map(String, Nullable(Int64)) -c5 Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) +c1 Nullable(String) +c2 Nullable(Int64) +c3 Array(Nullable(String)) +c4 Map(String, Nullable(Int64)) +c5 Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) 20240305 1 ['s','d'] {'a':2} ('222',33,{'abc':5}) From dc52ac3252a311941fb6403c4c9dd9ae461acb1a Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Wed, 20 Mar 2024 09:46:15 +0800 Subject: [PATCH 043/154] fix test missing tabs --- .../02977_csv_format_support_tuple.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference index a1c6c3880ed..be68d164cff 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.reference +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.reference @@ -1,6 +1,6 @@ -c1 Nullable(String) -c2 Nullable(Int64) -c3 Array(Nullable(String)) -c4 Map(String, Nullable(Int64)) -c5 Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) +c1 Nullable(String) +c2 Nullable(Int64) +c3 Array(Nullable(String)) +c4 Map(String, Nullable(Int64)) +c5 Tuple(Nullable(String), Nullable(Int64), Map(String, Nullable(Int64))) 20240305 1 ['s','d'] {'a':2} ('222',33,{'abc':5}) From 4d99ffbe1c1d91d30fe390c0b7ea446b7a86676a Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Fri, 22 Mar 2024 16:10:42 +0800 Subject: [PATCH 044/154] fix test failure --- tests/queries/0_stateless/02977_csv_format_support_tuple.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql index b4726391925..40adf2cf0cb 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql @@ -1,4 +1,5 @@ --- Tags: no-fasttest +-- Tags: no-parallel + insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)); desc file('02977_1.csv'); select * from file('02977_1.csv') settings max_threads=1; From 6a00296c2fdd5178f257ae3ea9f0c75dd1ed2c3b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Mar 2024 13:28:35 +0100 Subject: [PATCH 045/154] Fix stateful test --- .../1_stateful/00172_early_constant_folding.reference | 9 +++++++++ .../queries/1_stateful/00172_early_constant_folding.sql | 3 ++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00172_early_constant_folding.reference b/tests/queries/1_stateful/00172_early_constant_folding.reference index 6b72183c066..50a175942ee 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.reference +++ b/tests/queries/1_stateful/00172_early_constant_folding.reference @@ -7,3 +7,12 @@ ExpressionTransform × 10 ExpressionTransform (ReadFromPreparedSource) SourceFromSingleChunk 0 → 1 +(Expression) +ExpressionTransform × 10 + (MergingAggregated) + Resize 1 → 10 + MergingAggregatedTransform + (Expression) + ExpressionTransform + (ReadFromPreparedSource) + SourceFromSingleChunk 0 → 1 diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index 19f99f107ac..b4b58ba3cb0 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -2,4 +2,5 @@ set max_threads=10; set optimize_use_implicit_projections=1; -EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1; +EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS allow_experimental_analyzer = 0; +EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1 SETTINGS allow_experimental_analyzer = 1; From 411c535ac9b1ca4c3a655156398356177012e4e9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 22 Mar 2024 13:59:44 +0100 Subject: [PATCH 046/154] Fix integration tests --- tests/integration/test_backward_compatibility/test.py | 1 - .../test_backward_compatibility/test_aggregate_fixed_key.py | 1 - .../test_aggregate_function_state.py | 2 -- .../test_backward_compatibility/test_convert_ordinary.py | 1 - .../test_backward_compatibility/test_cte_distributed.py | 1 - .../integration/test_backward_compatibility/test_functions.py | 1 - .../test_backward_compatibility/test_insert_profile_events.py | 1 - .../test_ip_types_binary_compatibility.py | 1 - .../test_memory_bound_aggregation.py | 2 -- .../test_normalized_count_comparison.py | 1 - .../test_select_aggregate_alias_column.py | 1 - .../test_short_strings_aggregation.py | 2 -- .../test_vertical_merges_from_compact_parts.py | 1 - tests/integration/test_disk_over_web_server/test.py | 1 - .../test_distributed_insert_backward_compatibility/test.py | 1 - tests/integration/test_old_versions/test.py | 1 - tests/integration/test_polymorphic_parts/test.py | 1 - .../test_replicated_merge_tree_compatibility/test.py | 2 -- tests/integration/test_ttl_replicated/test.py | 3 --- tests/integration/test_version_update/test.py | 1 - tests/integration/test_version_update_after_mutation/test.py | 3 --- 21 files changed, 29 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index 00973640998..7de5f51921b 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,7 +10,6 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 263f2d0af55..6b385bf8402 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -9,7 +9,6 @@ node1 = cluster.add_instance( image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance("node2", with_zookeeper=True, use_old_analyzer=True) node3 = cluster.add_instance("node3", with_zookeeper=True, use_old_analyzer=True) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py index ad2b20a07ad..9878c1ed70e 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py @@ -10,7 +10,6 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,7 +18,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) node4 = cluster.add_instance("node4", with_zookeeper=False, use_old_analyzer=True) diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index ff296d01c18..b8db4e005a4 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -9,7 +9,6 @@ node = cluster.add_instance( stay_alive=True, with_zookeeper=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index 592ec4ecbcc..e0be009e874 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -11,7 +11,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index d95c58085f3..1cf5c3deb81 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -15,7 +15,6 @@ backward = cluster.add_instance( image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 7caf15335d0..a90453d045b 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -13,7 +13,6 @@ old_node = cluster.add_instance( image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py index f91f04da823..4752a589a44 100644 --- a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py +++ b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py @@ -10,7 +10,6 @@ node = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index 2869b91f984..b13e6c975e8 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -10,7 +10,6 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,7 +18,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index 830dec8c517..83be0e4c5a3 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -11,7 +11,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index 8ddcbb71cab..cbe147dc07b 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -11,7 +11,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 715d295ac9e..60375196366 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -10,7 +10,6 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -19,7 +18,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node3 = cluster.add_instance("node3", with_zookeeper=False, use_old_analyzer=True) diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 9740a8828b2..e36c3310e4a 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -11,7 +11,6 @@ node_old = cluster.add_instance( stay_alive=True, with_installed_binary=True, with_zookeeper=True, - use_old_analyzer=True, ) node_new = cluster.add_instance( "node2", diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 115161dcaad..3d349129909 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -39,7 +39,6 @@ def cluster(): with_installed_binary=True, image="clickhouse/clickhouse-server", tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, - use_old_analyzer=True, ) cluster.start() diff --git a/tests/integration/test_distributed_insert_backward_compatibility/test.py b/tests/integration/test_distributed_insert_backward_compatibility/test.py index 72cc74e627a..9e794555d49 100644 --- a/tests/integration/test_distributed_insert_backward_compatibility/test.py +++ b/tests/integration/test_distributed_insert_backward_compatibility/test.py @@ -14,7 +14,6 @@ node_dist = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index e8b4999af46..a5e62a380bf 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -10,7 +10,6 @@ node_oldest = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], - use_old_analyzer=True, ) old_nodes = [node_oldest] new_node = cluster.add_instance("node_new") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 714a89352bf..2b30170b203 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -369,7 +369,6 @@ node7 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node8 = cluster.add_instance( "node8", diff --git a/tests/integration/test_replicated_merge_tree_compatibility/test.py b/tests/integration/test_replicated_merge_tree_compatibility/test.py index c7e9933e1de..a70f3234c1e 100644 --- a/tests/integration/test_replicated_merge_tree_compatibility/test.py +++ b/tests/integration/test_replicated_merge_tree_compatibility/test.py @@ -9,7 +9,6 @@ node1 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -18,7 +17,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, stay_alive=True, with_installed_binary=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index c62e5c23438..f944adbea41 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -23,7 +23,6 @@ node4 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) node5 = cluster.add_instance( @@ -36,7 +35,6 @@ node5 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) node6 = cluster.add_instance( "node6", @@ -48,7 +46,6 @@ node6 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index 704b1bd2b21..b386a79c932 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -14,7 +14,6 @@ node2 = cluster.add_instance( tag=CLICKHOUSE_CI_MIN_TESTED_VERSION, with_installed_binary=True, stay_alive=True, - use_old_analyzer=True, ) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index e9cc34a8f27..9365498f89d 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -16,7 +16,6 @@ node1 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) node2 = cluster.add_instance( "node2", @@ -28,7 +27,6 @@ node2 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) node3 = cluster.add_instance( "node3", @@ -40,7 +38,6 @@ node3 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], - use_old_analyzer=True, ) From abbbd21b7eb66a4fa6d41e7038cdb1dc7717a640 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Mar 2024 14:28:49 +0000 Subject: [PATCH 047/154] CI: disable grpc tests on ARM --- tests/integration/test_grpc_protocol/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 0332e5bb90f..145cf03c58e 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -5,7 +5,7 @@ import time import pytz import uuid import grpc -from helpers.cluster import ClickHouseCluster, run_and_check +from helpers.cluster import ClickHouseCluster, is_arm, run_and_check from threading import Thread import gzip import lz4.frame @@ -20,6 +20,10 @@ import clickhouse_grpc_pb2, clickhouse_grpc_pb2_grpc # Execute pb2/generate.py GRPC_PORT = 9100 DEFAULT_ENCODING = "utf-8" +# GRPC is disabled on ARM build - skip tests +if is_arm(): + pytestmark = pytest.mark.skip + # Utilities From 6028d007c3fe784bb1ea332e69a0351cc7f7d32d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 22 Mar 2024 15:52:54 +0100 Subject: [PATCH 048/154] =?UTF-8?q?Revert=20"Revert=20"Fix=20bug=20when=20?= =?UTF-8?q?reading=20system.parts=20using=20UUID=20(issue=2061220)."=20?= =?UTF-8?q?=E2=80=A6"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 8c49c9bf9e119ecbfdf3ec9147190a226424a9d2. --- src/Storages/System/StorageSystemPartsBase.cpp | 4 ++-- .../03010_read_system_parts_table_test.reference | 10 ++++++++++ .../03010_read_system_parts_table_test.sql | 11 +++++++++++ 3 files changed, 23 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03010_read_system_parts_table_test.reference create mode 100644 tests/queries/0_stateless/03010_read_system_parts_table_test.sql diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 2acb5ebe221..4855dcfd6ed 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -190,7 +190,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared(), "table")); block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared(), "engine")); block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared(), "active")); - block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), "uuid")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), "storage_uuid")); if (rows) { @@ -203,7 +203,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, database_column = block_to_filter.getByName("database").column; table_column = block_to_filter.getByName("table").column; active_column = block_to_filter.getByName("active").column; - storage_uuid_column = block_to_filter.getByName("uuid").column; + storage_uuid_column = block_to_filter.getByName("storage_uuid").column; } class ReadFromSystemPartsBase : public SourceStepWithFilter diff --git a/tests/queries/0_stateless/03010_read_system_parts_table_test.reference b/tests/queries/0_stateless/03010_read_system_parts_table_test.reference new file mode 100644 index 00000000000..c2dd177f245 --- /dev/null +++ b/tests/queries/0_stateless/03010_read_system_parts_table_test.reference @@ -0,0 +1,10 @@ +00000000-0000-0000-0000-000000000000 1231_1_1_0 +00000000-0000-0000-0000-000000000000 6666_2_2_0 +00000000-0000-0000-0000-000000000000 1231_1_1_0 users +00000000-0000-0000-0000-000000000000 6666_2_2_0 users +00000000-0000-0000-0000-000000000000 1231_1_1_0 users uid +00000000-0000-0000-0000-000000000000 1231_1_1_0 users name +00000000-0000-0000-0000-000000000000 1231_1_1_0 users age +00000000-0000-0000-0000-000000000000 6666_2_2_0 users uid +00000000-0000-0000-0000-000000000000 6666_2_2_0 users name +00000000-0000-0000-0000-000000000000 6666_2_2_0 users age diff --git a/tests/queries/0_stateless/03010_read_system_parts_table_test.sql b/tests/queries/0_stateless/03010_read_system_parts_table_test.sql new file mode 100644 index 00000000000..a50005b2312 --- /dev/null +++ b/tests/queries/0_stateless/03010_read_system_parts_table_test.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS users; +CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree ORDER BY uid PARTITION BY uid; + +INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users VALUES (6666, 'Ksenia', 48); + +SELECT uuid, name from system.parts WHERE database = currentDatabase() AND table = 'users'; + +SELECT uuid, name, table from system.parts WHERE database = currentDatabase() AND table = 'users' AND uuid = '00000000-0000-0000-0000-000000000000'; +SELECT uuid, name, table, column from system.parts_columns WHERE database = currentDatabase() AND table = 'users' AND uuid = '00000000-0000-0000-0000-000000000000'; +DROP TABLE IF EXISTS users; From cff56a5c73fc987108c6bbb6e83fe86d34fba2fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 22 Mar 2024 15:05:10 +0000 Subject: [PATCH 049/154] Unify column names --- .../System/StorageSystemPartsBase.cpp | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 4855dcfd6ed..f2b1e1222c3 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -24,6 +24,13 @@ #include #include +namespace { +constexpr auto * database_column_name = "database"; +constexpr auto * table_column_name = "table"; +constexpr auto * engine_column_name = "engine"; +constexpr auto * active_column_name = "active"; +constexpr auto * storage_uuid_column_name = "storage_uuid"; +} namespace DB { @@ -112,7 +119,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, database_column_mut->insert(database.first); } block_to_filter.insert(ColumnWithTypeAndName( - std::move(database_column_mut), std::make_shared(), "database")); + std::move(database_column_mut), std::make_shared(), database_column_name)); /// Filter block_to_filter with column 'database'. if (filter_by_database) @@ -120,7 +127,7 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, rows = block_to_filter.rows(); /// Block contains new columns, update database_column. - ColumnPtr database_column_for_filter = block_to_filter.getByName("database").column; + ColumnPtr database_column_for_filter = block_to_filter.getByName(database_column_name).column; if (rows) { @@ -187,10 +194,10 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, } } - block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared(), "table")); - block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared(), "engine")); - block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared(), "active")); - block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), "storage_uuid")); + block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared(), table_column_name)); + block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared(), engine_column_name)); + block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared(), active_column_name)); + block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared(), storage_uuid_column_name)); if (rows) { @@ -200,10 +207,10 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, rows = block_to_filter.rows(); } - database_column = block_to_filter.getByName("database").column; - table_column = block_to_filter.getByName("table").column; - active_column = block_to_filter.getByName("active").column; - storage_uuid_column = block_to_filter.getByName("storage_uuid").column; + database_column = block_to_filter.getByName(database_column_name).column; + table_column = block_to_filter.getByName(table_column_name).column; + active_column = block_to_filter.getByName(active_column_name).column; + storage_uuid_column = block_to_filter.getByName(storage_uuid_column_name).column; } class ReadFromSystemPartsBase : public SourceStepWithFilter @@ -261,16 +268,16 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes) const auto * predicate = filter_actions_dag->getOutputs().at(0); Block block; - block.insert(ColumnWithTypeAndName({}, std::make_shared(), "database")); + block.insert(ColumnWithTypeAndName({}, std::make_shared(), database_column_name)); filter_by_database = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block); if (filter_by_database) VirtualColumnUtils::buildSetsForDAG(filter_by_database, context); - block.insert(ColumnWithTypeAndName({}, std::make_shared(), "table")); - block.insert(ColumnWithTypeAndName({}, std::make_shared(), "engine")); - block.insert(ColumnWithTypeAndName({}, std::make_shared(), "active")); - block.insert(ColumnWithTypeAndName({}, std::make_shared(), "uuid")); + block.insert(ColumnWithTypeAndName({}, std::make_shared(), table_column_name)); + block.insert(ColumnWithTypeAndName({}, std::make_shared(), engine_column_name)); + block.insert(ColumnWithTypeAndName({}, std::make_shared(), active_column_name)); + block.insert(ColumnWithTypeAndName({}, std::make_shared(), storage_uuid_column_name)); filter_by_other_columns = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block); if (filter_by_other_columns) From e1f9e496408d57acc5dcb2808f77bf38f1e8a524 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 22 Mar 2024 17:21:48 +0100 Subject: [PATCH 050/154] fix test for smt --- .../0_stateless/02916_move_partition_inactive_replica.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql index 25dd0365d97..ca153eea221 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -12,6 +12,7 @@ create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; system stop merges shard_0.from_1; +system stop merges shard_1.from_1; insert into shard_0.from_1 select number + 20 from numbers(10); insert into shard_0.from_1 select number + 30 from numbers(10); @@ -20,7 +21,7 @@ insert into shard_0.from_1 select number + 50 from numbers(10); system sync replica shard_1.from_1; -create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; +create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2, shared_merge_tree_disable_merges_and_mutations_assignment=1; create table shard_1.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; From 91e52ebee593c42a6d27055a70809e79e3128c0a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Mar 2024 16:40:17 +0000 Subject: [PATCH 051/154] disable mysql57 tests --- tests/integration/test_mysql57_database_engine/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_mysql57_database_engine/test.py b/tests/integration/test_mysql57_database_engine/test.py index 26db6637bc6..d10dd50f0a5 100644 --- a/tests/integration/test_mysql57_database_engine/test.py +++ b/tests/integration/test_mysql57_database_engine/test.py @@ -5,9 +5,13 @@ from string import Template import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, is_arm from helpers.network import PartitionManager + +if is_arm(): + pytestmark = pytest.mark.skip + cluster = ClickHouseCluster(__file__) clickhouse_node = cluster.add_instance( "node1", From 5207526910f0abb9dce985d23b6f6e0d0cb2fc1b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Mar 2024 16:55:01 +0000 Subject: [PATCH 052/154] Fix more tests with wirtual columns in StorageMerge. --- src/Functions/identity.cpp | 5 ++++ src/Functions/identity.h | 24 +++++++++++++++++ src/Planner/PlannerActionsVisitor.cpp | 6 +++++ src/Storages/StorageMerge.cpp | 39 ++++++++++++++++++++++----- tests/analyzer_tech_debt.txt | 3 --- 5 files changed, 67 insertions(+), 10 deletions(-) diff --git a/src/Functions/identity.cpp b/src/Functions/identity.cpp index 43cca76c801..2541e715cb1 100644 --- a/src/Functions/identity.cpp +++ b/src/Functions/identity.cpp @@ -14,4 +14,9 @@ REGISTER_FUNCTION(ScalarSubqueryResult) factory.registerFunction(); } +REGISTER_FUNCTION(ActionName) +{ + factory.registerFunction(); +} + } diff --git a/src/Functions/identity.h b/src/Functions/identity.h index c753625caa7..22dcd72d6e1 100644 --- a/src/Functions/identity.h +++ b/src/Functions/identity.h @@ -42,4 +42,28 @@ struct ScalarSubqueryResultName using FunctionIdentity = FunctionIdentityBase; using FunctionScalarSubqueryResult = FunctionIdentityBase; +class FunctionActionName : public IFunction +{ +public: + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + static constexpr auto name = "__actionName"; + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool isSuitableForConstantFolding() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + return arguments.front(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + return arguments.front().column; + } +}; + } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index af23e684f23..326dd683343 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -157,6 +157,12 @@ public: case QueryTreeNodeType::FUNCTION: { const auto & function_node = node->as(); + if (function_node.getFunctionName() == "__actionName") + { + result = toString(function_node.getArguments().getNodes().at(1)->as()->getValue()); + break; + } + String in_function_second_argument_node_name; if (isNameOfInFunction(function_node.getFunctionName())) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 3d1ca0c76b6..0b189f84ffa 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -901,15 +901,27 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!storage_snapshot_->tryGetColumn(get_column_options, "_table")) { - auto table_name_node = std::make_shared(current_storage_id.table_name); - table_name_node->setAlias("_table"); - column_name_to_node.emplace("_table", table_name_node); + auto table_name_node = std::make_shared(current_storage_id.table_name); //, std::make_shared(std::make_shared())); + auto table_name_alias = std::make_shared("__table1._table"); + + auto function_node = std::make_shared("__actionName"); + function_node->getArguments().getNodes().push_back(std::move(table_name_node)); + function_node->getArguments().getNodes().push_back(std::move(table_name_alias)); + function_node->resolveAsFunction(FunctionFactory::instance().get("__actionName", context)); + + column_name_to_node.emplace("_table", function_node); } if (!storage_snapshot_->tryGetColumn(get_column_options, "_database")) { - auto database_name_node = std::make_shared(current_storage_id.database_name); - database_name_node->setAlias("_database"); + auto database_name_node = std::make_shared(current_storage_id.database_name); //, std::make_shared(std::make_shared())); + auto database_name_alias = std::make_shared("__table1._table"); + + auto function_node = std::make_shared("__actionName"); + function_node->getArguments().getNodes().push_back(std::move(database_name_node)); + function_node->getArguments().getNodes().push_back(std::move(database_name_alias)); + function_node->resolveAsFunction(FunctionFactory::instance().get("__actionName", context)); + column_name_to_node.emplace("_database", database_name_node); } @@ -961,9 +973,15 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!column_name_to_node.empty()) { + // std::cerr << ">>>>>>>>>>>>>>>>\n"; + // std::cerr << modified_query_info.query_tree->dumpTree() << std::endl; + replaceColumns(modified_query_info.query_tree, replacement_table_expression, column_name_to_node); + + // std::cerr << "<<<<<<<<<<\n"; + // std::cerr << modified_query_info.query_tree->dumpTree() << std::endl; } modified_query_info.query = queryNodeToSelectQuery(modified_query_info.query_tree); @@ -1061,7 +1079,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; if (has_database_virtual_column && common_header.has(database_column) - && (storage_stage == QueryProcessingStage::FetchColumns || !pipe_header.has("'" + database_name + "'_String"))) + && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(database_column)) // || !pipe_header.has("'" + database_name + "'_String"))) { ColumnWithTypeAndName column; column.name = database_column; @@ -1077,7 +1095,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( } if (has_table_virtual_column && common_header.has(table_column) - && (storage_stage == QueryProcessingStage::FetchColumns || !pipe_header.has("'" + table_name + "'_String"))) + && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(table_column)) // || !pipe_header.has("'" + table_name + "'_String"))) { ColumnWithTypeAndName column; column.name = table_column; @@ -1236,6 +1254,10 @@ QueryPlan ReadFromMerge::createPlanForTable( } } + // WriteBufferFromOwnString buf; + // plan.explainPlan(buf, {.header=true, .actions=true}); + // std::cerr << buf.str() << std::endl; + return plan; } @@ -1547,6 +1569,9 @@ void ReadFromMerge::convertAndFilterSourceStream( row_policy_data_opt->addFilterTransform(builder); } + std::cerr << "============" << builder.getHeader().dumpStructure() << std::endl; + std::cerr << "============" << header.dumpStructure() << std::endl; + auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e2d8084df5e..0df31b5e78f 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,10 +1,7 @@ 00223_shard_distributed_aggregation_memory_efficient 00725_memory_tracking -01155_rename_move_materialized_view 01624_soft_constraints 02354_vector_search_queries -# Check after constants refactoring -02901_parallel_replicas_rollup # Flaky. Please don't delete them without fixing them: 01287_max_execution_speed 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET From cf91d5dcbca0bd06e30e505bc05137f8d23a8048 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Mar 2024 16:57:42 +0000 Subject: [PATCH 053/154] FIx --- src/Storages/StorageMerge.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0b189f84ffa..4a04ffe7ca7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1569,8 +1569,8 @@ void ReadFromMerge::convertAndFilterSourceStream( row_policy_data_opt->addFilterTransform(builder); } - std::cerr << "============" << builder.getHeader().dumpStructure() << std::endl; - std::cerr << "============" << header.dumpStructure() << std::endl; + // std::cerr << "============" << builder.getHeader().dumpStructure() << std::endl; + // std::cerr << "============" << header.dumpStructure() << std::endl; auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), From b2ae11d6b62e6830c20dc0c3dc3984e205c1e851 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Mar 2024 17:49:01 +0000 Subject: [PATCH 054/154] Fixing tests. --- src/Storages/StorageMerge.cpp | 4 ++-- .../02415_all_new_functions_must_be_documented.reference | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4a04ffe7ca7..ec27d324ade 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -915,14 +915,14 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!storage_snapshot_->tryGetColumn(get_column_options, "_database")) { auto database_name_node = std::make_shared(current_storage_id.database_name); //, std::make_shared(std::make_shared())); - auto database_name_alias = std::make_shared("__table1._table"); + auto database_name_alias = std::make_shared("__table1._database"); auto function_node = std::make_shared("__actionName"); function_node->getArguments().getNodes().push_back(std::move(database_name_node)); function_node->getArguments().getNodes().push_back(std::move(database_name_alias)); function_node->resolveAsFunction(FunctionFactory::instance().get("__actionName", context)); - column_name_to_node.emplace("_database", database_name_node); + column_name_to_node.emplace("_database", function_node); } auto storage_columns = storage_snapshot_->metadata->getColumns(); diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 8b85ac48c16..3ddf165dec0 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -57,6 +57,7 @@ URLPathHierarchy UUIDNumToString UUIDStringToNum _CAST +__actionName __bitBoolMaskAnd __bitBoolMaskOr __bitSwapLastTwo From 5b7a3c276b3ce134d487956fdb7cc79057cabe9f Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 22 Mar 2024 20:15:08 +0000 Subject: [PATCH 055/154] fix prewhere on join using column that changed type to supertype --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 51 +++++++++++++---------- 1 file changed, 29 insertions(+), 22 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 12adb762a9a..981b162d17a 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7696,27 +7696,36 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } -class ReplaceColumnsVisitor : public InDepthQueryTreeVisitor +/** Replace all columns with their versions from the source table. + * It's possible that after a JOIN, a column in the projection has a type different from the column in the source table. + * However, the column in the projection still refers to the table as its source. + * This visitor restores column nodes to their source versions. + */ +class ReplaceColumnsToSourceVisitor : public InDepthQueryTreeVisitor { public: - explicit ReplaceColumnsVisitor(const QueryTreeNodePtrWithHashMap & replacement_map_, const ContextPtr & context_) - : replacement_map(replacement_map_) - , context(context_) + explicit ReplaceColumnsToSourceVisitor(const IdentifierResolveScope & scope_) + : scope(scope_) {} void visitImpl(QueryTreeNodePtr & node) { - if (auto it = replacement_map.find(node); it != replacement_map.end()) - node = it->second; + if (auto * column_node = node->as(); column_node && column_node->getColumnSource()) + { + const auto & table_expession_data = scope.getTableExpressionDataOrThrow(column_node->getColumnSource()); + const auto & source_column_it = table_expession_data.column_name_to_column_node.find(column_node->getColumnName()); + if (source_column_it != table_expession_data.column_name_to_column_node.end()) + node = source_column_it->second; + } + if (auto * function_node = node->as()) - rerunFunctionResolve(function_node, context); + rerunFunctionResolve(function_node, scope.context); } bool shouldTraverseTopToBottom() const { return false; } private: - const QueryTreeNodePtrWithHashMap & replacement_map; - const ContextPtr & context; + const IdentifierResolveScope & scope; }; /** Resolve query. @@ -7910,19 +7919,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { resolveExpressionNode(prewhere_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (scope.join_use_nulls) - { - /** Expression in PREWHERE with JOIN should not be modified by join_use_nulls. - * Example: SELECT * FROM t1 JOIN t2 USING (id) PREWHERE b = 1 - * Column `a` should be resolved from table and should not change its type to Nullable. - * More complicated example when column is somewhere inside an expression: - * SELECT a + 1 as b FROM t1 JOIN t2 USING (id) PREWHERE b = 1 - * expression `a + 1 as b` in projection and in PREWHERE should have different `a`. - */ - prewhere_node = prewhere_node->clone(); - ReplaceColumnsVisitor replace_visitor(scope.nullable_join_columns, scope.context); - replace_visitor.visit(prewhere_node); - } + /** Expressions in PREWHERE with JOIN should not change their type. + * Example: SELECT * FROM t1 JOIN t2 USING (a) PREWHERE a = 1 + * Column `a` in PREWHERE should be resolved from the left table + * and should not change its type to Nullable or to the supertype of `a` from t1 and t2. + * Here's a more complicated example where the column is somewhere inside an expression: + * SELECT a + 1 as b FROM t1 JOIN t2 USING (id) PREWHERE b = 1 + * The expression `a + 1 as b` in the projection and in PREWHERE should have different `a`. + */ + prewhere_node = prewhere_node->clone(); + ReplaceColumnsToSourceVisitor replace_visitor(scope); + replace_visitor.visit(prewhere_node); } if (query_node_typed.getWhere()) From ee7d1cf7d975ee72cb1bace596d0b981b2cb8a19 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 23 Mar 2024 10:03:31 +0000 Subject: [PATCH 056/154] another fix prewhere on join using column that changed type to supertype tag: #ci_set_analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 54 +++++++++++-------- ...nalyzer_using_functional_args.reference.j2 | 8 +++ ...2955_analyzer_using_functional_args.sql.j2 | 3 ++ 3 files changed, 43 insertions(+), 22 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 981b162d17a..7f890e45b94 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -777,7 +777,13 @@ struct IdentifierResolveScope std::unordered_map table_expression_node_to_data; QueryTreeNodePtrWithHashSet nullable_group_by_keys; - QueryTreeNodePtrWithHashMap nullable_join_columns; + + /** It's possible that after a JOIN, a column in the projection has a type different from the column in the source table. + * (For example, after join_use_nulls or USING column casted to supertype) + * However, the column in the projection still refers to the table as its source. + * This map is used to revert these columns back to their original columns in the source table. + */ + QueryTreeNodePtrWithHashMap join_columns_with_cahnged_types; /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; @@ -1300,7 +1306,7 @@ private: if (!resolved_expression->getResultType()->equals(*new_result_type)) resolved_expression = buildCastFunction(resolved_expression, new_result_type, scope.context, true); } - scope.nullable_join_columns[nullable_resolved_identifier] = resolved_identifier; + scope.join_columns_with_cahnged_types[nullable_resolved_identifier] = resolved_identifier; return nullable_resolved_identifier; } return nullptr; @@ -3309,11 +3315,11 @@ static bool resolvedIdenfiersFromJoinAreEquals( const QueryTreeNodePtr & right_resolved_identifier, const IdentifierResolveScope & scope) { - auto lit = scope.nullable_join_columns.find(left_resolved_identifier); - const auto & left_resolved_to_compare = lit != scope.nullable_join_columns.end() ? lit->second : left_resolved_identifier; + auto lit = scope.join_columns_with_cahnged_types.find(left_resolved_identifier); + const auto & left_resolved_to_compare = lit != scope.join_columns_with_cahnged_types.end() ? lit->second : left_resolved_identifier; - auto rit = scope.nullable_join_columns.find(right_resolved_identifier); - const auto & right_resolved_to_compare = rit != scope.nullable_join_columns.end() ? rit->second : right_resolved_identifier; + auto rit = scope.join_columns_with_cahnged_types.find(right_resolved_identifier); + const auto & right_resolved_to_compare = rit != scope.join_columns_with_cahnged_types.end() ? rit->second : right_resolved_identifier; return left_resolved_to_compare->isEqual(*right_resolved_to_compare, IQueryTreeNode::CompareOptions{.compare_aliases = false}); } @@ -3451,6 +3457,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto result_column_node = inner_column_node->clone(); auto & result_column = result_column_node->as(); result_column.setColumnType(using_column_node.getColumnType()); + scope.join_columns_with_cahnged_types[result_column_node] = using_expression_list.getNodes().at(0); resolved_identifier = std::move(result_column_node); } @@ -3530,6 +3537,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto left_resolved_column_clone = std::static_pointer_cast(left_resolved_column.clone()); left_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); resolved_identifier = std::move(left_resolved_column_clone); + scope.join_columns_with_cahnged_types[resolved_identifier] = using_column_node_it->second; } } } @@ -3552,6 +3560,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto right_resolved_column_clone = std::static_pointer_cast(right_resolved_column.clone()); right_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); resolved_identifier = std::move(right_resolved_column_clone); + scope.join_columns_with_cahnged_types[resolved_identifier] = using_column_node_it->second; } } } @@ -4312,6 +4321,7 @@ void QueryAnalyzer::updateMatchedColumnsFromJoinUsing( node_to_projection_name.emplace(matched_column_node, it->second); matched_column_node->as().setColumnType(join_using_column_node.getResultType()); + scope.join_columns_with_cahnged_types[matched_column_node] = join_using_column_nodes.at(0); } } } @@ -4571,6 +4581,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( matched_column_node = matched_column_node->clone(); matched_column_node->as().setColumnType(join_using_column_node.getResultType()); + scope.join_columns_with_cahnged_types[matched_column_node] = join_using_column_nodes.at(0); table_expression_column_names_to_skip.insert(join_using_column_name); matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), join_using_column_name); @@ -7696,36 +7707,35 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } -/** Replace all columns with their versions from the source table. - * It's possible that after a JOIN, a column in the projection has a type different from the column in the source table. - * However, the column in the projection still refers to the table as its source. - * This visitor restores column nodes to their source versions. - */ -class ReplaceColumnsToSourceVisitor : public InDepthQueryTreeVisitor +class ReplaceColumnsVisitor : public InDepthQueryTreeVisitor { public: - explicit ReplaceColumnsToSourceVisitor(const IdentifierResolveScope & scope_) - : scope(scope_) + explicit ReplaceColumnsVisitor(const QueryTreeNodePtrWithHashMap & replacement_map_, const ContextPtr & context_) + : replacement_map(replacement_map_) + , context(context_) {} void visitImpl(QueryTreeNodePtr & node) { - if (auto * column_node = node->as(); column_node && column_node->getColumnSource()) + while (true) { - const auto & table_expession_data = scope.getTableExpressionDataOrThrow(column_node->getColumnSource()); - const auto & source_column_it = table_expession_data.column_name_to_column_node.find(column_node->getColumnName()); - if (source_column_it != table_expession_data.column_name_to_column_node.end()) - node = source_column_it->second; + /// Apply replacement transitively + auto it = replacement_map.find(node); + if (it != replacement_map.end()) + node = it->second; + else + break; } if (auto * function_node = node->as()) - rerunFunctionResolve(function_node, scope.context); + rerunFunctionResolve(function_node, context); } bool shouldTraverseTopToBottom() const { return false; } private: - const IdentifierResolveScope & scope; + const QueryTreeNodePtrWithHashMap & replacement_map; + const ContextPtr & context; }; /** Resolve query. @@ -7928,7 +7938,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * The expression `a + 1 as b` in the projection and in PREWHERE should have different `a`. */ prewhere_node = prewhere_node->clone(); - ReplaceColumnsToSourceVisitor replace_visitor(scope); + ReplaceColumnsVisitor replace_visitor(scope.join_columns_with_cahnged_types, scope.context); replace_visitor.visit(prewhere_node); } diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 index 7d8fcf5bce0..e0c6a439112 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.reference.j2 @@ -78,6 +78,14 @@ SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) a b +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 0; +369 +3693 +6666 +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 1; +369 +3693 +6666 DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 index 6fb88a02e5e..f5b81231afe 100644 --- a/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 +++ b/tests/queries/0_stateless/02955_analyzer_using_functional_args.sql.j2 @@ -57,6 +57,9 @@ SELECT y FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) SELECT s FROM t1 FULL JOIN t2 USING (y) ORDER BY ALL; SELECT s FROM (SELECT s, y FROM t1) t1 FULL JOIN (SELECT y FROM t2) t2 USING (y) ORDER BY ALL; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 0; +SELECT y FROM t1 FULL JOIN t2 USING (y) PREWHERE y * 2 > 2 ORDER BY ALL SETTINGS allow_experimental_analyzer = 1, join_use_nulls = 1; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t3; From 44f1f62904a214f299162875f80ced0a137396d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 23 Mar 2024 14:29:08 +0000 Subject: [PATCH 057/154] fix ReplaceColumnsVisitor #ci_set_analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7f890e45b94..aa59d6f3ce9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -7727,12 +7727,22 @@ public: break; } - if (auto * function_node = node->as()) + if (auto * function_node = node->as(); function_node && function_node->isResolved()) rerunFunctionResolve(function_node, context); } + /// We want to re-run resolve for function _after_ its arguments are replaced bool shouldTraverseTopToBottom() const { return false; } + bool needChildVisit(QueryTreeNodePtr & /* parent */, QueryTreeNodePtr & child) + { + /// Visit only expressions, but not subqueries + return child->getNodeType() == QueryTreeNodeType::IDENTIFIER + || child->getNodeType() == QueryTreeNodeType::LIST + || child->getNodeType() == QueryTreeNodeType::FUNCTION + || child->getNodeType() == QueryTreeNodeType::COLUMN; + } + private: const QueryTreeNodePtrWithHashMap & replacement_map; const ContextPtr & context; From adbc9c66e900bd9df763641541e08e1ba428a768 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 23 Mar 2024 18:33:06 +0000 Subject: [PATCH 058/154] fix infinite loop in ReplaceColumnsVisitor #ci_set_analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 131 +++++++++++++--------- 1 file changed, 78 insertions(+), 53 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index aa59d6f3ce9..b1b70c8ef51 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -783,7 +783,7 @@ struct IdentifierResolveScope * However, the column in the projection still refers to the table as its source. * This map is used to revert these columns back to their original columns in the source table. */ - QueryTreeNodePtrWithHashMap join_columns_with_cahnged_types; + QueryTreeNodePtrWithHashMap join_columns_with_changed_types; /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; @@ -1306,7 +1306,8 @@ private: if (!resolved_expression->getResultType()->equals(*new_result_type)) resolved_expression = buildCastFunction(resolved_expression, new_result_type, scope.context, true); } - scope.join_columns_with_cahnged_types[nullable_resolved_identifier] = resolved_identifier; + if (!nullable_resolved_identifier->isEqual(*resolved_identifier)) + scope.join_columns_with_changed_types[nullable_resolved_identifier] = resolved_identifier; return nullable_resolved_identifier; } return nullptr; @@ -3309,17 +3310,74 @@ QueryTreeNodePtr checkIsMissedObjectJSONSubcolumn(const QueryTreeNodePtr & left_ return {}; } +/// Used to replace columns that changed type because of JOIN to their original type +class ReplaceColumnsVisitor : public InDepthQueryTreeVisitor +{ +public: + explicit ReplaceColumnsVisitor(const QueryTreeNodePtrWithHashMap & replacement_map_, const ContextPtr & context_) + : replacement_map(replacement_map_) + , context(context_) + {} + + /// Apply replacement transitively, because column may change it's type twice, one to have a supertype and then because of `joun_use_nulls` + static QueryTreeNodePtr findTransitiveReplacement(QueryTreeNodePtr node, const QueryTreeNodePtrWithHashMap & replacement_map_) + { + auto it = replacement_map_.find(node); + QueryTreeNodePtr result_node = nullptr; + for (; it != replacement_map_.end(); it = replacement_map_.find(result_node)) + { + if (result_node && result_node->isEqual(*it->second)) + { + Strings map_dump; + for (const auto & [k, v]: replacement_map_) + map_dump.push_back(fmt::format("{} -> {} (is_equals: {}, is_same: {})", + k.node->dumpTree(), v->dumpTree(), k.node->isEqual(*v), k.node == v)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Infinite loop in query tree replacement map: {}", fmt::join(map_dump, "; ")); + } + chassert(it->second); + + result_node = it->second; + } + return result_node; + } + + void visitImpl(QueryTreeNodePtr & node) + { + if (auto replacement_node = findTransitiveReplacement(node, replacement_map)) + node = replacement_node; + + if (auto * function_node = node->as(); function_node && function_node->isResolved()) + rerunFunctionResolve(function_node, context); + } + + /// We want to re-run resolve for function _after_ its arguments are replaced + bool shouldTraverseTopToBottom() const { return false; } + + bool needChildVisit(QueryTreeNodePtr & /* parent */, QueryTreeNodePtr & child) + { + /// Visit only expressions, but not subqueries + return child->getNodeType() == QueryTreeNodeType::IDENTIFIER + || child->getNodeType() == QueryTreeNodeType::LIST + || child->getNodeType() == QueryTreeNodeType::FUNCTION + || child->getNodeType() == QueryTreeNodeType::COLUMN; + } + +private: + const QueryTreeNodePtrWithHashMap & replacement_map; + const ContextPtr & context; +}; + /// Compare resolved identifiers considering columns that become nullable after JOIN static bool resolvedIdenfiersFromJoinAreEquals( const QueryTreeNodePtr & left_resolved_identifier, const QueryTreeNodePtr & right_resolved_identifier, const IdentifierResolveScope & scope) { - auto lit = scope.join_columns_with_cahnged_types.find(left_resolved_identifier); - const auto & left_resolved_to_compare = lit != scope.join_columns_with_cahnged_types.end() ? lit->second : left_resolved_identifier; + auto left_original_node = ReplaceColumnsVisitor::findTransitiveReplacement(left_resolved_identifier, scope.join_columns_with_changed_types); + const auto & left_resolved_to_compare = left_original_node ? left_original_node : left_resolved_identifier; - auto rit = scope.join_columns_with_cahnged_types.find(right_resolved_identifier); - const auto & right_resolved_to_compare = rit != scope.join_columns_with_cahnged_types.end() ? rit->second : right_resolved_identifier; + auto right_original_node = ReplaceColumnsVisitor::findTransitiveReplacement(right_resolved_identifier, scope.join_columns_with_changed_types); + const auto & right_resolved_to_compare = right_original_node ? right_original_node : right_resolved_identifier; return left_resolved_to_compare->isEqual(*right_resolved_to_compare, IQueryTreeNode::CompareOptions{.compare_aliases = false}); } @@ -3457,7 +3515,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto result_column_node = inner_column_node->clone(); auto & result_column = result_column_node->as(); result_column.setColumnType(using_column_node.getColumnType()); - scope.join_columns_with_cahnged_types[result_column_node] = using_expression_list.getNodes().at(0); + + const auto & join_using_left_column = using_expression_list.getNodes().at(0); + if (!result_column_node->isEqual(*join_using_left_column)) + scope.join_columns_with_changed_types[result_column_node] = join_using_left_column; resolved_identifier = std::move(result_column_node); } @@ -3537,7 +3598,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto left_resolved_column_clone = std::static_pointer_cast(left_resolved_column.clone()); left_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); resolved_identifier = std::move(left_resolved_column_clone); - scope.join_columns_with_cahnged_types[resolved_identifier] = using_column_node_it->second; + + if (!resolved_identifier->isEqual(*using_column_node_it->second)) + scope.join_columns_with_changed_types[resolved_identifier] = using_column_node_it->second; } } } @@ -3560,7 +3623,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo auto right_resolved_column_clone = std::static_pointer_cast(right_resolved_column.clone()); right_resolved_column_clone->setColumnType(using_column_node_it->second->getColumnType()); resolved_identifier = std::move(right_resolved_column_clone); - scope.join_columns_with_cahnged_types[resolved_identifier] = using_column_node_it->second; + if (!resolved_identifier->isEqual(*using_column_node_it->second)) + scope.join_columns_with_changed_types[resolved_identifier] = using_column_node_it->second; } } } @@ -4321,7 +4385,8 @@ void QueryAnalyzer::updateMatchedColumnsFromJoinUsing( node_to_projection_name.emplace(matched_column_node, it->second); matched_column_node->as().setColumnType(join_using_column_node.getResultType()); - scope.join_columns_with_cahnged_types[matched_column_node] = join_using_column_nodes.at(0); + if (!matched_column_node->isEqual(*join_using_column_nodes.at(0))) + scope.join_columns_with_changed_types[matched_column_node] = join_using_column_nodes.at(0); } } } @@ -4581,7 +4646,8 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( matched_column_node = matched_column_node->clone(); matched_column_node->as().setColumnType(join_using_column_node.getResultType()); - scope.join_columns_with_cahnged_types[matched_column_node] = join_using_column_nodes.at(0); + if (!matched_column_node->isEqual(*join_using_column_nodes.at(0))) + scope.join_columns_with_changed_types[matched_column_node] = join_using_column_nodes.at(0); table_expression_column_names_to_skip.insert(join_using_column_name); matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), join_using_column_name); @@ -7707,47 +7773,6 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } -class ReplaceColumnsVisitor : public InDepthQueryTreeVisitor -{ -public: - explicit ReplaceColumnsVisitor(const QueryTreeNodePtrWithHashMap & replacement_map_, const ContextPtr & context_) - : replacement_map(replacement_map_) - , context(context_) - {} - - void visitImpl(QueryTreeNodePtr & node) - { - while (true) - { - /// Apply replacement transitively - auto it = replacement_map.find(node); - if (it != replacement_map.end()) - node = it->second; - else - break; - } - - if (auto * function_node = node->as(); function_node && function_node->isResolved()) - rerunFunctionResolve(function_node, context); - } - - /// We want to re-run resolve for function _after_ its arguments are replaced - bool shouldTraverseTopToBottom() const { return false; } - - bool needChildVisit(QueryTreeNodePtr & /* parent */, QueryTreeNodePtr & child) - { - /// Visit only expressions, but not subqueries - return child->getNodeType() == QueryTreeNodeType::IDENTIFIER - || child->getNodeType() == QueryTreeNodeType::LIST - || child->getNodeType() == QueryTreeNodeType::FUNCTION - || child->getNodeType() == QueryTreeNodeType::COLUMN; - } - -private: - const QueryTreeNodePtrWithHashMap & replacement_map; - const ContextPtr & context; -}; - /** Resolve query. * This function modifies query node during resolve. It is caller responsibility to clone query node before resolve * if it is needed for later use. @@ -7948,7 +7973,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * The expression `a + 1 as b` in the projection and in PREWHERE should have different `a`. */ prewhere_node = prewhere_node->clone(); - ReplaceColumnsVisitor replace_visitor(scope.join_columns_with_cahnged_types, scope.context); + ReplaceColumnsVisitor replace_visitor(scope.join_columns_with_changed_types, scope.context); replace_visitor.visit(prewhere_node); } From ee24aa6a520a711b3fabc7a51a07f56ef8c66203 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 24 Mar 2024 08:15:34 +0000 Subject: [PATCH 059/154] bump From 0e7bedc671a34a6fa5f46304ecbe94e165ddd4bb Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 24 Mar 2024 17:30:51 +0000 Subject: [PATCH 060/154] Fix resolvedIdenfiersFromJoinAreEquals function visibility --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b1b70c8ef51..569bdd7bb4f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3368,7 +3368,7 @@ private: }; /// Compare resolved identifiers considering columns that become nullable after JOIN -static bool resolvedIdenfiersFromJoinAreEquals( +bool resolvedIdenfiersFromJoinAreEquals( const QueryTreeNodePtr & left_resolved_identifier, const QueryTreeNodePtr & right_resolved_identifier, const IdentifierResolveScope & scope) From 67d39a22acbb91ed5e8aa022f1d6622eb4a7366f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 24 Mar 2024 21:06:13 +0000 Subject: [PATCH 061/154] Fix cluster.py --- tests/integration/helpers/cluster.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6f8040afcb5..445ef92549a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4397,10 +4397,7 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if ( - os.environ.get("CLICKHOUSE_USE_OLD_ANALYZER") is not None - or self.use_old_analyzer - ): + if self.use_old_analyzer: write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): From 65650809b4dc5e8888c5529f7f85828d1b60c5b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 23:11:35 +0100 Subject: [PATCH 062/154] Double precision of geoDistance if the arguments are Float64 --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Functions/greatCircleDistance.cpp | 366 ++++++++++++++------------ src/IO/VarInt.h | 6 +- 4 files changed, 201 insertions(+), 173 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9b7c81ffd17..57a72799c23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -896,6 +896,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ + M(Bool, geo_distance_returns_float64_on_float64_arguments, true, "If all four arguments to `geoDistance`, `greatCircleDistance`, `greatCircleAngle` functions are Float64, return Float64 and use double precision for internal calculations. In previous ClickHouse versions, the functions always returned Float32.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 9bf0288ebc7..a2d947aa44b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -113,6 +113,7 @@ static std::map sett {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, + {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index d1d1a101187..9d97f8560a8 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -42,121 +42,6 @@ namespace ErrorCodes namespace { -constexpr double PI = std::numbers::pi_v; -constexpr float PI_F = std::numbers::pi_v; - -constexpr float RAD_IN_DEG = static_cast(PI / 180.0); -constexpr float RAD_IN_DEG_HALF = static_cast(PI / 360.0); - -constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% -constexpr float COS_LUT_SIZE_F = 1024.0f; // maxerr 0.00063% -constexpr size_t ASIN_SQRT_LUT_SIZE = 512; -constexpr size_t METRIC_LUT_SIZE = 1024; - -/** Earth radius in meters using WGS84 authalic radius. - * We use this value to be consistent with H3 library. - */ -constexpr float EARTH_RADIUS = 6371007.180918475f; -constexpr float EARTH_DIAMETER = 2 * EARTH_RADIUS; - - -float cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table -float asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table - -float sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude -float sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude -float wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude - - -inline double sqr(double v) -{ - return v * v; -} - -inline float sqrf(float v) -{ - return v * v; -} - -void geodistInit() -{ - for (size_t i = 0; i <= COS_LUT_SIZE; ++i) - cos_lut[i] = static_cast(cos(2 * PI * i / COS_LUT_SIZE)); // [0, 2 * pi] -> [0, COS_LUT_SIZE] - - for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) - asin_sqrt_lut[i] = static_cast(asin( - sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE))); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] - - for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) - { - double latitude = i * (PI / METRIC_LUT_SIZE) - PI * 0.5; // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] - - /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), - /// depending on the latitude (in radians). - - /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 - wgs84_metric_meters_lut[i * 2] = static_cast(sqr(111132.09 - 566.05 * cos(2 * latitude) + 1.20 * cos(4 * latitude))); - wgs84_metric_meters_lut[i * 2 + 1] = static_cast(sqr(111415.13 * cos(latitude) - 94.55 * cos(3 * latitude) + 0.12 * cos(5 * latitude))); - - sphere_metric_meters_lut[i] = static_cast(sqr((EARTH_DIAMETER * PI / 360) * cos(latitude))); - - sphere_metric_lut[i] = static_cast(sqr(cos(latitude))); - } -} - -inline NO_SANITIZE_UNDEFINED size_t floatToIndex(float x) -{ - /// Implementation specific behaviour on overflow or infinite value. - return static_cast(x); -} - -inline float geodistDegDiff(float f) -{ - f = fabsf(f); - if (f > 180) - f = 360 - f; - return f; -} - -inline float geodistFastCos(float x) -{ - float y = fabsf(x) * (COS_LUT_SIZE_F / PI_F / 2.0f); - size_t i = floatToIndex(y); - y -= i; - i &= (COS_LUT_SIZE - 1); - return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; -} - -inline float geodistFastSin(float x) -{ - float y = fabsf(x) * (COS_LUT_SIZE_F / PI_F / 2.0f); - size_t i = floatToIndex(y); - y -= i; - i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 - return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; -} - -/// fast implementation of asin(sqrt(x)) -/// max error in floats 0.00369%, in doubles 0.00072% -inline float geodistFastAsinSqrt(float x) -{ - if (x < 0.122f) - { - // distance under 4546 km, Taylor error under 0.00072% - float y = sqrtf(x); - return y + x * y * 0.166666666666666f + x * x * y * 0.075f + x * x * x * y * 0.044642857142857f; - } - if (x < 0.948f) - { - // distance under 17083 km, 512-entry LUT error under 0.00072% - x *= ASIN_SQRT_LUT_SIZE; - size_t i = floatToIndex(x); - return asin_sqrt_lut[i] + (asin_sqrt_lut[i + 1] - asin_sqrt_lut[i]) * (x - i); - } - return asinf(sqrtf(x)); // distance over 17083 km, just compute exact -} - - enum class Method { SPHERE_DEGREES, @@ -164,18 +49,123 @@ enum class Method WGS84_METERS, }; -} +constexpr size_t ASIN_SQRT_LUT_SIZE = 512; +constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% +constexpr size_t METRIC_LUT_SIZE = 1024; + +template +struct Impl +{ + static constexpr T PI = std::numbers::pi_v; + static constexpr T RAD_IN_DEG = static_cast(PI / T(180.0)); + static constexpr T RAD_IN_DEG_HALF = static_cast(PI / T(360.0)); + + static constexpr T COS_LUT_SIZE_F = T(1024.0); + + /** Earth radius in meters using WGS84 authalic radius. + * We use this value to be consistent with H3 library. + */ + static constexpr T EARTH_RADIUS = T(6371007.180918475); + static constexpr T EARTH_DIAMETER = 2 * EARTH_RADIUS; + + T cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table + T asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table + + T sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude + T sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude + T wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude + + static T sqr(T v) { return v * v; } + + Impl() + { + for (size_t i = 0; i <= COS_LUT_SIZE; ++i) + cos_lut[i] = std::cos(2 * PI * i / COS_LUT_SIZE); // [0, 2 * pi] -> [0, COS_LUT_SIZE] + + for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) + asin_sqrt_lut[i] = std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE)); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] + + for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) + { + T latitude = i * (PI / METRIC_LUT_SIZE) - PI * T(0.5); // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] + + /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), + /// depending on the latitude (in radians). + + /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 + wgs84_metric_meters_lut[i * 2] = sqr(T(111132.09) - T(566.05) * std::cos(T(2.0) * latitude) + T(1.20) * std::cos(T(4.0) * latitude)); + wgs84_metric_meters_lut[i * 2 + 1] = sqr(T(111415.13) * std::cos(latitude) - T(94.55) * std::cos(T(3.0) * latitude) + T(0.12) * std::cos(T(5.0) * latitude)); + sphere_metric_meters_lut[i] = sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude)); + + sphere_metric_lut[i] = sqr(std::cos(latitude)); + } + } + + static inline NO_SANITIZE_UNDEFINED size_t toIndex(T x) + { + /// Implementation specific behaviour on overflow or infinite value. + return static_cast(x); + } + + static inline T degDiff(T f) + { + f = std::abs(f); + if (f > 180) + f = 360 - f; + return f; + } + + inline T fastCos(T x) + { + T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + size_t i = toIndex(y); + y -= i; + i &= (COS_LUT_SIZE - 1); + return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; + } + + inline T fastSin(T x) + { + T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + size_t i = toIndex(y); + y -= i; + i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 + return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; + } + + /// fast implementation of asin(sqrt(x)) + /// max error in floats 0.00369%, in doubles 0.00072% + inline T fastAsinSqrt(T x) + { + if (x < T(0.122)) + { + // distance under 4546 km, Taylor error under 0.00072% + T y = std::sqrt(x); + return y + x * y * T(0.166666666666666) + x * x * y * T(0.075) + x * x * x * y * T(0.044642857142857); + } + if (x < T(0.948)) + { + // distance under 17083 km, 512-entry LUT error under 0.00072% + x *= ASIN_SQRT_LUT_SIZE; + size_t i = toIndex(x); + return asin_sqrt_lut[i] + (asin_sqrt_lut[i + 1] - asin_sqrt_lut[i]) * (x - i); + } + return std::asin(std::sqrt(x)); /// distance is over 17083 km, just compute exact + } +}; + +template Impl impl; DECLARE_MULTITARGET_CODE( namespace { -template -float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) +template +T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) { - float lat_diff = geodistDegDiff(lat1deg - lat2deg); - float lon_diff = geodistDegDiff(lon1deg - lon2deg); + T lat_diff = impl.degDiff(lat1deg - lat2deg); + T lon_diff = impl.degDiff(lon1deg - lon2deg); if (lon_diff < 13) { @@ -187,51 +177,51 @@ float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) /// (Remember how a plane flies from Amsterdam to New York) /// But if longitude is close but latitude is different enough, there is no difference between meridian and great circle line. - float latitude_midpoint = (lat1deg + lat2deg + 180) * METRIC_LUT_SIZE / 360; // [-90, 90] degrees -> [0, METRIC_LUT_SIZE] indexes - size_t latitude_midpoint_index = floatToIndex(latitude_midpoint) & (METRIC_LUT_SIZE - 1); + T latitude_midpoint = (lat1deg + lat2deg + 180) * METRIC_LUT_SIZE / 360; // [-90, 90] degrees -> [0, METRIC_LUT_SIZE] indexes + size_t latitude_midpoint_index = impl.toIndex(latitude_midpoint) & (METRIC_LUT_SIZE - 1); /// This is linear interpolation between two table items at index "latitude_midpoint_index" and "latitude_midpoint_index + 1". - float k_lat{}; - float k_lon{}; + T k_lat{}; + T k_lon{}; if constexpr (method == Method::SPHERE_DEGREES) { k_lat = 1; - k_lon = sphere_metric_lut[latitude_midpoint_index] - + (sphere_metric_lut[latitude_midpoint_index + 1] - sphere_metric_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.sphere_metric_lut[latitude_midpoint_index] + + (impl.sphere_metric_lut[latitude_midpoint_index + 1] - impl.sphere_metric_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); } else if constexpr (method == Method::SPHERE_METERS) { - k_lat = sqrf(EARTH_DIAMETER * PI_F / 360.0f); + k_lat = impl.sqr(impl.EARTH_DIAMETER * impl.PI / T(360.0)); - k_lon = sphere_metric_meters_lut[latitude_midpoint_index] - + (sphere_metric_meters_lut[latitude_midpoint_index + 1] - sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.sphere_metric_meters_lut[latitude_midpoint_index] + + (impl.sphere_metric_meters_lut[latitude_midpoint_index + 1] - impl.sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); } else if constexpr (method == Method::WGS84_METERS) { - k_lat = wgs84_metric_meters_lut[latitude_midpoint_index * 2] - + (wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2] - wgs84_metric_meters_lut[latitude_midpoint_index * 2]) * (latitude_midpoint - latitude_midpoint_index); + k_lat = impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2] + + (impl.wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2] - impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2]) * (latitude_midpoint - latitude_midpoint_index); - k_lon = wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1] - + (wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2 + 1] - wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1] + + (impl.wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2 + 1] - impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1]) * (latitude_midpoint - latitude_midpoint_index); } /// Metric on a tangent plane: it differs from Euclidean metric only by scale of coordinates. - return sqrtf(k_lat * lat_diff * lat_diff + k_lon * lon_diff * lon_diff); + return std::sqrt(k_lat * lat_diff * lat_diff + k_lon * lon_diff * lon_diff); } else { // points too far away; use haversine - float a = sqrf(geodistFastSin(lat_diff * RAD_IN_DEG_HALF)) - + geodistFastCos(lat1deg * RAD_IN_DEG) * geodistFastCos(lat2deg * RAD_IN_DEG) * sqrf(geodistFastSin(lon_diff * RAD_IN_DEG_HALF)); + T a = impl.sqr(impl.fastSin(lat_diff * impl.RAD_IN_DEG_HALF)) + + impl.fastCos(lat1deg * impl.RAD_IN_DEG) * impl.fastCos(lat2deg * impl.RAD_IN_DEG) * impl.sqr(impl.fastSin(lon_diff * impl.RAD_IN_DEG_HALF)); if constexpr (method == Method::SPHERE_DEGREES) - return (360.0f / PI_F) * geodistFastAsinSqrt(a); + return (T(360.0) / impl.PI) * impl.fastAsinSqrt(a); else - return EARTH_DIAMETER * geodistFastAsinSqrt(a); + return impl.EARTH_DIAMETER * impl.fastAsinSqrt(a); } } @@ -241,13 +231,24 @@ template class FunctionGeoDistance : public IFunction { public: - static constexpr auto name = - (method == Method::SPHERE_DEGREES) ? "greatCircleAngle" - : ((method == Method::SPHERE_METERS) ? "greatCircleDistance" - : "geoDistance"); + FunctionGeoDistance(ContextPtr context) + { + always_float32 = !context->getSettingsRef().geo_distance_returns_float64_on_float64_arguments; + } private: - String getName() const override { return name; } + bool always_float32; + + String getName() const override + { + if constexpr (method == Method::SPHERE_DEGREES) + return "greatCircleAngle"; + if constexpr (method == Method::SPHERE_METERS) + return "greatCircleDistance"; + else + return "geoDistance"; + } + size_t getNumberOfArguments() const override { return 4; } bool useDefaultImplementationForConstants() const override { return true; } @@ -255,22 +256,31 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - for (const auto arg_idx : collections::range(0, arguments.size())) + bool has_float64 = false; + + for (size_t arg_idx = 0; arg_idx < 4; ++arg_idx) { - const auto * arg = arguments[arg_idx].get(); - if (!isNumber(WhichDataType(arg))) + WhichDataType which(arguments[arg_idx]); + + if (!isNumber(which)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument {} of function {}. " - "Must be numeric", arg->getName(), std::to_string(arg_idx + 1), getName()); + "Must be numeric", arguments[arg_idx]->getName(), std::to_string(arg_idx + 1), getName()); + + if (which.isFloat64()) + has_float64 = true; } - return std::make_shared(); + if (has_float64 && !always_float32) + return std::make_shared(); + else + return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - auto dst = ColumnVector::create(); - auto & dst_data = dst->getData(); - dst_data.resize(input_rows_count); + bool returns_float64 = WhichDataType(result_type).isFloat64(); + + auto dst = result_type->createColumn(); auto arguments_copy = arguments; for (auto & argument : arguments_copy) @@ -280,10 +290,24 @@ private: argument.type = result_type; } - const auto * col_lon1 = convertArgumentColumnToFloat32(arguments_copy, 0); - const auto * col_lat1 = convertArgumentColumnToFloat32(arguments_copy, 1); - const auto * col_lon2 = convertArgumentColumnToFloat32(arguments_copy, 2); - const auto * col_lat2 = convertArgumentColumnToFloat32(arguments_copy, 3); + if (returns_float64) + run(arguments_copy, dst, input_rows_count); + else + run(arguments_copy, dst, input_rows_count); + + return dst; + } + + template + void run(const ColumnsWithTypeAndName & arguments, MutableColumnPtr & dst, size_t input_rows_count) const + { + const auto * col_lon1 = convertArgumentColumn(arguments, 0); + const auto * col_lat1 = convertArgumentColumn(arguments, 1); + const auto * col_lon2 = convertArgumentColumn(arguments, 2); + const auto * col_lat2 = convertArgumentColumn(arguments, 3); + + auto & dst_data = assert_cast &>(*dst).getData(); + dst_data.resize(input_rows_count); for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { @@ -291,20 +315,20 @@ private: col_lon1->getData()[row_num], col_lat1->getData()[row_num], col_lon2->getData()[row_num], col_lat2->getData()[row_num]); } - - return dst; } - const ColumnFloat32 * convertArgumentColumnToFloat32(const ColumnsWithTypeAndName & arguments, size_t argument_index) const + template + const ColumnVector * convertArgumentColumn(const ColumnsWithTypeAndName & arguments, size_t argument_index) const { - const auto * column_typed = checkAndGetColumn(arguments[argument_index].column.get()); + const auto * column_typed = checkAndGetColumn>(arguments[argument_index].column.get()); if (!column_typed) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal type {} of argument {} of function {}. Must be Float32.", + "Illegal type {} of argument {} of function {}. Must be {}.", arguments[argument_index].type->getName(), argument_index + 1, - getName()); + getName(), + TypeName); return column_typed; } @@ -316,18 +340,19 @@ template class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance { public: - explicit FunctionGeoDistance(ContextPtr context) : selector(context) + explicit FunctionGeoDistance(ContextPtr context) + : TargetSpecific::Default::FunctionGeoDistance(context), selector(context) { selector.registerImplementation>(); + TargetSpecific::Default::FunctionGeoDistance>(context); #if USE_MULTITARGET_CODE selector.registerImplementation>(); + TargetSpecific::AVX::FunctionGeoDistance>(context); selector.registerImplementation>(); + TargetSpecific::AVX2::FunctionGeoDistance>(context); selector.registerImplementation>(); + TargetSpecific::AVX512F::FunctionGeoDistance>(context); #endif } @@ -345,12 +370,13 @@ private: ImplementationSelector selector; }; +} + REGISTER_FUNCTION(GeoDistance) { - geodistInit(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction("greatCircleAngle", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); + factory.registerFunction("greatCircleDistance", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); + factory.registerFunction("geoDistance", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); } } diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 12a86ac55f6..9e72705341d 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -79,7 +79,7 @@ inline char * writeVarInt(Int64 x, char * ostr) return writeVarUInt(static_cast((x << 1) ^ (x >> 63)), ostr); } -namespace impl +namespace varint_impl { template @@ -106,8 +106,8 @@ inline void readVarUInt(UInt64 & x, ReadBuffer & istr) inline void readVarUInt(UInt64 & x, ReadBuffer & istr) { if (istr.buffer().end() - istr.position() >= 10) - return impl::readVarUInt(x, istr); - return impl::readVarUInt(x, istr); + return varint_impl::readVarUInt(x, istr); + return varint_impl::readVarUInt(x, istr); } inline void readVarUInt(UInt64 & x, std::istream & istr) From a2a90467ed6f0dac325c39bc0255412aa2855e4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:11:19 +0100 Subject: [PATCH 063/154] Better --- src/Functions/greatCircleDistance.cpp | 57 +++++++++---------- .../0_stateless/01043_geo_distance.sql | 2 + .../0_stateless/01678_great_circle_angle.sql | 2 + 3 files changed, 31 insertions(+), 30 deletions(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 9d97f8560a8..0f39cc648dd 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -53,51 +53,45 @@ constexpr size_t ASIN_SQRT_LUT_SIZE = 512; constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% constexpr size_t METRIC_LUT_SIZE = 1024; +/// Earth radius in meters using WGS84 authalic radius. +/// We use this value to be consistent with H3 library. +constexpr double EARTH_RADIUS = 6371007.180918475; +constexpr double EARTH_DIAMETER = 2.0 * EARTH_RADIUS; +constexpr double PI = std::numbers::pi_v; + +template +T sqr(T v) { return v * v; } + template struct Impl { - static constexpr T PI = std::numbers::pi_v; - static constexpr T RAD_IN_DEG = static_cast(PI / T(180.0)); - static constexpr T RAD_IN_DEG_HALF = static_cast(PI / T(360.0)); - - static constexpr T COS_LUT_SIZE_F = T(1024.0); - - /** Earth radius in meters using WGS84 authalic radius. - * We use this value to be consistent with H3 library. - */ - static constexpr T EARTH_RADIUS = T(6371007.180918475); - static constexpr T EARTH_DIAMETER = 2 * EARTH_RADIUS; - T cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table T asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table - T sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude T sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude T wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude - static T sqr(T v) { return v * v; } - Impl() { for (size_t i = 0; i <= COS_LUT_SIZE; ++i) - cos_lut[i] = std::cos(2 * PI * i / COS_LUT_SIZE); // [0, 2 * pi] -> [0, COS_LUT_SIZE] + cos_lut[i] = T(std::cos(2 * PI * static_cast(i) / COS_LUT_SIZE)); // [0, 2 * pi] -> [0, COS_LUT_SIZE] for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) - asin_sqrt_lut[i] = std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE)); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] + asin_sqrt_lut[i] = T(std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE))); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) { - T latitude = i * (PI / METRIC_LUT_SIZE) - PI * T(0.5); // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] + double latitude = i * (PI / METRIC_LUT_SIZE) - PI * 0.5; // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), /// depending on the latitude (in radians). /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 - wgs84_metric_meters_lut[i * 2] = sqr(T(111132.09) - T(566.05) * std::cos(T(2.0) * latitude) + T(1.20) * std::cos(T(4.0) * latitude)); - wgs84_metric_meters_lut[i * 2 + 1] = sqr(T(111415.13) * std::cos(latitude) - T(94.55) * std::cos(T(3.0) * latitude) + T(0.12) * std::cos(T(5.0) * latitude)); - sphere_metric_meters_lut[i] = sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude)); + wgs84_metric_meters_lut[i * 2] = T(sqr(111132.09 - 566.05 * std::cos(2.0 * latitude) + 1.20 * std::cos(4.0 * latitude))); + wgs84_metric_meters_lut[i * 2 + 1] = T(sqr(111415.13 * std::cos(latitude) - 94.55 * std::cos(3.0 * latitude) + 0.12 * std::cos(5.0 * latitude))); + sphere_metric_meters_lut[i] = T(sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude))); - sphere_metric_lut[i] = sqr(std::cos(latitude)); + sphere_metric_lut[i] = T(sqr(std::cos(latitude))); } } @@ -117,7 +111,7 @@ struct Impl inline T fastCos(T x) { - T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + T y = std::abs(x) * (T(COS_LUT_SIZE) / T(PI) / T(2.0)); size_t i = toIndex(y); y -= i; i &= (COS_LUT_SIZE - 1); @@ -126,7 +120,7 @@ struct Impl inline T fastSin(T x) { - T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + T y = std::abs(x) * (T(COS_LUT_SIZE) / T(PI) / T(2.0)); size_t i = toIndex(y); y -= i; i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 @@ -194,7 +188,7 @@ T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) } else if constexpr (method == Method::SPHERE_METERS) { - k_lat = impl.sqr(impl.EARTH_DIAMETER * impl.PI / T(360.0)); + k_lat = sqr(T(EARTH_DIAMETER) * T(PI) / T(360.0)); k_lon = impl.sphere_metric_meters_lut[latitude_midpoint_index] + (impl.sphere_metric_meters_lut[latitude_midpoint_index + 1] - impl.sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); @@ -213,15 +207,18 @@ T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) } else { - // points too far away; use haversine + /// Points are too far away: use Haversine. - T a = impl.sqr(impl.fastSin(lat_diff * impl.RAD_IN_DEG_HALF)) - + impl.fastCos(lat1deg * impl.RAD_IN_DEG) * impl.fastCos(lat2deg * impl.RAD_IN_DEG) * impl.sqr(impl.fastSin(lon_diff * impl.RAD_IN_DEG_HALF)); + static constexpr T RAD_IN_DEG = T(PI / 180.0); + static constexpr T RAD_IN_DEG_HALF = T(PI / 360.0); + + T a = sqr(impl.fastSin(lat_diff * RAD_IN_DEG_HALF)) + + impl.fastCos(lat1deg * RAD_IN_DEG) * impl.fastCos(lat2deg * RAD_IN_DEG) * sqr(impl.fastSin(lon_diff * RAD_IN_DEG_HALF)); if constexpr (method == Method::SPHERE_DEGREES) - return (T(360.0) / impl.PI) * impl.fastAsinSqrt(a); + return (T(360.0) / T(PI)) * impl.fastAsinSqrt(a); else - return impl.EARTH_DIAMETER * impl.fastAsinSqrt(a); + return T(EARTH_DIAMETER) * impl.fastAsinSqrt(a); } } diff --git a/tests/queries/0_stateless/01043_geo_distance.sql b/tests/queries/0_stateless/01043_geo_distance.sql index c1fb29b9eb7..5897e1fae9b 100644 --- a/tests/queries/0_stateless/01043_geo_distance.sql +++ b/tests/queries/0_stateless/01043_geo_distance.sql @@ -1,3 +1,5 @@ +SET geo_distance_returns_float64_on_float64_arguments = 0; + SELECT greatCircleDistance(0., 0., 0., 1.); SELECT greatCircleDistance(0., 89., 0, 90.); diff --git a/tests/queries/0_stateless/01678_great_circle_angle.sql b/tests/queries/0_stateless/01678_great_circle_angle.sql index 124c7bfadf2..bcf0d751e1c 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.sql +++ b/tests/queries/0_stateless/01678_great_circle_angle.sql @@ -1,3 +1,5 @@ +SET geo_distance_returns_float64_on_float64_arguments = 0; + SELECT round(greatCircleAngle(0, 45, 0.1, 45.1), 4); SELECT round(greatCircleAngle(0, 45, 1, 45), 4); SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); From c7a48f601ae0c822371f38010543ee7f297e6142 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:12:11 +0100 Subject: [PATCH 064/154] More tests --- src/Functions/greatCircleDistance.cpp | 1 - .../0_stateless/01043_geo_distance.reference | 8 ++++++++ tests/queries/0_stateless/01043_geo_distance.sql | 14 ++++++++++++++ .../0_stateless/01678_great_circle_angle.reference | 5 +++++ .../0_stateless/01678_great_circle_angle.sql | 9 +++++++++ 5 files changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 0f39cc648dd..1b5fe1ac35b 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include diff --git a/tests/queries/0_stateless/01043_geo_distance.reference b/tests/queries/0_stateless/01043_geo_distance.reference index cd8a8a6dfe9..33e1285872e 100644 --- a/tests/queries/0_stateless/01043_geo_distance.reference +++ b/tests/queries/0_stateless/01043_geo_distance.reference @@ -6,3 +6,11 @@ 10007555 10007554 10001780 +111195.05197522942 +111195.05197522942 +110567.32686882635 +111699.2516454354 +10007554.677770648 +10007554.677770648 +10007554.677770648 +10001780.1 diff --git a/tests/queries/0_stateless/01043_geo_distance.sql b/tests/queries/0_stateless/01043_geo_distance.sql index 5897e1fae9b..cf877d05b9f 100644 --- a/tests/queries/0_stateless/01043_geo_distance.sql +++ b/tests/queries/0_stateless/01043_geo_distance.sql @@ -11,3 +11,17 @@ SELECT greatCircleDistance(0., 0., 0., 90.); SELECT geoDistance(0., 0., 90., 0.); SELECT geoDistance(0., 0., 0., 90.); + +SET geo_distance_returns_float64_on_float64_arguments = 1; + +SELECT greatCircleDistance(0., 0., 0., 1.); +SELECT greatCircleDistance(0., 89., 0, 90.); + +SELECT geoDistance(0., 0., 0., 1.); +SELECT geoDistance(0., 89., 0., 90.); + +SELECT greatCircleDistance(0., 0., 90., 0.); +SELECT greatCircleDistance(0., 0., 0., 90.); + +SELECT geoDistance(0., 0., 90., 0.); +SELECT geoDistance(0., 0., 0., 90.); diff --git a/tests/queries/0_stateless/01678_great_circle_angle.reference b/tests/queries/0_stateless/01678_great_circle_angle.reference index f3382476d4a..a409e1d84b8 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.reference +++ b/tests/queries/0_stateless/01678_great_circle_angle.reference @@ -3,3 +3,8 @@ 0.7135 10007555 10007554 +0.1224 +0.7071 +0.7135 +10007555 +10007554 diff --git a/tests/queries/0_stateless/01678_great_circle_angle.sql b/tests/queries/0_stateless/01678_great_circle_angle.sql index bcf0d751e1c..595622822f1 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.sql +++ b/tests/queries/0_stateless/01678_great_circle_angle.sql @@ -6,3 +6,12 @@ SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); SELECT round(greatCircleDistance(0, 0, 0, 90), 4); SELECT round(greatCircleDistance(0, 0, 90, 0), 4); + +SET geo_distance_returns_float64_on_float64_arguments = 1; + +SELECT round(greatCircleAngle(0, 45, 0.1, 45.1), 4); +SELECT round(greatCircleAngle(0, 45, 1, 45), 4); +SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); + +SELECT round(greatCircleDistance(0, 0, 0, 90), 4); +SELECT round(greatCircleDistance(0, 0, 90, 0), 4); From a8866dbf74f9e18398432ec9de3383f35ba6b8e9 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Mon, 25 Mar 2024 16:12:16 +0800 Subject: [PATCH 065/154] fix test failure --- tests/queries/0_stateless/00309_formats_case_insensitive.sql | 2 ++ tests/queries/0_stateless/02977_csv_format_support_tuple.sql | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00309_formats_case_insensitive.sql b/tests/queries/0_stateless/00309_formats_case_insensitive.sql index b4037ed9861..0b45e9c22a9 100644 --- a/tests/queries/0_stateless/00309_formats_case_insensitive.sql +++ b/tests/queries/0_stateless/00309_formats_case_insensitive.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + SELECT '-- test FORMAT clause --'; SET output_format_write_statistics = 0; SELECT number, 'Hello & world' FROM numbers(3) FORMAT Tsv; diff --git a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql index 40adf2cf0cb..d00cc00e097 100644 --- a/tests/queries/0_stateless/02977_csv_format_support_tuple.sql +++ b/tests/queries/0_stateless/02977_csv_format_support_tuple.sql @@ -1,5 +1,5 @@ -- Tags: no-parallel -insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)); +insert into function file('02977_1.csv') select '20240305', 1, ['s', 'd'], map('a', 2), tuple('222', 33, map('abc', 5)) SETTINGS engine_file_truncate_on_insert=1; desc file('02977_1.csv'); select * from file('02977_1.csv') settings max_threads=1; From 80a7cda59a0cdd9576cf52c9eb13904726802081 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 25 Mar 2024 09:21:45 +0000 Subject: [PATCH 066/154] Fix style --- src/Storages/System/StorageSystemPartsBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index f2b1e1222c3..b1ea2dd3f2b 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -24,7 +24,8 @@ #include #include -namespace { +namespace +{ constexpr auto * database_column_name = "database"; constexpr auto * table_column_name = "table"; constexpr auto * engine_column_name = "engine"; From 1bfd588035a1be6aa9a0ddd68236e39c585f0d87 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Mar 2024 11:00:50 +0100 Subject: [PATCH 067/154] Remove PoolBase::AllocateNewBypassingPool The code that it uses had been removed in #58845. Introduced in #49732 Signed-off-by: Azat Khuzhin --- src/Common/PoolBase.h | 87 ++++++++++++++----------------------------- 1 file changed, 28 insertions(+), 59 deletions(-) diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index ef35002c45a..d6fc1656eca 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -1,11 +1,9 @@ #pragma once -#include #include -#include -#include -#include +#include #include +#include #include #include @@ -17,6 +15,14 @@ namespace ProfileEvents extern const Event ConnectionPoolIsFullMicroseconds; } +namespace DB +{ + namespace ErrorCodes + { + extern const int LOGICAL_ERROR; + } +} + /** A class from which you can inherit and get a pool of something. Used for database connection pools. * Descendant class must provide a method for creating a new object to place in the pool. */ @@ -29,22 +35,6 @@ public: using ObjectPtr = std::shared_ptr; using Ptr = std::shared_ptr>; - enum class BehaviourOnLimit - { - /** - * Default behaviour - when limit on pool size is reached, callers will wait until object will be returned back in pool. - */ - Wait, - - /** - * If no free objects in pool - allocate a new object, but not store it in pool. - * This behaviour is needed when we simply don't want to waste time waiting or if we cannot guarantee that query could be processed using fixed amount of connections. - * For example, when we read from table on s3, one GetObject request corresponds to the whole FileSystemCache segment. This segments are shared between different - * reading tasks, so in general case connection could be taken from pool by one task and returned back by another one. And these tasks are processed completely independently. - */ - AllocateNewBypassingPool, - }; - private: /** The object with the flag, whether it is currently used. */ @@ -99,53 +89,37 @@ public: Object & operator*() && = delete; const Object & operator*() const && = delete; - Object * operator->() & { return castToObjectPtr(); } - const Object * operator->() const & { return castToObjectPtr(); } - Object & operator*() & { return *castToObjectPtr(); } - const Object & operator*() const & { return *castToObjectPtr(); } + Object * operator->() & { return &*data->data.object; } + const Object * operator->() const & { return &*data->data.object; } + Object & operator*() & { return *data->data.object; } + const Object & operator*() const & { return *data->data.object; } /** * Expire an object to make it reallocated later. */ void expire() { - if (data.index() == 1) - std::get<1>(data)->data.is_expired = true; + data->data.is_expired = true; } - bool isNull() const { return data.index() == 0 ? !std::get<0>(data) : !std::get<1>(data); } + bool isNull() const { return data == nullptr; } + + PoolBase * getPool() const + { + if (!data) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Attempt to get pool from uninitialized entry"); + return &data->data.pool; + } private: - /** - * Plain object will be stored instead of PoolEntryHelper if fallback was made in get() (see BehaviourOnLimit::AllocateNewBypassingPool). - */ - std::variant> data; + std::shared_ptr data; - explicit Entry(ObjectPtr && object) : data(std::move(object)) { } - - explicit Entry(PooledObject & object) : data(std::make_shared(object)) { } - - auto castToObjectPtr() const - { - return std::visit( - [](const auto & ptr) - { - using T = std::decay_t; - if constexpr (std::is_same_v) - return ptr.get(); - else - return ptr->data.object.get(); - }, - data); - } + explicit Entry(PooledObject & object) : data(std::make_shared(object)) {} }; virtual ~PoolBase() = default; - /** Allocates the object. - * If 'behaviour_on_limit' is Wait - wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. - * If 'behaviour_on_limit' is AllocateNewBypassingPool and there is no free object - a new object will be created but not stored in the pool. - */ + /** Allocates the object. Wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. */ Entry get(Poco::Timespan::TimeDiff timeout) { std::unique_lock lock(mutex); @@ -176,9 +150,6 @@ public: return Entry(*items.back()); } - if (behaviour_on_limit == BehaviourOnLimit::AllocateNewBypassingPool) - return Entry(allocObject()); - Stopwatch blocked; if (timeout < 0) { @@ -213,8 +184,6 @@ private: /** The maximum size of the pool. */ unsigned max_items; - BehaviourOnLimit behaviour_on_limit; - /** Pool. */ Objects items; @@ -225,8 +194,8 @@ private: protected: LoggerPtr log; - PoolBase(unsigned max_items_, LoggerPtr log_, BehaviourOnLimit behaviour_on_limit_ = BehaviourOnLimit::Wait) - : max_items(max_items_), behaviour_on_limit(behaviour_on_limit_), log(log_) + PoolBase(unsigned max_items_, LoggerPtr log_) + : max_items(max_items_), log(log_) { items.reserve(max_items); } From 3d63881b9927eef2e9d5f341820c1146ffd186f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Mar 2024 13:26:14 +0100 Subject: [PATCH 068/154] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 5a84a0dd8a3..16f153253d6 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -1,3 +1,4 @@ 00725_memory_tracking 01624_soft_constraints 02354_vector_search_queries +02901_parallel_replicas_rollup From 6d5f2cd7b54687ef4956e0a9dcf770b59188bc6d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Mar 2024 12:44:28 +0000 Subject: [PATCH 069/154] Cleanup --- src/Functions/identity.h | 24 +++++++----------------- src/Storages/StorageMerge.cpp | 18 ++++-------------- 2 files changed, 11 insertions(+), 31 deletions(-) diff --git a/src/Functions/identity.h b/src/Functions/identity.h index 22dcd72d6e1..3422342e20b 100644 --- a/src/Functions/identity.h +++ b/src/Functions/identity.h @@ -42,28 +42,18 @@ struct ScalarSubqueryResultName using FunctionIdentity = FunctionIdentityBase; using FunctionScalarSubqueryResult = FunctionIdentityBase; -class FunctionActionName : public IFunction +struct ActionNameName +{ + static constexpr auto name = "__actionName"; +}; + +class FunctionActionName : public FunctionIdentityBase { public: + using FunctionIdentityBase::FunctionIdentityBase; static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - static constexpr auto name = "__actionName"; - - String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - bool isSuitableForConstantFolding() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - return arguments.front(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - return arguments.front().column; - } }; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ec27d324ade..e5ee6089fa3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -901,7 +901,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!storage_snapshot_->tryGetColumn(get_column_options, "_table")) { - auto table_name_node = std::make_shared(current_storage_id.table_name); //, std::make_shared(std::make_shared())); + auto table_name_node = std::make_shared(current_storage_id.table_name); auto table_name_alias = std::make_shared("__table1._table"); auto function_node = std::make_shared("__actionName"); @@ -914,7 +914,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!storage_snapshot_->tryGetColumn(get_column_options, "_database")) { - auto database_name_node = std::make_shared(current_storage_id.database_name); //, std::make_shared(std::make_shared())); + auto database_name_node = std::make_shared(current_storage_id.database_name); auto database_name_alias = std::make_shared("__table1._database"); auto function_node = std::make_shared("__actionName"); @@ -973,15 +973,9 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo if (!column_name_to_node.empty()) { - // std::cerr << ">>>>>>>>>>>>>>>>\n"; - // std::cerr << modified_query_info.query_tree->dumpTree() << std::endl; - replaceColumns(modified_query_info.query_tree, replacement_table_expression, column_name_to_node); - - // std::cerr << "<<<<<<<<<<\n"; - // std::cerr << modified_query_info.query_tree->dumpTree() << std::endl; } modified_query_info.query = queryNodeToSelectQuery(modified_query_info.query_tree); @@ -1079,7 +1073,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( String table_column = table_alias.empty() || processed_stage == QueryProcessingStage::FetchColumns ? "_table" : table_alias + "._table"; if (has_database_virtual_column && common_header.has(database_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(database_column)) // || !pipe_header.has("'" + database_name + "'_String"))) + && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(database_column)) { ColumnWithTypeAndName column; column.name = database_column; @@ -1095,7 +1089,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( } if (has_table_virtual_column && common_header.has(table_column) - && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(table_column)) // || !pipe_header.has("'" + table_name + "'_String"))) + && storage_stage == QueryProcessingStage::FetchColumns && !pipe_header.has(table_column)) { ColumnWithTypeAndName column; column.name = table_column; @@ -1254,10 +1248,6 @@ QueryPlan ReadFromMerge::createPlanForTable( } } - // WriteBufferFromOwnString buf; - // plan.explainPlan(buf, {.header=true, .actions=true}); - // std::cerr << buf.str() << std::endl; - return plan; } From 75150033c85006ae9db3d93a304ec028d816b3ad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Mar 2024 12:45:20 +0000 Subject: [PATCH 070/154] Cleanup --- src/Storages/StorageMerge.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index e5ee6089fa3..caec03c95b3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1559,9 +1559,6 @@ void ReadFromMerge::convertAndFilterSourceStream( row_policy_data_opt->addFilterTransform(builder); } - // std::cerr << "============" << builder.getHeader().dumpStructure() << std::endl; - // std::cerr << "============" << header.dumpStructure() << std::endl; - auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), convert_actions_match_columns_mode); From 12a7f6afff6be728d3b6718148e59ae7fe7640b9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Mar 2024 21:52:17 +0100 Subject: [PATCH 071/154] Fix columns of a materialized views after MODIFY QUERY. --- src/Storages/StorageMaterializedView.cpp | 22 ++++++++++++++----- src/Storages/WindowView/StorageWindowView.cpp | 20 +++++++++++++++++ 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index bbb06e9be08..344b5dfce9b 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -70,6 +70,16 @@ static void removeNonCommonColumns(const Block & src_header, Block & target_head target_header.erase(target_only_positions); } +namespace +{ + void checkTargetTableHasQueryOutputColumns(const ColumnsDescription & target_table_columns, const ColumnsDescription & select_query_output_columns) + { + for (const auto & column : select_query_output_columns) + if (!target_table_columns.has(column.name)) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the materialized view's inner table", column.name); + } +} + StorageMaterializedView::StorageMaterializedView( const StorageID & table_id_, ContextPtr local_context, @@ -402,11 +412,13 @@ void StorageMaterializedView::alter( /// Check the materialized view's inner table structure. if (has_inner_table) { - const Block & block = InterpreterSelectWithUnionQuery::getSampleBlock(new_select.select_query, local_context); - const auto & inner_table_metadata = tryGetTargetTable()->getInMemoryMetadata().columns; - for (const auto & name : block.getNames()) - if (!inner_table_metadata.has(name)) - throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the materialized view's inner table", name); + /// If this materialized view has an inner table it should always have the same columns as this materialized view. + /// Try to find mistakes in the select query (it shouldn't have columns which are not in the inner table). + auto target_table_metadata = getTargetTable()->getInMemoryMetadataPtr(); + const auto & select_query_output_columns = new_metadata.columns; /// AlterCommands::alter() analyzed the query and assigned `new_metadata.columns` before. + checkTargetTableHasQueryOutputColumns(target_table_metadata->columns, select_query_output_columns); + /// We need to copy the target table's columns (after checkTargetTableHasQueryOutputColumns() they can be still different - e.g. the data types of those columns can differ). + new_metadata.columns = target_table_metadata->columns; } DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index b1984a947c8..0b822b9aab3 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -71,6 +71,7 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW; extern const int SUPPORT_IS_DISABLED; extern const int TABLE_WAS_NOT_DROPPED; + extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int NOT_IMPLEMENTED; extern const int UNSUPPORTED_METHOD; } @@ -339,6 +340,13 @@ namespace table_expr->children.push_back(table_expr->database_and_table_name); return fetch_query; } + + void checkTargetTableHasQueryOutputColumns(const ColumnsDescription & target_table_columns, const ColumnsDescription & select_query_output_columns) + { + for (const auto & column : select_query_output_columns) + if (!target_table_columns.has(column.name)) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "Column {} does not exist in the window view's inner table", column.name); + } } static void extractDependentTable(ContextPtr context, ASTPtr & query, String & select_database_name, String & select_table_name) @@ -482,6 +490,18 @@ void StorageWindowView::alter( new_metadata.setSelectQuery(new_select); + /// Check the window view's inner target table structure. + if (has_inner_target_table) + { + /// If this window view has an inner target table it should always have the same columns as this window view. + /// Try to find mistakes in the select query (it shouldn't have columns which are not in the inner target table). + auto target_table_metadata = getTargetTable()->getInMemoryMetadataPtr(); + const auto & select_query_output_columns = new_metadata.columns; /// AlterCommands::alter() analyzed the query and assigned `new_metadata.columns` before. + checkTargetTableHasQueryOutputColumns(target_table_metadata->columns, select_query_output_columns); + /// We need to copy the target table's columns (after checkTargetTableHasQueryOutputColumns() they can be still different - e.g. in data types). + new_metadata.columns = target_table_metadata->columns; + } + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); setInMemoryMetadata(new_metadata); From 565db532f8e7263f4eef98a4084882194a61b60e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 21 Mar 2024 21:52:27 +0100 Subject: [PATCH 072/154] Add tests. --- ...ackup_matview_after_modify_query.reference | 16 ++++ ...03001_backup_matview_after_modify_query.sh | 35 ++++++++ ...tview_columns_after_modify_query.reference | 86 +++++++++++++++++++ ...3001_matview_columns_after_modify_query.sh | 74 ++++++++++++++++ 4 files changed, 211 insertions(+) create mode 100644 tests/queries/0_stateless/03001_backup_matview_after_modify_query.reference create mode 100755 tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh create mode 100644 tests/queries/0_stateless/03001_matview_columns_after_modify_query.reference create mode 100755 tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh diff --git a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.reference b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.reference new file mode 100644 index 00000000000..f59f19d62b3 --- /dev/null +++ b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.reference @@ -0,0 +1,16 @@ +mv before: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +BACKUP_CREATED +RESTORED + +mv after: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 diff --git a/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh new file mode 100755 index 00000000000..d49f1c41c69 --- /dev/null +++ b/tests/queries/0_stateless/03001_backup_matview_after_modify_query.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env bash +# Tags: no-ordinary-database, no-replicated-database +# Tag no-ordinary-database: TO DO + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +db="$CLICKHOUSE_DATABASE" +db_2="${db}_2" +backup_name="${db}_backup" + +${CLICKHOUSE_CLIENT} --multiquery " +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS mv; +CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; +" + +${CLICKHOUSE_CLIENT} -q "CREATE MATERIALIZED VIEW mv(timestamp DateTime, c12 Nullable(String)) ENGINE=MergeTree ORDER BY timestamp AS SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO src SELECT '2024-02-22'::DateTime + number, number, number FROM numbers(3)" + +echo 'mv before:' +${CLICKHOUSE_CLIENT} -q "SELECT * FROM ${db}.mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes" + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src" + +echo +${CLICKHOUSE_CLIENT} -q "BACKUP DATABASE $db TO Disk('backups', '${backup_name}')" | grep -o "BACKUP_CREATED" +${CLICKHOUSE_CLIENT} -q "RESTORE DATABASE $db AS ${db_2} FROM Disk('backups', '${backup_name}')" | grep -o "RESTORED" + +echo $'\nmv after:' +${CLICKHOUSE_CLIENT} -q "SELECT * FROM ${db_2}.mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes" + +${CLICKHOUSE_CLIENT} -q "DROP DATABASE ${db_2}" diff --git a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.reference b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.reference new file mode 100644 index 00000000000..c9b11e46e26 --- /dev/null +++ b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.reference @@ -0,0 +1,86 @@ +src: +Timestamp c1 c2 +DateTime64(9) String String +2024-02-22 00:00:00.000000000 0 0 +2024-02-22 00:00:01.000000000 1 1 +2024-02-22 00:00:02.000000000 2 2 + +mv: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +inner: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +Test 1. MODIFY QUERY doesn't change columns. + +mv: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +inner: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +Test 2. MODIFY QUERY with explicit data types doesn't change columns. + +mv: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +inner: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +Test 3. MODIFY QUERY can even fix wrong columns. + +Before MODIFY QUERY: + +mv: +timestamp c12 +DateTime64(9) String +2024-02-22 00:00:00.000000000 00 +2024-02-22 00:00:01.000000000 11 +2024-02-22 00:00:02.000000000 22 + +inner: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +After MODIFY QUERY: + +mv: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 + +inner: +timestamp c12 +DateTime Nullable(String) +2024-02-22 00:00:00 00 +2024-02-22 00:00:01 11 +2024-02-22 00:00:02 22 diff --git a/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh new file mode 100755 index 00000000000..2ec5832fac6 --- /dev/null +++ b/tests/queries/0_stateless/03001_matview_columns_after_modify_query.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiquery " +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS mv; +CREATE TABLE src(Timestamp DateTime64(9), c1 String, c2 String) ENGINE=MergeTree ORDER BY Timestamp; +" + +${CLICKHOUSE_CLIENT} -q "CREATE MATERIALIZED VIEW mv(timestamp DateTime, c12 Nullable(String)) ENGINE=MergeTree ORDER BY timestamp AS SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src" + +mv_uuid=$(${CLICKHOUSE_CLIENT} -q "SELECT uuid FROM system.tables WHERE table='mv' AND database=currentDatabase()") +if [ "${mv_uuid}" != "00000000-0000-0000-0000-000000000000" ]; then + inner_table_name=".inner_id.${mv_uuid}" +else + inner_table_name=".inner.mv" +fi +#echo "inner_table_name=$inner_table_name" + +${CLICKHOUSE_CLIENT} -q "INSERT INTO src SELECT '2024-02-22'::DateTime + number, number, number FROM numbers(3)" + +echo $'src:' +${CLICKHOUSE_CLIENT} -q "SELECT * FROM src ORDER BY Timestamp FORMAT TSVWithNamesAndTypes" + +function show_mv_and_inner() +{ + echo $'\nmv:' + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM mv ORDER BY timestamp FORMAT TSVWithNamesAndTypes" + + echo $'\ninner:' + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM \`$inner_table_name\` ORDER BY timestamp FORMAT TSVWithNamesAndTypes" +} + +show_mv_and_inner + +################# + +echo $'\nTest 1. MODIFY QUERY doesn\'t change columns.' + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src" + +show_mv_and_inner + +################# + +echo $'\nTest 2. MODIFY QUERY with explicit data types doesn\'t change columns.' + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp::DateTime64(9) as timestamp, (c1 || c2)::String as c12 FROM src" + +show_mv_and_inner + +################# + +echo $'\nTest 3. MODIFY QUERY can even fix wrong columns.' # We need that because of https://github.com/ClickHouse/ClickHouse/issues/60369 + +mv_metadata_path=$(${CLICKHOUSE_CLIENT} -q "SELECT metadata_path FROM system.tables WHERE table='mv' AND database=currentDatabase()") +${CLICKHOUSE_CLIENT} -q "DETACH TABLE mv" + +#cat $mv_metadata_path +sed -i -e 's/`timestamp` DateTime,/`timestamp` DateTime64(9),/g' -e 's/`c12` Nullable(String)/`c12` String/g' "$mv_metadata_path" +#cat $mv_metadata_path + +${CLICKHOUSE_CLIENT} -q "ATTACH TABLE mv" + +echo $'\nBefore MODIFY QUERY:' +show_mv_and_inner + +${CLICKHOUSE_CLIENT} -q "ALTER TABLE mv MODIFY QUERY SELECT Timestamp as timestamp, c1 || c2 as c12 FROM src" + +echo $'\nAfter MODIFY QUERY:' +show_mv_and_inner From 52bdee27046d812d465bb5e34b7f21971370e0a7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 22 Mar 2024 02:24:39 +0100 Subject: [PATCH 073/154] Update other test. --- .../0_stateless/02932_refreshable_materialized_views.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index aa76806da9d..2eb41590af1 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -4,7 +4,7 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n) <3: time difference at least> 1000 <4: next refresh in> 2 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` Int16\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src +CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled Finished 2054-01-01 00:00:00 From 38b15fb2a3fa364ef7dc66168de3d04f150e8eaf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Mar 2024 14:59:37 +0100 Subject: [PATCH 074/154] Fix 02494_zero_copy_and_projection_and_mutation_work_together.sql with shared merge tree --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 42badd15a73..3d5f9b813ef 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -815,10 +815,12 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch addProjectionPart(projection.name, std::move(part)); } } - else if (checksums.has(path)) + else if (check_consistency && checksums.has(path)) { auto part = getProjectionPartBuilder(projection.name).withPartFormatFromDisk().build(); - part->setBrokenReason("Projection directory " + path + " does not exist while loading projections", ErrorCodes::NO_FILE_IN_DATA_PART); + part->setBrokenReason( + "Projection directory " + path + " does not exist while loading projections. Stacktrace: " + StackTrace().toString(), + ErrorCodes::NO_FILE_IN_DATA_PART); addProjectionPart(projection.name, std::move(part)); has_broken_projection = true; } From 6b50f5bf1005c0a361fa584b75d677c6ca5a3c84 Mon Sep 17 00:00:00 2001 From: Pervakov Grigorii Date: Wed, 20 Mar 2024 19:27:33 +0000 Subject: [PATCH 075/154] Reload certificate chain during certificate reload --- src/Server/CertificateReloader.cpp | 20 +++++++-- src/Server/CertificateReloader.h | 2 +- .../configs/WithChain.crt | 43 +++++++++++++++++++ .../configs/WithChain.key | 27 ++++++++++++ .../test_reload_certificate/test.py | 15 +++++++ 5 files changed, 103 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_reload_certificate/configs/WithChain.crt create mode 100644 tests/integration/test_reload_certificate/configs/WithChain.key diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index c974f450c9a..c93ef380665 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -30,8 +30,22 @@ int CertificateReloader::setCertificate(SSL * ssl) if (!current) return -1; - SSL_use_certificate(ssl, const_cast(current->cert.certificate())); - SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key))); + if (current->certs_chain.size() < 1) + return -1; + + int ret; + ret = SSL_clear_chain_certs(ssl); + if (!ret) + return ret; + ret = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate())); + if (!ret) + return ret; + for (auto cert = current->certs_chain.begin() + 1; cert != current->certs_chain.end(); cert++) { + ret = SSL_add1_chain_cert(ssl, const_cast(cert->certificate())); + if (!ret) + return ret; + } + ret = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key))); int err = SSL_check_private_key(ssl); if (err != 1) @@ -100,7 +114,7 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf CertificateReloader::Data::Data(std::string cert_path, std::string key_path, std::string pass_phrase) - : cert(cert_path), key(/* public key */ "", /* private key */ key_path, pass_phrase) + : certs_chain(Poco::Crypto::X509Certificate::readPEM(cert_path)), key(/* public key */ "", /* private key */ key_path, pass_phrase) { } diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 028914e682f..5ab799037d5 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -70,7 +70,7 @@ private: struct Data { - Poco::Crypto::X509Certificate cert; + Poco::Crypto::X509Certificate::List certs_chain; Poco::Crypto::EVPPKey key; Data(std::string cert_path, std::string key_path, std::string pass_phrase); diff --git a/tests/integration/test_reload_certificate/configs/WithChain.crt b/tests/integration/test_reload_certificate/configs/WithChain.crt new file mode 100644 index 00000000000..c75ee533f26 --- /dev/null +++ b/tests/integration/test_reload_certificate/configs/WithChain.crt @@ -0,0 +1,43 @@ +-----BEGIN CERTIFICATE----- +MIIDkzCCAnugAwIBAgIUSChEeHqJus9jzKmD/L3Tw0x4OwcwDQYJKoZIhvcNAQEL +BQAwQTEaMBgGA1UEAwwRZGVtby5tbG9wc2h1Yi5jb20xCzAJBgNVBAYTAlVTMRYw +FAYDVQQHDA1TYW4gRnJhbnNpc2NvMCAXDTI0MDMyMDE5MTE1OVoYDzIxMjQwMjI1 +MTkxMTU5WjBxMQswCQYDVQQGEwJVUzETMBEGA1UECAwKQ2FsaWZvcm5pYTEWMBQG +A1UEBwwNU2FuIEZyYW5zaXNjbzEPMA0GA1UECgwGRm9vQmFyMRAwDgYDVQQLDAdG +b28gQmFyMRIwEAYDVQQDDAlsb2NhbGhvc3QwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQDM//0s6AX988J86HzxX95irGa9cJIfY24UDBXVkO3gREiwj1Uf +bpvpxCcwADMuFdggsJlppWa3q+PNJ/eoVwdl3gG0WXaZp1rcuv6ltxdQAUtgfMAb +5p7HwsO7rCTGJBwa62Jg+E79j+V8rZWfaJRfNtY0p7eauWIrqLA0Gyse+lRayPHI +hsR9+0qedF+qziFpbNxpW8DHrpIrLb8LEao1BCYQ44koBXjkrXeR6OidXw/gek8+ +9M2GLxy6ubQ7hrcYwVWpFOKLLZLmyYDgescM6AIU904o1bN0yJ5rM7a1+f150qp6 +ttZlya0sJH0lm3gzEsqb6Fbh+Dw/9Lsp66sDAgMBAAGjUTBPMB8GA1UdIwQYMBaA +FMjyDUyXujACeQa2G+4I8ic5HVBiMAkGA1UdEwQCMAAwCwYDVR0PBAQDAgTwMBQG +A1UdEQQNMAuCCWxvY2FsaG9zdDANBgkqhkiG9w0BAQsFAAOCAQEABEWEaVDqv9Za +bFpD/JEMIFVqe5ANKAnbbff0/vFJQ7yFmsL8/G4bbdd8uidRgE4WgoeNUscCnfk+ +kjb1vFjvY4/cBYITn41Pz5I7lQMH+BaR5gHb1oJVlJavQ2vsaeMuyDJaNmumejmU +YnoKZRwwb6SCXujS2MKgKl+jL5OkZk60i+nJhIXfxwMNmlvtqADSU5Z3VMagq8hj +DnEsxTz8PptuVaLVT4kcZm9gZpDEW2KPMZhNCv/g7EzQv8r3WnFGqumMGBO82ZE0 +mUh/Chrhss/meVK0FqTTBjOlex7R0GiJBCDfZGYTWIVdND4ICdZ1OpGWid5CXbfQ +sWBrbBaEyw== +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIDYzCCAkugAwIBAgIUR0PldYYSk3DAifgb600kvqjaCAIwDQYJKoZIhvcNAQEL +BQAwQTEaMBgGA1UEAwwRZGVtby5tbG9wc2h1Yi5jb20xCzAJBgNVBAYTAlVTMRYw +FAYDVQQHDA1TYW4gRnJhbnNpc2NvMB4XDTI0MDMyMDE5MTEzMVoXDTI1MDMxMTE5 +MTEzMVowQTEaMBgGA1UEAwwRZGVtby5tbG9wc2h1Yi5jb20xCzAJBgNVBAYTAlVT +MRYwFAYDVQQHDA1TYW4gRnJhbnNpc2NvMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8A +MIIBCgKCAQEAqL3k6Pexu1KR8fc84fRmu0hYon7+xOPmtFSzspeN+DJNe5oDO6x0 +RzTQkgtDoxTcq32O290r3uURnDmnvNubz5yTpM1Zcz/kuSNpHLJh4yyZsXRsB21v +lb3bhjqyn6rkfoQzIMekt7clPQS0dWdU2T+lwn6XBVShOyB/W7ysP309ofQGXV+T +VFyU+lgZc2WjK6611QDCpTXgRc/UKUfU5460BnTCylP6jzBOWBZb8FX6dYBzS4U2 +yISvOXagxJVruoWjscc35ln6HBQ8bu/fI8Q0n1/ROlm785Bsd/LpVw465kklwQwS +FY3FQkiedD1fyszXO4Yq5PARw54AGKbAyQIDAQABo1MwUTAdBgNVHQ4EFgQUyPIN +TJe6MAJ5BrYb7gjyJzkdUGIwHwYDVR0jBBgwFoAUyPINTJe6MAJ5BrYb7gjyJzkd +UGIwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAYyl27sxwrjz5 +qcqwbLYv2eFIR6VZzvHfknSK1ht1jzMkXxvAOjETbYwqwWquWwMHOD2X4q5d57nu +qYAE9YE27HFzknPQkDdzwJ0u4dKi28PK8tM6dqDK46LSal/MEUxXGuzW3TRyJXrl +lPi+Wh6gZRRiANJ+giEwvQ+8k6eURHrhtL7yZqT+swi+jP4h6S4mmHmsaOj4VoP/ +NCFoRZud5SCd7RZV+fzNfxhLHI9I2c2gFycBDZOEdlrIZHM6EoaDb3i9kDVbnZqG +Zj/+k/NwCKg5UiDap6Z7Xj7w0chSppg3DMcsxGeQ9vQcMtydNu5fSK4CozNqxObb +hGBJrQylAw== +-----END CERTIFICATE----- diff --git a/tests/integration/test_reload_certificate/configs/WithChain.key b/tests/integration/test_reload_certificate/configs/WithChain.key new file mode 100644 index 00000000000..eb55458b143 --- /dev/null +++ b/tests/integration/test_reload_certificate/configs/WithChain.key @@ -0,0 +1,27 @@ +-----BEGIN RSA PRIVATE KEY----- +MIIEpQIBAAKCAQEAzP/9LOgF/fPCfOh88V/eYqxmvXCSH2NuFAwV1ZDt4ERIsI9V +H26b6cQnMAAzLhXYILCZaaVmt6vjzSf3qFcHZd4BtFl2mada3Lr+pbcXUAFLYHzA +G+aex8LDu6wkxiQcGutiYPhO/Y/lfK2Vn2iUXzbWNKe3mrliK6iwNBsrHvpUWsjx +yIbEfftKnnRfqs4haWzcaVvAx66SKy2/CxGqNQQmEOOJKAV45K13kejonV8P4HpP +PvTNhi8curm0O4a3GMFVqRTiiy2S5smA4HrHDOgCFPdOKNWzdMieazO2tfn9edKq +erbWZcmtLCR9JZt4MxLKm+hW4fg8P/S7KeurAwIDAQABAoIBAQDIT6LtzFJ+kT+L +mgjsOocs19UUavj9XsTjJeP36UPnDhIRJB1DN2t2Astf5fIcqA+l8aoWRx6Vfop7 +nwAqJ8/8w+/acEipX5Qzdyz4dG19Kaajw4JcQP8JptBng4/zZLlpvHNG2ZslvQO6 +zksTlrbUStsIXJHVyCubCZwTbjC2dJlc97705DZAk/8L7FkljmyJws2xwQZlxdQG +FZ+8IrAqpWJC55EPfKo6+QsKEc9hh4N/MQW483V5yCuw9dLShcEMDGuxcHGFHif8 +BrwImih0rIwj9tTDY9pw6aJ5+80tVStNLDk+1eQRME6Fy/c7RG/sm/lj+P0YOI7F +jH4wyXVRAoGBAOc+rFNpM/CgRZD4VXQ+gV1+PLdaqUUU3UXAPfNWkrMIr0u4+tmg +OMMkXaM7B/ps7o5+Rcj+tO1VAvl6m7uM376szC3Ex8JA0XBDmJrJKfEQCHkzvkdf +wywVBeAR4f3D2+9Meh1XSNRqGU+Anb48neTyVYzPNIoK8ZmtZID49GfVAoGBAOLy +EX1TT1xwE/VwDy5BeJQzyZ+xwevie/960RIYooLeb31NLhOcX142b8U7XRMtiqdd +wfsT5SbjnrATBponKZELO7LwE+Z4djo2+O6JZjYB5/t/Z6r7qfOaTTlJEl8VJKo4 +F+qAsqKo0Q9EpkRUeNdcOjDzkuEikw9IlhS0VEt3AoGAWLHoRQH4AxZmOGmX1UNY +OTT/MtCaVj3fdS58VIZjNDpjiibESI601txu8fnlYH9BrPPv7l0LpnBR+MC3VON+ +ulLq6a8tc2uLKYUz1kLMTIL6zQo0tImdgZ36p+wUA1KJXCq4N+LPs3GSjbTmTB5R +7Yuplp2vKDd0XZ5tCy7yB5UCgYEA3ppoE1DTSC1bNmSLT2jCuEOv4ic+unw1+lti +lWh6hvldzD8XEf9RAB1PNtvKqNQD67SoX/mczK956OVQlYYSXIXzMcoRCwBfnyxq +sbct/Y2TGXpXmjwt8JcKZkVJcuBPTXOl6cwA7FHAdkR0/hMJUNzS608PZCtAqj4d +kANtp3MCgYEA3plv6RYYDZsUdJmIcdIvr/LxWTSqYf37LQM//k+OXo+zWMxgnUNv +AEqDlNWP+bw3yJlU1bQS2o1Z+hKzDgqDZtaVVVDgdVsUaDSW0EsJpedWXoGqJdrw +yxhB7RYi1tQsXHbR1iyT5hH0ZlV7s0XIKRU4U8MP05av099+++YKhks= +-----END RSA PRIVATE KEY----- diff --git a/tests/integration/test_reload_certificate/test.py b/tests/integration/test_reload_certificate/test.py index 1718e440629..86140c83dfd 100644 --- a/tests/integration/test_reload_certificate/test.py +++ b/tests/integration/test_reload_certificate/test.py @@ -13,6 +13,8 @@ node = cluster.add_instance( "configs/second.key", "configs/ECcert.crt", "configs/ECcert.key", + "configs/WithChain.crt", + "configs/WithChain.key", "configs/WithPassPhrase.crt", "configs/WithPassPhrase.key", "configs/cert.xml", @@ -158,3 +160,16 @@ def test_cert_with_pass_phrase(): check_certificate_switch( "first", "WithPassPhrase", pass_phrase_second=pass_phrase_for_cert ) + + +def test_chain_reload(): + """Check cert chain reload""" + check_certificate_switch("first", "WithChain") + assert ( + node.exec_in_container([ + "bash", + "-c", + "openssl s_client -showcerts -servername localhost -connect localhost:8443 /dev/null | grep 'BEGIN CERTIFICATE' | wc -l", + ]) + == "2\n" + ) From 570dc32b077994e31257fc3b99cfd0f1bf4287bc Mon Sep 17 00:00:00 2001 From: Pervakov Grigorii Date: Thu, 21 Mar 2024 08:58:20 +0000 Subject: [PATCH 076/154] fix style --- src/Server/CertificateReloader.cpp | 47 ++++++++++--------- .../test_reload_certificate/test.py | 12 +++-- 2 files changed, 33 insertions(+), 26 deletions(-) diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index c93ef380665..c01e1fc9808 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -22,7 +22,6 @@ int callSetCertificate(SSL * ssl, [[maybe_unused]] void * arg) } - /// This is callback for OpenSSL. It will be called on every connection to obtain a certificate and private key. int CertificateReloader::setCertificate(SSL * ssl) { @@ -30,31 +29,37 @@ int CertificateReloader::setCertificate(SSL * ssl) if (!current) return -1; - if (current->certs_chain.size() < 1) + if (current->certs_chain.empty()) return -1; - int ret; - ret = SSL_clear_chain_certs(ssl); - if (!ret) - return ret; - ret = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate())); - if (!ret) - return ret; - for (auto cert = current->certs_chain.begin() + 1; cert != current->certs_chain.end(); cert++) { - ret = SSL_add1_chain_cert(ssl, const_cast(cert->certificate())); - if (!ret) - return ret; - } - ret = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key))); - - int err = SSL_check_private_key(ssl); - if (err != 1) + if (auto err = SSL_clear_chain_certs(ssl)) { - std::string msg = Poco::Net::Utility::getLastError(); - LOG_ERROR(log, "Unusable key-pair {}", msg); + LOG_ERROR(log, "Clear certificates {}", Poco::Net::Utility::getLastError()); + return -1; + } + if (auto err = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate()))) + { + LOG_ERROR(log, "Use certificate {}", Poco::Net::Utility::getLastError()); + return -1; + } + for (auto cert = current->certs_chain.begin() + 1; cert != current->certs_chain.end(); cert++) + { + if (auto err = SSL_add1_chain_cert(ssl, const_cast(cert->certificate()))) + { + LOG_ERROR(log, "Add certificate to chain {}", Poco::Net::Utility::getLastError()); + return -1; + } + } + if (auto err = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key)))) + { + LOG_ERROR(log, "Use private key {}", Poco::Net::Utility::getLastError()); + return -1; + } + if (auto err = SSL_check_private_key(ssl)) + { + LOG_ERROR(log, "Unusable key-pair {}", Poco::Net::Utility::getLastError()); return -1; } - return 1; } diff --git a/tests/integration/test_reload_certificate/test.py b/tests/integration/test_reload_certificate/test.py index 86140c83dfd..f0efc4e0bbd 100644 --- a/tests/integration/test_reload_certificate/test.py +++ b/tests/integration/test_reload_certificate/test.py @@ -166,10 +166,12 @@ def test_chain_reload(): """Check cert chain reload""" check_certificate_switch("first", "WithChain") assert ( - node.exec_in_container([ - "bash", - "-c", - "openssl s_client -showcerts -servername localhost -connect localhost:8443 /dev/null | grep 'BEGIN CERTIFICATE' | wc -l", - ]) + node.exec_in_container( + [ + "bash", + "-c", + "openssl s_client -showcerts -servername localhost -connect localhost:8443 /dev/null | grep 'BEGIN CERTIFICATE' | wc -l", + ] + ) == "2\n" ) From d45a6fe9b9187e27405bfe90be840cf03b279aae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 15:46:55 +0100 Subject: [PATCH 077/154] Fix tidy --- src/Functions/greatCircleDistance.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 1b5fe1ac35b..01184f74b13 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -227,7 +227,7 @@ template class FunctionGeoDistance : public IFunction { public: - FunctionGeoDistance(ContextPtr context) + explicit FunctionGeoDistance(ContextPtr context) { always_float32 = !context->getSettingsRef().geo_distance_returns_float64_on_float64_arguments; } From 3752b13753ed631c539e865e73853d18152f844f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Mar 2024 15:36:01 +0000 Subject: [PATCH 078/154] See what would happen if constant fold _CAST for scalar subqueries properly. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 6719504eac3..499bc7048eb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2168,10 +2168,13 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden !nearest_query_scope) { auto constant_value = std::make_shared(std::move(scalar_value), scalar_type); - auto constant_node = std::make_shared(std::move(constant_value), node); + auto constant_node = std::make_shared(constant_value, node); if (constant_node->getValue().isNull()) + { node = buildCastFunction(constant_node, constant_node->getResultType(), context); + node = std::make_shared(std::move(constant_value), node); + } else node = std::move(constant_node); From 8bd49aa30382319c95503ddfe17baeed051c8d27 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 25 Mar 2024 17:06:42 +0100 Subject: [PATCH 079/154] Try to fix perf tests --- tests/performance/scripts/eqmed.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/scripts/eqmed.sql b/tests/performance/scripts/eqmed.sql index 94e6733a3d7..372534f6f48 100644 --- a/tests/performance/scripts/eqmed.sql +++ b/tests/performance/scripts/eqmed.sql @@ -67,4 +67,5 @@ from select throwIf(uniq((test, query)) != 1) from table ) check_single_query -- this subselect checks that there is only one query in the input table; -- written this way so that it is not optimized away (#10523) +SETTINGS allow_experimental_analyzer = 0 ; From 81bf65c555a31e370b29d507c76cf603f68f4139 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 25 Mar 2024 16:14:06 +0000 Subject: [PATCH 080/154] fix tests flakiness --- .../configs/with_delay_config.xml | 2 +- tests/integration/test_undrop_query/test.py | 69 +++++-------------- 2 files changed, 18 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_undrop_query/configs/with_delay_config.xml b/tests/integration/test_undrop_query/configs/with_delay_config.xml index c672b187b01..94e5da2193b 100644 --- a/tests/integration/test_undrop_query/configs/with_delay_config.xml +++ b/tests/integration/test_undrop_query/configs/with_delay_config.xml @@ -1,3 +1,3 @@ - 5 + 80 diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index 2aa5bdacf9d..8aec1d60473 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -20,65 +20,30 @@ def started_cluster(): finally: cluster.shutdown() - def test_undrop_drop_and_undrop_loop(started_cluster): - count = 0 - while count < 10: - random_sec = random.randint(0, 10) + uuid_list = [] + + for i in range(10): table_uuid = uuid.uuid1().__str__() - logging.info( - "random_sec: " + random_sec.__str__() + ", table_uuid: " + table_uuid - ) - + uuid_list.append(table_uuid) + logging.info(f"table_uuid: {table_uuid}") + node.query( - "CREATE TABLE test_undrop_loop" - + count.__str__() - + " UUID '" - + table_uuid - + "' (id Int32) ENGINE = MergeTree() ORDER BY id;" + f"CREATE TABLE test_undrop_{i} + UUID '{table_uuid}' (id Int32) + ENGINE = MergeTree() ORDER BY id;" ) - node.query("DROP TABLE test_undrop_loop" + count.__str__() + ";") + node.query(f"DROP TABLE test_undrop_{i};") - time.sleep(random_sec) - - if random_sec >= 5: + for i in range(10): + if i >= 8: # -> setting for table to live after drop = 80 seconds error = node.query_and_get_error( - "UNDROP TABLE test_undrop_loop" - + count.__str__() - + " UUID '" - + table_uuid - + "';" + f"UNDROP TABLE test_undrop_loop_{i} UUID '{table_uuid}';" ) assert "UNKNOWN_TABLE" in error - elif random_sec <= 3: - # (*) - node.query( - "UNDROP TABLE test_undrop_loop" - + count.__str__() - + " UUID '" - + table_uuid - + "';" - ) - count = count + 1 else: - try: - node.query( - "UNDROP TABLE test_undrop_loop" - + count.__str__() - + " UUID '" - + table_uuid - + "';" - ) - count = count + 1 - except: - error = node.query_and_get_error( - "UNDROP TABLE test_undrop_loop" - + count.__str__() - + " UUID '" - + table_uuid - + "';" - ) - assert "UNKNOWN_TABLE" in error - # ignore random_sec = 4 result to account for communication delay with the database. - # if we don't do that, then the second case (*) may find the table already dropped and receive an unexpected exception from the database (Bug #55167) + node.query( + f"UNDROP TABLE test_undrop_loop_{i} UUID '{table_uuid}';" + ) + time.sleep(10) From b0cbbe6ec0e595a57d9628596ce6c97e662d52c0 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 25 Mar 2024 16:24:46 +0000 Subject: [PATCH 081/154] fix --- tests/integration/test_undrop_query/test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index a5e50f01fa6..e6de76b96f7 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -1,5 +1,6 @@ import pytest import uuid +import logging import time from helpers.cluster import ClickHouseCluster @@ -37,11 +38,11 @@ def test_undrop_drop_and_undrop_loop(started_cluster): for i in range(10): if i >= 8: # -> setting for table to live after drop = 80 seconds error = node.query_and_get_error( - f"UNDROP TABLE test_undrop_loop_{i} UUID '{table_uuid}';" + f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';" ) assert "UNKNOWN_TABLE" in error else: node.query( - f"UNDROP TABLE test_undrop_loop_{i} UUID '{table_uuid}';" + f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';" ) time.sleep(10) From 67ba86e7d3c76699ad5accf6e34f159af5f6d6f3 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 25 Mar 2024 17:30:01 +0100 Subject: [PATCH 082/154] Update test.py --- tests/integration/test_undrop_query/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index e6de76b96f7..e01ab942e64 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -28,9 +28,7 @@ def test_undrop_drop_and_undrop_loop(started_cluster): logging.info(f"table_uuid: {table_uuid}") node.query( - f"CREATE TABLE test_undrop_{i} - UUID '{table_uuid}' (id Int32) - ENGINE = MergeTree() ORDER BY id;" + f"CREATE TABLE test_undrop_{i} UUID '{table_uuid}' (id Int32) ENGINE = MergeTree() ORDER BY id;" ) node.query(f"DROP TABLE test_undrop_{i};") From 9d17f3106c60aa387be0318bc28c6a27b13c7bac Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 25 Mar 2024 17:53:01 +0100 Subject: [PATCH 083/154] Revert "Add --now option to enable and start the service" --- packages/clickhouse-server.postinstall | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/clickhouse-server.postinstall b/packages/clickhouse-server.postinstall index 41d4405a790..d3b49db758f 100644 --- a/packages/clickhouse-server.postinstall +++ b/packages/clickhouse-server.postinstall @@ -36,7 +36,7 @@ if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then fi /bin/systemctl daemon-reload - /bin/systemctl enable --now clickhouse-server + /bin/systemctl enable clickhouse-server else # If you downgrading to version older than 1.1.54336 run: systemctl disable clickhouse-server if [ -x "/etc/init.d/clickhouse-server" ]; then From a2ad832dfe4aba122cb208e132fa5acc6026e020 Mon Sep 17 00:00:00 2001 From: Pervakov Grigorii Date: Mon, 25 Mar 2024 17:13:30 +0000 Subject: [PATCH 084/154] fix error handling --- src/Server/CertificateReloader.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index c01e1fc9808..311ece67bce 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -32,30 +32,30 @@ int CertificateReloader::setCertificate(SSL * ssl) if (current->certs_chain.empty()) return -1; - if (auto err = SSL_clear_chain_certs(ssl)) + if (auto err = SSL_clear_chain_certs(ssl); err != 1) { LOG_ERROR(log, "Clear certificates {}", Poco::Net::Utility::getLastError()); return -1; } - if (auto err = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate()))) + if (auto err = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate())); err != 1) { LOG_ERROR(log, "Use certificate {}", Poco::Net::Utility::getLastError()); return -1; } for (auto cert = current->certs_chain.begin() + 1; cert != current->certs_chain.end(); cert++) { - if (auto err = SSL_add1_chain_cert(ssl, const_cast(cert->certificate()))) + if (auto err = SSL_add1_chain_cert(ssl, const_cast(cert->certificate())); err != 1) { LOG_ERROR(log, "Add certificate to chain {}", Poco::Net::Utility::getLastError()); return -1; } } - if (auto err = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key)))) + if (auto err = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key))); err != 1) { LOG_ERROR(log, "Use private key {}", Poco::Net::Utility::getLastError()); return -1; } - if (auto err = SSL_check_private_key(ssl)) + if (auto err = SSL_check_private_key(ssl); err != 1) { LOG_ERROR(log, "Unusable key-pair {}", Poco::Net::Utility::getLastError()); return -1; From 30df4589edc67f818c11e2a87340ea95f544f620 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 25 Mar 2024 18:18:17 +0100 Subject: [PATCH 085/154] black check --- tests/integration/test_undrop_query/test.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_undrop_query/test.py b/tests/integration/test_undrop_query/test.py index e01ab942e64..b6123312320 100644 --- a/tests/integration/test_undrop_query/test.py +++ b/tests/integration/test_undrop_query/test.py @@ -19,6 +19,7 @@ def started_cluster(): finally: cluster.shutdown() + def test_undrop_drop_and_undrop_loop(started_cluster): uuid_list = [] @@ -26,7 +27,7 @@ def test_undrop_drop_and_undrop_loop(started_cluster): table_uuid = uuid.uuid1().__str__() uuid_list.append(table_uuid) logging.info(f"table_uuid: {table_uuid}") - + node.query( f"CREATE TABLE test_undrop_{i} UUID '{table_uuid}' (id Int32) ENGINE = MergeTree() ORDER BY id;" ) @@ -34,13 +35,11 @@ def test_undrop_drop_and_undrop_loop(started_cluster): node.query(f"DROP TABLE test_undrop_{i};") for i in range(10): - if i >= 8: # -> setting for table to live after drop = 80 seconds + if i >= 8: # -> setting for table to live after drop = 80 seconds error = node.query_and_get_error( f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';" ) assert "UNKNOWN_TABLE" in error else: - node.query( - f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';" - ) + node.query(f"UNDROP TABLE test_undrop_loop_{i} UUID '{uuid_list[i]}';") time.sleep(10) From 727aeb7e37d50c69f22941b77c517703a384060f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Mar 2024 16:39:50 +0000 Subject: [PATCH 086/154] CI: set timeout for install check #job_Install_packages_amd64 #job_style_check --- tests/ci/ci_config.py | 15 ++++++++------- tests/ci/ci_utils.py | 12 ++++++++++++ tests/ci/install_check.py | 6 +++++- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 573976f1f84..b4fe6b516fc 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -220,7 +220,7 @@ class JobConfig: digest: DigestConfig = field(default_factory=DigestConfig) # will be triggered for the job if omited in CI workflow yml run_command: str = "" - # job timeout + # job timeout, seconds timeout: Optional[int] = None # sets number of batches for multi-batch job num_batches: int = 1 @@ -517,10 +517,11 @@ clickbench_test_params = { ), "run_command": 'clickbench.py "$CHECK_NAME"', } -install_test_params = { - "digest": install_check_digest, - "run_command": 'install_check.py "$CHECK_NAME"', -} +install_test_params = JobConfig( + digest=install_check_digest, + run_command='install_check.py "$CHECK_NAME"', + timeout=900, +) @dataclass @@ -1105,10 +1106,10 @@ CI_CONFIG = CIConfig( }, test_configs={ JobNames.INSTALL_TEST_AMD: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**install_test_params) # type: ignore + Build.PACKAGE_RELEASE, job_config=install_test_params ), JobNames.INSTALL_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**install_test_params) # type: ignore + Build.PACKAGE_AARCH64, job_config=install_test_params ), JobNames.STATEFUL_TEST_ASAN: TestConfig( Build.PACKAGE_ASAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 2967ec2f309..97d42f9845b 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,5 +1,6 @@ from contextlib import contextmanager import os +import signal from typing import Any, List, Union, Iterator from pathlib import Path @@ -48,3 +49,14 @@ class GHActions: for line in lines: print(line) print("::endgroup::") + + +def set_job_timeout(): + def timeout_handler(_signum, _frame): + print("Timeout expired") + raise TimeoutError("Job's KILL_TIMEOUT expired") + + kill_timeout = int(os.getenv("KILL_TIMEOUT", "0")) + assert kill_timeout > 0, "kill timeout must be provided in KILL_TIMEOUT env" + signal.signal(signal.SIGALRM, timeout_handler) + signal.alarm(kill_timeout) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 4fc112c6d9f..71e2198f533 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -14,10 +14,11 @@ from build_download_helper import download_builds_filter from compress_files import compress_fast from docker_images_helper import DockerImage, pull_image, get_docker_image -from env_helper import REPORT_PATH, TEMP_PATH as TEMP +from env_helper import CI, REPORT_PATH, TEMP_PATH as TEMP from report import JobReport, TestResults, TestResult, FAILURE, FAIL, OK, SUCCESS from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_utils import set_job_timeout RPM_IMAGE = "clickhouse/install-rpm-test" @@ -255,6 +256,9 @@ def main(): args = parse_args() + if CI: + set_job_timeout() + TEMP_PATH.mkdir(parents=True, exist_ok=True) LOGS_PATH.mkdir(parents=True, exist_ok=True) From 11bc6c3e6e7ba947abe52320edaf0c367751fc2f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 25 Mar 2024 18:43:29 +0100 Subject: [PATCH 087/154] Test added --- .../03031_low_cardinality_logical_error.reference | 6 ++++++ .../03031_low_cardinality_logical_error.sql | 14 ++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03031_low_cardinality_logical_error.reference create mode 100644 tests/queries/0_stateless/03031_low_cardinality_logical_error.sql diff --git a/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference b/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference new file mode 100644 index 00000000000..931811041f2 --- /dev/null +++ b/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference @@ -0,0 +1,6 @@ +[] 0 ['2'] +['0'] 2 ['0'] +['0'] 2 ['0'] +['1'] 1 [] + +[] 3 [] diff --git a/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql b/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql new file mode 100644 index 00000000000..02ef0585b00 --- /dev/null +++ b/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql @@ -0,0 +1,14 @@ +SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; From f9f830228a71225841b04d0eec39d51d399be116 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Mar 2024 16:08:53 +0000 Subject: [PATCH 088/154] CI: MQ support --- .github/workflows/pull_request.yml | 11 +++-------- tests/ci/ci.py | 27 ++++++--------------------- tests/ci/pr_info.py | 1 + 3 files changed, 10 insertions(+), 29 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 2dddde9aa14..ff0adee1443 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -6,6 +6,7 @@ env: PYTHONUNBUFFERED: 1 on: # yamllint disable-line rule:truthy + merge_group: pull_request: types: - synchronize @@ -29,6 +30,7 @@ jobs: fetch-depth: 0 # to get version filter: tree:0 - name: Labels check + if: ${{ github.event_name != 'merge_group' }} run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py @@ -56,16 +58,9 @@ jobs: echo 'EOF' } >> "$GITHUB_OUTPUT" - name: Re-create GH statuses for skipped jobs if any + if: ${{ github.event_name != 'merge_group' }} run: | python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --update-gh-statuses - - name: Style check early - # hack to run style check before the docker build job if possible (style-check image not changed) - if: contains(fromJson(steps.runconfig.outputs.CI_DATA).jobs_data.jobs_to_do, 'Style check early') - run: | - DOCKER_TAG=$(echo '${{ toJson(fromJson(steps.runconfig.outputs.CI_DATA).docker_data.images) }}' | tr -d '\n') - export DOCKER_TAG=$DOCKER_TAG - python3 ./tests/ci/style_check.py --no-push - python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --post --job-name 'Style check' BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 514189a8b8a..a3cea281a56 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1033,22 +1033,6 @@ def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> raise AssertionError(f"Unexpected type for 'res': {type(result)}") -def _check_and_update_for_early_style_check(jobs_data: dict, docker_data: dict) -> None: - """ - This is temporary hack to start style check before docker build if possible - FIXME: need better solution to do style check as soon as possible and as fast as possible w/o dependency on docker job - """ - jobs_to_do = jobs_data.get("jobs_to_do", []) - docker_to_build = docker_data.get("missing_multi", []) - if ( - JobNames.STYLE_CHECK in jobs_to_do - and docker_to_build - and "clickhouse/style-test" not in docker_to_build - ): - index = jobs_to_do.index(JobNames.STYLE_CHECK) - jobs_to_do[index] = "Style check early" - - def _update_config_for_docs_only(jobs_data: dict) -> None: DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] print(f"NOTE: Will keep only docs related jobs: [{DOCS_CHECK_JOBS}]") @@ -1306,6 +1290,12 @@ def _configure_jobs( if params["num_batches"] > 1: params["batches"] = list(requested_batches) + if pr_info.is_merge_queue(): + # FIXME: Quick support for MQ workflow which is only StyleCheck for now + jobs_to_do = [JobNames.STYLE_CHECK] + jobs_to_skip = [] + print(f"NOTE: This is Merge Queue CI: set jobs to do: [{jobs_to_do}]") + return { "digests": digests, "jobs_to_do": jobs_to_do, @@ -1752,11 +1742,6 @@ def main() -> int: else {} ) - # # FIXME: Early style check manipulates with job names might be not robust with await feature - # if pr_info.number != 0: - # # FIXME: it runs style check before docker build if possible (style-check images is not changed) - # # find a way to do style check always before docker build and others - # _check_and_update_for_early_style_check(jobs_data, docker_data) if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): _update_config_for_docs_only(jobs_data) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 84f2db4002d..ddf59c49e1f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -215,6 +215,7 @@ class PRInfo: .replace("{base}", base_sha) .replace("{head}", self.sha) ) + self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" elif "commits" in github_event: self.event_type = EventType.PUSH From db724b129361e7c3799785384d34af28a2a408c5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 25 Mar 2024 19:36:48 +0100 Subject: [PATCH 089/154] Update create.sql --- docker/test/stateful/create.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateful/create.sql b/docker/test/stateful/create.sql index f3e869402d9..14e592ab5e3 100644 --- a/docker/test/stateful/create.sql +++ b/docker/test/stateful/create.sql @@ -138,7 +138,7 @@ ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) -SETTINGS disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/', max_size = '4G', +SETTINGS disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/stateful/', max_size = '4G', disk = disk(type = web, endpoint = 'https://clickhouse-datasets-web.s3.us-east-1.amazonaws.com/')); ATTACH TABLE datasets.visits_v1 UUID '5131f834-711f-4168-98a5-968b691a104b' @@ -329,5 +329,5 @@ ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) -SETTINGS disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/', max_size = '4G', +SETTINGS disk = disk(type = cache, path = '/var/lib/clickhouse/filesystem_caches/stateful/', max_size = '4G', disk = disk(type = web, endpoint = 'https://clickhouse-datasets-web.s3.us-east-1.amazonaws.com/')); From 03f8334ea1bd46839adb7ad45dabfb7fac30c042 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 22:36:58 +0300 Subject: [PATCH 090/154] Remove SonarCloud (#61880) --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index e00ce42a60b..2b97bd25d70 100644 --- a/README.md +++ b/README.md @@ -28,7 +28,6 @@ curl https://clickhouse.com/ | sh * [Slack](https://clickhouse.com/slack) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events. * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlighting, powered by github.dev. -* [Static Analysis (SonarCloud)](https://sonarcloud.io/project/issues?resolved=false&id=ClickHouse_ClickHouse) proposes C++ quality improvements. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Monthly Release & Community Call From b6057bd83c89ba12ce83d530cb2d05836423703b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 25 Mar 2024 20:34:24 +0000 Subject: [PATCH 091/154] Test: tuple elimination with analyzer --- .../03031_tuple_elimination_analyzer.reference | 1 + .../0_stateless/03031_tuple_elimination_analyzer.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/03031_tuple_elimination_analyzer.reference create mode 100644 tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql diff --git a/tests/queries/0_stateless/03031_tuple_elimination_analyzer.reference b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.reference new file mode 100644 index 00000000000..84ab67a85e0 --- /dev/null +++ b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.reference @@ -0,0 +1 @@ +(0) diff --git a/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql new file mode 100644 index 00000000000..97a19cda7d3 --- /dev/null +++ b/tests/queries/0_stateless/03031_tuple_elimination_analyzer.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test; + +SET allow_suspicious_low_cardinality_types = true, allow_experimental_analyzer = true; + +CREATE TABLE test (`id` LowCardinality(UInt32)) ENGINE = MergeTree ORDER BY id AS SELECT 0; + +SELECT tuple(tuple(id) = tuple(1048576)) FROM test; + +DROP TABLE test; From faf82d53b3fc618e96e2e94b7c3b585951e227ea Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Mon, 25 Mar 2024 20:56:00 +0000 Subject: [PATCH 092/154] no smaller unit in date_trunc --- src/Functions/date_trunc.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 8493df17a2f..de5e71e09a8 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -56,6 +56,10 @@ public: if (!IntervalKind::tryParseString(datepart_param, datepart_kind)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName()); + if (datepart_kind == IntervalKind::Kind::Nanosecond || datepart_kind == IntervalKind::Kind::Microsecond + || datepart_kind == IntervalKind::Kind::Millisecond) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't support {}", getName(), datepart_param); + result_type_is_date = (datepart_kind == IntervalKind::Kind::Year) || (datepart_kind == IntervalKind::Kind::Quarter) || (datepart_kind == IntervalKind::Kind::Month) || (datepart_kind == IntervalKind::Kind::Week); From cad66f3e688a759cad50ee9b69d1061c127c6ed6 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 25 Mar 2024 20:03:02 -0400 Subject: [PATCH 093/154] Fix ALTER QUERY MODIFY SQL SECURITY (#61480) --- src/Databases/DatabasesCommon.cpp | 11 +++++++++++ src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Parsers/ASTCreateQuery.cpp | 2 +- src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserAlterQuery.cpp | 14 ++++++++++---- src/Storages/StorageInMemoryMetadata.cpp | 2 ++ ...create_view_with_sql_security_option.reference | 2 ++ .../02884_create_view_with_sql_security_option.sh | 15 +++++++-------- 8 files changed, 35 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index d40e5f98aaa..4dffb16e486 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -71,6 +71,17 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo query->replace(ast_create_query.refresh_strategy, metadata.refresh); } + if (metadata.sql_security_type) + { + auto new_sql_security = std::make_shared(); + new_sql_security->type = metadata.sql_security_type; + + if (metadata.definer) + new_sql_security->definer = std::make_shared(*metadata.definer); + + ast_create_query.sql_security = std::move(new_sql_security); + } + /// MaterializedView, Dictionary are types of CREATE query without storage. if (ast_create_query.storage) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8b777a758f6..d90d2446fca 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1881,7 +1881,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. - if (!sql_security.type.has_value()) + if (!sql_security.type) { SQLSecurityType default_security; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index de5eb40837f..0403dc33164 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -14,7 +14,7 @@ namespace DB void ASTSQLSecurity::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - if (!type.has_value()) + if (!type) return; if (definer || is_definer_current_user) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 545422aaf59..1a9fd7824d1 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -299,6 +299,7 @@ namespace DB MR_MACROS(MOD, "MOD") \ MR_MACROS(MODIFY_COLUMN, "MODIFY COLUMN") \ MR_MACROS(MODIFY_COMMENT, "MODIFY COMMENT") \ + MR_MACROS(MODIFY_DEFINER, "MODIFY DEFINER") \ MR_MACROS(MODIFY_ORDER_BY, "MODIFY ORDER BY") \ MR_MACROS(MODIFY_QUERY, "MODIFY QUERY") \ MR_MACROS(MODIFY_REFRESH, "MODIFY REFRESH") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index c0c43381585..1baff45113b 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -41,6 +41,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_reset_setting(Keyword::RESET_SETTING); ParserKeyword s_modify_query(Keyword::MODIFY_QUERY); ParserKeyword s_modify_sql_security(Keyword::MODIFY_SQL_SECURITY); + ParserKeyword s_modify_definer(Keyword::MODIFY_DEFINER); ParserKeyword s_modify_refresh(Keyword::MODIFY_REFRESH); ParserKeyword s_add_index(Keyword::ADD_INDEX); @@ -862,11 +863,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::MODIFY_QUERY; } - else if (s_modify_sql_security.ignore(pos, expected)) + else if (s_modify_sql_security.checkWithoutMoving(pos, expected)) { - /// This is a hack so we can reuse parser from create and don't have to write `MODIFY SQL SECURITY SQL SECURITY INVOKER` - --pos; - --pos; + s_modify.ignore(pos, expected); + if (!sql_security_p.parse(pos, command_sql_security, expected)) + return false; + command->type = ASTAlterCommand::MODIFY_SQL_SECURITY; + } + else if (s_modify_definer.checkWithoutMoving(pos, expected)) + { + s_modify.ignore(pos, expected); if (!sql_security_p.parse(pos, command_sql_security, expected)) return false; command->type = ASTAlterCommand::MODIFY_SQL_SECURITY; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 8e5195d497f..1ac739f03fd 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -94,6 +94,8 @@ void StorageInMemoryMetadata::setSQLSecurity(const ASTSQLSecurity & sql_security { if (sql_security.definer) definer = sql_security.definer->toString(); + else + definer = std::nullopt; sql_security_type = sql_security.type; } diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 79728fadc04..6d9d1f07ec2 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -12,8 +12,10 @@ OK 2 2 OK +1 ===== MaterializedView ===== OK +1 0 0 OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index a4ab3ed0024..bead7db8450 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -1,18 +1,17 @@ #!/usr/bin/env bash +# Tags: no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -user1="user02884_1_$RANDOM$RANDOM" -user2="user02884_2_$RANDOM$RANDOM" -user3="user02884_3_$RANDOM$RANDOM" -db="db02884_$RANDOM$RANDOM" +user1="user02884_1_${CLICKHOUSE_DATABASE}_$RANDOM" +user2="user02884_2_${CLICKHOUSE_DATABASE}_$RANDOM" +user3="user02884_3_${CLICKHOUSE_DATABASE}_$RANDOM" +db=${CLICKHOUSE_DATABASE} ${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE $db.test_view_10" | grep -c "SQL SECURITY INVOKER" echo "===== MaterializedView =====" @@ -136,6 +136,7 @@ ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_5 TO $user2" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $db.test_mv_5 MODIFY SQL SECURITY NONE" ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_5" +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE $db.test_mv_5" | grep -c "SQL SECURITY NONE" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_1 TO $user2" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_3 TO $user2" @@ -221,6 +222,4 @@ EOF ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_row_2" - -${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS $db;" ${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS $user1, $user2, $user3"; From 94f137967732e8177570df5ac570f5566b17bf66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 04:02:32 +0100 Subject: [PATCH 094/154] Move KQL trash where it is supposed to be --- src/Parsers/CommonParsers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 1a9fd7824d1..49964b5c728 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -556,7 +556,7 @@ namespace DB MR_MACROS(SSH_KEY, "SSH_KEY") \ MR_MACROS(SSL_CERTIFICATE, "SSL_CERTIFICATE") \ MR_MACROS(STRICTLY_ASCENDING, "STRICTLY_ASCENDING") \ - MR_MACROS(WITH_ITEMINDEX, "with_itemindex") \ + MR_MACROS(WITH_ITEMINDEX, "WITH_ITEMINDEX") \ enum class Keyword : size_t { From 12a619f38881904c802e274feb99e3ac03af5bc4 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Mon, 25 Mar 2024 21:56:13 -0600 Subject: [PATCH 095/154] Modify cmake command for building clickhouse on MacOS This seems subtle, but for some reason it makes a big difference. If you set the environment variables before running `cmake`, they get ignored. After many, many attempts I finally figured out this solution - which worked great on my Mac. --- docs/en/development/build-osx.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 21b9446aa66..a6c49f1f476 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -55,9 +55,7 @@ To build using Homebrew's vanilla Clang compiler (the only **recommended** way): cd ClickHouse mkdir build export PATH=$(brew --prefix llvm)/bin:$PATH -export CC=$(brew --prefix llvm)/bin/clang -export CXX=$(brew --prefix llvm)/bin/clang++ -cmake -G Ninja -DCMAKE_BUILD_TYPE=RelWithDebInfo -S . -B build +cmake -G Ninja -DCMAKE_BUILD_TYPE=RelWithDebInfo -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -S . -B build cmake --build build # The resulting binary will be created at: build/programs/clickhouse ``` From 2a820a14c5eb2f9d999ea43883cbb4e83dae8a22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 05:49:21 +0100 Subject: [PATCH 096/154] Changelog for 24.3 --- CHANGELOG.md | 174 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 174 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9df678d4b9a..9117289028d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,184 @@ ### Table of Contents +**[ClickHouse release v24.3 LTS, 2024-03-26](#243)**
**[ClickHouse release v24.2, 2024-02-29](#242)**
**[ClickHouse release v24.1, 2024-01-30](#241)**
**[Changelog for 2023](https://clickhouse.com/docs/en/whats-new/changelog/2023/)**
# 2024 Changelog +### ClickHouse release 24.3 LTS, 2024-03-26 + +#### Upgrade Notes +* The setting `allow_experimental_analyzer` is enabled by default and it switches the query analysis to a new implementation, which has better compatibility and feature completeness. The feature "analyzer" is considered beta instead of experimental. You can turn the old behavior by setting the `compatibility` to `24.2` or disabling the `allow_experimental_analyzer` setting. Watch the [video on YouTube](https://www.youtube.com/watch?v=zhrOYQpgvkk). +* ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. This is controlled by the settings, `output_format_parquet_string_as_string`, `output_format_orc_string_as_string`, `output_format_arrow_string_as_string`. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases. Parquet/ORC/Arrow supports many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools lack support for the faster `lz4` compression method, that's why we set `zstd` by default. This is controlled by the settings `output_format_parquet_compression_method`, `output_format_orc_compression_method`, and `output_format_arrow_compression_method`. We changed the default to `zstd` for Parquet and ORC, but not Arrow (it is emphasized for low-level usages). [#61817](https://github.com/ClickHouse/ClickHouse/pull/61817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the new ClickHouse version, the functions `geoDistance`, `greatCircleDistance`, and `greatCircleAngle` will use 64-bit double precision floating point data type for internal calculations and return type if all the arguments are Float64. This closes [#58476](https://github.com/ClickHouse/ClickHouse/issues/58476). In previous versions, the function always used Float32. You can switch to the old behavior by setting `geo_distance_returns_float64_on_float64_arguments` to `false` or setting `compatibility` to `24.2` or earlier. [#61848](https://github.com/ClickHouse/ClickHouse/pull/61848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). Co-authored with [Geet Patel](https://github.com/geetptl). +* The obsolete in-memory data parts have been deprecated since version 23.5 and have not been supported since version 23.10. Now the remaining code is removed. Continuation of [#55186](https://github.com/ClickHouse/ClickHouse/issues/55186) and [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). It is unlikely that you have used in-memory data parts because they were available only before version 23.5 and only when you enabled them manually by specifying the corresponding SETTINGS for a MergeTree table. To check if you have in-memory data parts, run the following query: `SELECT part_type, count() FROM system.parts GROUP BY part_type ORDER BY part_type`. To disable the usage of in-memory data parts, do `ALTER TABLE ... MODIFY SETTING min_bytes_for_compact_part = DEFAULT, min_rows_for_compact_part = DEFAULT`. Before upgrading from old ClickHouse releases, first check that you don't have in-memory data parts. If there are in-memory data parts, disable them first, then wait while there are no in-memory data parts and continue the upgrade. [#61127](https://github.com/ClickHouse/ClickHouse/pull/61127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changed the column name from `duration_ms` to `duration_microseconds` in the `system.zookeeper` table to reflect the reality that the duration is in the microsecond resolution. [#60774](https://github.com/ClickHouse/ClickHouse/pull/60774) ([Duc Canh Le](https://github.com/canhld94)). +* Reject incoming INSERT queries in case when query-level settings `async_insert` and `deduplicate_blocks_in_dependent_materialized_views` are enabled together. This behaviour is controlled by a setting `throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert` and enabled by default. This is a continuation of https://github.com/ClickHouse/ClickHouse/pull/59699 needed to unblock https://github.com/ClickHouse/ClickHouse/pull/59915. [#60888](https://github.com/ClickHouse/ClickHouse/pull/60888) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Utility `clickhouse-copier` is moved to a separate repository on GitHub: https://github.com/ClickHouse/copier. It is no longer included in the bundle but is still available as a separate download. This closes: [#60734](https://github.com/ClickHouse/ClickHouse/issues/60734) This closes: [#60540](https://github.com/ClickHouse/ClickHouse/issues/60540) This closes: [#60250](https://github.com/ClickHouse/ClickHouse/issues/60250) This closes: [#52917](https://github.com/ClickHouse/ClickHouse/issues/52917) This closes: [#51140](https://github.com/ClickHouse/ClickHouse/issues/51140) This closes: [#47517](https://github.com/ClickHouse/ClickHouse/issues/47517) This closes: [#47189](https://github.com/ClickHouse/ClickHouse/issues/47189) This closes: [#46598](https://github.com/ClickHouse/ClickHouse/issues/46598) This closes: [#40257](https://github.com/ClickHouse/ClickHouse/issues/40257) This closes: [#36504](https://github.com/ClickHouse/ClickHouse/issues/36504) This closes: [#35485](https://github.com/ClickHouse/ClickHouse/issues/35485) This closes: [#33702](https://github.com/ClickHouse/ClickHouse/issues/33702) This closes: [#26702](https://github.com/ClickHouse/ClickHouse/issues/26702). +* To increase compatibility with MySQL, the compatibility alias `locate` now accepts arguments `(needle, haystack[, start_pos])` by default. The previous behavior `(haystack, needle, [, start_pos])` can be restored by setting `function_locate_has_mysql_compatible_argument_order = 0`. [#61092](https://github.com/ClickHouse/ClickHouse/pull/61092) ([Robert Schulze](https://github.com/rschu1ze)). +* Forbid `SimpleAggregateFunction` in `ORDER BY` of `MergeTree` tables (like `AggregateFunction` is forbidden, but they are forbidden because they are not comparable) by default (use `allow_suspicious_primary_key` to allow them). [#61399](https://github.com/ClickHouse/ClickHouse/pull/61399) ([Azat Khuzhin](https://github.com/azat)). +* The `Ordinary` database engine is deprecated. You will receive a warning in clickhouse-client if your server is using it. This closes [#52229](https://github.com/ClickHouse/ClickHouse/issues/52229). [#56942](https://github.com/ClickHouse/ClickHouse/pull/56942) ([shabroo](https://github.com/shabroo)). + +#### New Feature +* Support reading and writing backups as `tar` (in addition to `zip`). [#59535](https://github.com/ClickHouse/ClickHouse/pull/59535) ([josh-hildred](https://github.com/josh-hildred)). +* Implemented support for S3 Express buckets. [#59965](https://github.com/ClickHouse/ClickHouse/pull/59965) ([Nikita Taranov](https://github.com/nickitat)). +* Allow to attach parts from a different disk (using copy instead of hard link). [#60112](https://github.com/ClickHouse/ClickHouse/pull/60112) ([Unalian](https://github.com/Unalian)). +* Size-capped `Memory` tables: controlled by their settings, `min_bytes_to_keep, max_bytes_to_keep, min_rows_to_keep` and `max_rows_to_keep`. [#60612](https://github.com/ClickHouse/ClickHouse/pull/60612) ([Jake Bamrah](https://github.com/JakeBamrah)). +* Separate limits on number of waiting and executing queries. Added new server setting `max_waiting_queries` that limits the number of queries waiting due to `async_load_databases`. Existing limits on number of executing queries no longer count waiting queries. [#61053](https://github.com/ClickHouse/ClickHouse/pull/61053) ([Sergei Trifonov](https://github.com/serxa)). +* Added a table `system.keywords` which contains all the keywords from parser. Mostly needed and will be used for better fuzzing and syntax highlighting. [#51808](https://github.com/ClickHouse/ClickHouse/pull/51808) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add support for `ATTACH PARTITION ALL`. [#61107](https://github.com/ClickHouse/ClickHouse/pull/61107) ([Kirill Nikiforov](https://github.com/allmazz)). +* Add a new function, `getClientHTTPHeader`. This closes [#54665](https://github.com/ClickHouse/ClickHouse/issues/54665). Co-authored with @lingtaolf. [#61820](https://github.com/ClickHouse/ClickHouse/pull/61820) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `generate_series` as a table function (compatibility alias for PostgreSQL to the existing `numbers` function). This function generates table with an arithmetic progression with natural numbers. [#59390](https://github.com/ClickHouse/ClickHouse/pull/59390) ([divanik](https://github.com/divanik)). +* A mode for `topK`/`topkWeighed` support mode, which return count of values and its error. [#54508](https://github.com/ClickHouse/ClickHouse/pull/54508) ([UnamedRus](https://github.com/UnamedRus)). +* Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60281](https://github.com/ClickHouse/ClickHouse/pull/60281) ([Shaun Struwig](https://github.com/Blargian)). +* Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make `/` redirect to the Play UI. [#60390](https://github.com/ClickHouse/ClickHouse/pull/60390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement +* Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). +* 30x faster printing for 256-bit integers. [#61100](https://github.com/ClickHouse/ClickHouse/pull/61100) ([Raúl Marín](https://github.com/Algunenano)). +* If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting `primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns` with the value `0.9` by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. [#60255](https://github.com/ClickHouse/ClickHouse/pull/60255) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. [#60420](https://github.com/ClickHouse/ClickHouse/pull/60420) ([豪肥肥](https://github.com/HowePa)). I appreciate if you will continue to write it correctly, e.g., `JSON` 😇, not `Json` 🤮, but we don't mind if you spell it as you prefer. +* Improve the performance of serialized aggregation method when involving multiple `Nullable` columns. [#55809](https://github.com/ClickHouse/ClickHouse/pull/55809) ([Amos Bird](https://github.com/amosbird)). +* Lazy build JSON's output to improve performance of ALL JOIN. [#58278](https://github.com/ClickHouse/ClickHouse/pull/58278) ([LiuNeng](https://github.com/liuneng1994)). +* Make HTTP/HTTPs connections with external services, such as AWS S3 reusable for all uses cases. Even when response is 3xx or 4xx. [#58845](https://github.com/ClickHouse/ClickHouse/pull/58845) ([Sema Checherinda](https://github.com/CheSema)). +* Improvements to aggregate functions `argMin` / `argMax` / `any` / `anyLast` / `anyHeavy`, as well as `ORDER BY {u8/u16/u32/u64/i8/i16/u32/i64) LIMIT 1` queries. [#58640](https://github.com/ClickHouse/ClickHouse/pull/58640) ([Raúl Marín](https://github.com/Algunenano)). +* Trivial optimization for column's filter. Peak memory can be reduced to 44% of the original in some cases. [#59698](https://github.com/ClickHouse/ClickHouse/pull/59698) ([李扬](https://github.com/taiyang-li)). +* Execute `multiIf` function in a columnar fashion when the result type's underlying type is a number. [#60384](https://github.com/ClickHouse/ClickHouse/pull/60384) ([李扬](https://github.com/taiyang-li)). +* Faster (almost 2x) mutexes. [#60823](https://github.com/ClickHouse/ClickHouse/pull/60823) ([Azat Khuzhin](https://github.com/azat)). +* Drain multiple connections in parallel when a distributed query is finishing. [#60845](https://github.com/ClickHouse/ClickHouse/pull/60845) ([lizhuoyu5](https://github.com/lzydmxy)). +* Optimize data movement between columns of a Nullable number or a Nullable string, which improves some micro-benchmarks. [#60846](https://github.com/ClickHouse/ClickHouse/pull/60846) ([李扬](https://github.com/taiyang-li)). +* Operations with the filesystem cache will suffer less from the lock contention. [#61066](https://github.com/ClickHouse/ClickHouse/pull/61066) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Optimize array join and other JOINs by preventing a wrong compiler's optimization. Close [#61074](https://github.com/ClickHouse/ClickHouse/issues/61074). [#61075](https://github.com/ClickHouse/ClickHouse/pull/61075) ([李扬](https://github.com/taiyang-li)). +* If a query with a syntax error contained `COLUMNS` matcher with a regular expression, the regular expression was compiled each time during the parser's backtracking, instead of being compiled once. This was a fundamental error. The compiled regexp was put to AST. But the letter A in AST means "abstract" which means it should not contain heavyweight objects. Parts of AST can be created and discarded during parsing, including a large number of backtracking. This leads to slowness on the parsing side and consequently allows DoS by a readonly user. But the main problem is that it prevents progress in fuzzers. [#61543](https://github.com/ClickHouse/ClickHouse/pull/61543) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a new analyzer pass to optimize the IN operator for a single value. [#61564](https://github.com/ClickHouse/ClickHouse/pull/61564) ([LiuNeng](https://github.com/liuneng1994)). +* DNSResolver shuffles set of resolved IPs which is needed to uniformly utilize multiple endpoints of AWS S3. [#60965](https://github.com/ClickHouse/ClickHouse/pull/60965) ([Sema Checherinda](https://github.com/CheSema)). + +#### Experimental Feature +* Support parallel reading for Azure blob storage. This improves the performance of the experimental Azure object storage. [#61503](https://github.com/ClickHouse/ClickHouse/pull/61503) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Add asynchronous WriteBuffer for Azure blob storage similar to S3. This improves the performance of the experimental Azure object storage. [#59929](https://github.com/ClickHouse/ClickHouse/pull/59929) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Use managed identity for backups IO when using Azure Blob Storage. Add a setting to prevent ClickHouse from attempting to create a non-existent container, which requires permissions at the storage account level. [#61785](https://github.com/ClickHouse/ClickHouse/pull/61785) ([Daniel Pozo Escalona](https://github.com/danipozo)). +* Add a setting `parallel_replicas_allow_in_with_subquery = 1` which allows subqueries for IN work with parallel replicas. [#60950](https://github.com/ClickHouse/ClickHouse/pull/60950) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* A change for the "zero-copy" replication: all zero copy locks related to a table have to be dropped when the table is dropped. The directory which contains these locks has to be removed also. [#57575](https://github.com/ClickHouse/ClickHouse/pull/57575) ([Sema Checherinda](https://github.com/CheSema)). + +#### Improvement +* Use `MergeTree` as a default table engine. [#60524](https://github.com/ClickHouse/ClickHouse/pull/60524) ([Alexey Milovidov](https://github.com/alexey-milovidov)) +* Enable `output_format_pretty_row_numbers` by default. It is better for usability. [#61791](https://github.com/ClickHouse/ClickHouse/pull/61791) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the previous version, some numbers in Pretty formats were not pretty enough. [#61794](https://github.com/ClickHouse/ClickHouse/pull/61794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A long value in Pretty formats won't be cut if it is the single value in the resultset, such as in the result of the `SHOW CREATE TABLE` query. [#61795](https://github.com/ClickHouse/ClickHouse/pull/61795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Similarly to `clickhouse-local`, `clickhouse-client` will accept the `--output-format` option as a synonym to the `--format` option. This closes [#59848](https://github.com/ClickHouse/ClickHouse/issues/59848). [#61797](https://github.com/ClickHouse/ClickHouse/pull/61797) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If stdout is a terminal and the output format is not specified, `clickhouse-client` and similar tools will use `PrettyCompact` by default, similarly to the interactive mode. `clickhouse-client` and `clickhouse-local` will handle command line arguments for input and output formats in a unified fashion. This closes [#61272](https://github.com/ClickHouse/ClickHouse/issues/61272). [#61800](https://github.com/ClickHouse/ClickHouse/pull/61800) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Underscore digit groups in Pretty formats for better readability. This is controlled by a new setting, `output_format_pretty_highlight_digit_groups`. [#61802](https://github.com/ClickHouse/ClickHouse/pull/61802) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add ability to override initial INSERT settings via `SYSTEM FLUSH DISTRIBUTED`. [#61832](https://github.com/ClickHouse/ClickHouse/pull/61832) ([Azat Khuzhin](https://github.com/azat)). +* Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. [#61096](https://github.com/ClickHouse/ClickHouse/pull/61096) ([Azat Khuzhin](https://github.com/azat)). +* Support files without format extension in Filesystem database. [#60795](https://github.com/ClickHouse/ClickHouse/pull/60795) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `none_only_active` mode for `distributed_ddl_output_mode` setting. [#60340](https://github.com/ClickHouse/ClickHouse/pull/60340) ([Alexander Tokmakov](https://github.com/tavplubix)). +* The advanced dashboard has slightly better colors for multi-line graphs. [#60391](https://github.com/ClickHouse/ClickHouse/pull/60391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. [#60692](https://github.com/ClickHouse/ClickHouse/pull/60692) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* While running the `MODIFY COLUMN` query for materialized views, check the inner table's structure to ensure every column exists. [#47427](https://github.com/ClickHouse/ClickHouse/pull/47427) ([sunny](https://github.com/sunny19930321)). +* String types and Enums can be used in the same context, such as: arrays, UNION queries, conditional expressions. This closes [#60726](https://github.com/ClickHouse/ClickHouse/issues/60726). [#60727](https://github.com/ClickHouse/ClickHouse/pull/60727) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow declaring Enums in the structure of external data for query processing (this is an immediate temporary table that you can provide for your query). [#57857](https://github.com/ClickHouse/ClickHouse/pull/57857) ([Duc Canh Le](https://github.com/canhld94)). +* Consider lightweight deleted rows when selecting parts to merge, so the disk size of the resulting part will be estimated better. [#58223](https://github.com/ClickHouse/ClickHouse/pull/58223) ([Zhuo Qiu](https://github.com/jewelzqiu)). +* Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. [#59016](https://github.com/ClickHouse/ClickHouse/pull/59016) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like `_part_offset`. [#59033](https://github.com/ClickHouse/ClickHouse/pull/59033) ([Amos Bird](https://github.com/amosbird)). Improved overall usability of virtual columns. Now it is allowed to use virtual columns in `PREWHERE` (it's worthwhile for non-const virtual columns like `_part_offset`). Now a builtin documentation is available for virtual columns as a comment of column in `DESCRIBE` query with enabled setting `describe_include_virtual_columns`. [#60205](https://github.com/ClickHouse/ClickHouse/pull/60205) ([Anton Popov](https://github.com/CurtizJ)). +* Instead using a constant key, now object storage generates key for determining remove objects capability. [#59495](https://github.com/ClickHouse/ClickHouse/pull/59495) ([Sema Checherinda](https://github.com/CheSema)). +* Allow "local" as object storage type instead of "local_blob_storage". [#60165](https://github.com/ClickHouse/ClickHouse/pull/60165) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Parallel flush of pending INSERT blocks of Distributed engine on `DETACH`/server shutdown and `SYSTEM FLUSH DISTRIBUTED` (Parallelism will work only if you have multi-disk policy for a table (like everything in the Distributed engine right now)). [#60225](https://github.com/ClickHouse/ClickHouse/pull/60225) ([Azat Khuzhin](https://github.com/azat)). +* Add a setting to force read-through cache for merges. [#60308](https://github.com/ClickHouse/ClickHouse/pull/60308) ([Kseniia Sumarokova](https://github.com/kssenii)). +* An improvement for the MySQL compatibility protocol. The issue [#57598](https://github.com/ClickHouse/ClickHouse/issues/57598) mentions a variant behaviour regarding transaction handling. An issued COMMIT/ROLLBACK when no transaction is active is reported as an error contrary to MySQL behaviour. [#60338](https://github.com/ClickHouse/ClickHouse/pull/60338) ([PapaToemmsn](https://github.com/PapaToemmsn)). +* Function `substring` now has a new alias `byteSlice`. [#60494](https://github.com/ClickHouse/ClickHouse/pull/60494) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed server setting `dns_cache_max_size` to `dns_cache_max_entries` to reduce ambiguity. [#60500](https://github.com/ClickHouse/ClickHouse/pull/60500) ([Kirill Nikiforov](https://github.com/allmazz)). +* `SHOW INDEX | INDEXES | INDICES | KEYS` no longer sorts by the primary key columns (which was unintuitive). [#60514](https://github.com/ClickHouse/ClickHouse/pull/60514) ([Robert Schulze](https://github.com/rschu1ze)). +* Keeper improvement: abort during startup if an invalid snapshot is detected to avoid data loss. [#60537](https://github.com/ClickHouse/ClickHouse/pull/60537) ([Antonio Andelic](https://github.com/antonio2368)). +* Update tzdata to 2024a. [#60768](https://github.com/ClickHouse/ClickHouse/pull/60768) ([Raúl Marín](https://github.com/Algunenano)). +* Keeper improvement: support `leadership_expiry_ms` in Keeper's settings. [#60806](https://github.com/ClickHouse/ClickHouse/pull/60806) ([Brokenice0415](https://github.com/Brokenice0415)). +* Always infer exponential numbers in JSON formats regardless of the setting `input_format_try_infer_exponent_floats`. Add setting `input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects` that allows to use String type for ambiguous paths instead of an exception during named Tuples inference from JSON objects. [#60808](https://github.com/ClickHouse/ClickHouse/pull/60808) ([Kruglov Pavel](https://github.com/Avogar)). +* Add support for `START TRANSACTION` syntax typically used in MySQL syntax, resolving https://github.com/ClickHouse/ClickHouse/discussions/60865. [#60886](https://github.com/ClickHouse/ClickHouse/pull/60886) ([Zach Naimon](https://github.com/ArctypeZach)). +* Add a flag for the full-sorting merge join algorithm to treat null as biggest/smallest. So the behavior can be compitable with other SQL systems, like Apache Spark. [#60896](https://github.com/ClickHouse/ClickHouse/pull/60896) ([loudongfeng](https://github.com/loudongfeng)). +* Support detect output format by file exctension in `clickhouse-client` and `clickhouse-local`. [#61036](https://github.com/ClickHouse/ClickHouse/pull/61036) ([豪肥肥](https://github.com/HowePa)). +* Update memory limit in runtime when Linux's CGroups value changed. [#61049](https://github.com/ClickHouse/ClickHouse/pull/61049) ([Han Fei](https://github.com/hanfei1991)). +* Add the function `toUInt128OrZero`, which was missed by mistake (the mistake is related to https://github.com/ClickHouse/ClickHouse/pull/945). The compatibility aliases `FROM_UNIXTIME` and `DATE_FORMAT` (they are not ClickHouse-native and only exist for MySQL compatibility) have been made case insensitive, as expected for SQL-compatibility aliases. [#61114](https://github.com/ClickHouse/ClickHouse/pull/61114) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improvements for the access checks, allowing to revoke of unpossessed rights in case the target user doesn't have the revoking grants either. Example: `GRANT SELECT ON *.* TO user1; REVOKE SELECT ON system.* FROM user1;`. [#61115](https://github.com/ClickHouse/ClickHouse/pull/61115) ([pufit](https://github.com/pufit)). +* Fix `has()` function with `Nullable` column (fixes [#60214](https://github.com/ClickHouse/ClickHouse/issues/60214)). [#61249](https://github.com/ClickHouse/ClickHouse/pull/61249) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Now it's possible to specify the attribute `merge="true"` in config substitutions for subtrees ``. In case this attribute specified, clickhouse will merge subtree with existing configuration, otherwise default behavior is append new content to configuration. [#61299](https://github.com/ClickHouse/ClickHouse/pull/61299) ([alesapin](https://github.com/alesapin)). +* Add async metrics for virtual memory mappings: `VMMaxMapCount` & `VMNumMaps`. Closes [#60662](https://github.com/ClickHouse/ClickHouse/issues/60662). [#61354](https://github.com/ClickHouse/ClickHouse/pull/61354) ([Tuan Pham Anh](https://github.com/tuanpavn)). +* Use `temporary_files_codec` setting in all places where we create temporary data, for example external memory sorting and external memory GROUP BY. Before it worked only in `partial_merge` JOIN algorithm. [#61456](https://github.com/ClickHouse/ClickHouse/pull/61456) ([Maksim Kita](https://github.com/kitaisreal)). +* Add a new setting `max_parser_backtracks` which allows to limit the complexity of query parsing. [#61502](https://github.com/ClickHouse/ClickHouse/pull/61502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Less contention during dynamic resize of the filesystem cache. [#61524](https://github.com/ClickHouse/ClickHouse/pull/61524) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disallow sharded mode of StorageS3 queue, because it will be rewritten. [#61537](https://github.com/ClickHouse/ClickHouse/pull/61537) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed typo: from `use_leagcy_max_level` to `use_legacy_max_level`. [#61545](https://github.com/ClickHouse/ClickHouse/pull/61545) ([William Schoeffel](https://github.com/wiledusc)). +* Remove some duplicate entries in `system.blob_storage_log`. [#61622](https://github.com/ClickHouse/ClickHouse/pull/61622) ([YenchangChan](https://github.com/YenchangChan)). +* Added `current_user` function as a compatibility alias for MySQL. [#61770](https://github.com/ClickHouse/ClickHouse/pull/61770) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix inconsistent floating point aggregate function states in mixed x86-64 / ARM clusters [#60610](https://github.com/ClickHouse/ClickHouse/pull/60610) ([Harry Lee](https://github.com/HarryLeeIBM)). + +#### Build/Testing/Packaging Improvement +* The real-time query profiler now works on AArch64. In previous versions, it worked only when a program didn't spend time inside a syscall. [#60807](https://github.com/ClickHouse/ClickHouse/pull/60807) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ClickHouse version has been added to docker labels. Closes [#54224](https://github.com/ClickHouse/ClickHouse/issues/54224). [#60949](https://github.com/ClickHouse/ClickHouse/pull/60949) ([Nikolay Monkov](https://github.com/nikmonkov)). +* Upgrade `prqlc` to 0.11.3. [#60616](https://github.com/ClickHouse/ClickHouse/pull/60616) ([Maximilian Roos](https://github.com/max-sixty)). +* Add generic query text fuzzer in `clickhouse-local`. [#61508](https://github.com/ClickHouse/ClickHouse/pull/61508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix finished_mutations_to_keep=0 for MergeTree (as docs says 0 is to keep everything) [#60031](https://github.com/ClickHouse/ClickHouse/pull/60031) ([Azat Khuzhin](https://github.com/azat)). +* Something was wrong with the FINAL optimization, here is how the author describes it: "PartsSplitter invalid ranges for the same part". [#60041](https://github.com/ClickHouse/ClickHouse/pull/60041) ([Maksim Kita](https://github.com/kitaisreal)). +* Something was wrong with Apache Hive, which is experimental and not supported. [#60262](https://github.com/ClickHouse/ClickHouse/pull/60262) ([shanfengp](https://github.com/Aed-p)). +* An improvement for experimental parallel replicas: force reanalysis if parallel replicas changed [#60362](https://github.com/ClickHouse/ClickHouse/pull/60362) ([Raúl Marín](https://github.com/Algunenano)). +* Fix usage of plain metadata type with new disks configuration option [#60396](https://github.com/ClickHouse/ClickHouse/pull/60396) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't allow to set max_parallel_replicas to 0 as it doesn't make sense [#60430](https://github.com/ClickHouse/ClickHouse/pull/60430) ([Kruglov Pavel](https://github.com/Avogar)). +* Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike [#60451](https://github.com/ClickHouse/ClickHouse/pull/60451) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid calculation of scalar subqueries for CREATE TABLE. [#60464](https://github.com/ClickHouse/ClickHouse/pull/60464) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix deadlock in parallel parsing when lots of rows are skipped due to errors [#60516](https://github.com/ClickHouse/ClickHouse/pull/60516) ([Kruglov Pavel](https://github.com/Avogar)). +* Something was wrong with experimental KQL (Kusto) support: fix `max_query_size_for_kql_compound_operator`: [#60534](https://github.com/ClickHouse/ClickHouse/pull/60534) ([Yong Wang](https://github.com/kashwy)). +* Keeper fix: add timeouts when waiting for commit logs [#60544](https://github.com/ClickHouse/ClickHouse/pull/60544) ([Antonio Andelic](https://github.com/antonio2368)). +* Don't output number tips for date types [#60577](https://github.com/ClickHouse/ClickHouse/pull/60577) ([Raúl Marín](https://github.com/Algunenano)). +* Fix reading from MergeTree with non-deterministic functions in filter [#60586](https://github.com/ClickHouse/ClickHouse/pull/60586) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix logical error on bad compatibility setting value type [#60596](https://github.com/ClickHouse/ClickHouse/pull/60596) ([Kruglov Pavel](https://github.com/Avogar)). +* fix(prql): Robust panic handler [#60615](https://github.com/ClickHouse/ClickHouse/pull/60615) ([Maximilian Roos](https://github.com/max-sixty)). +* Fix `intDiv` for decimal and date arguments [#60672](https://github.com/ClickHouse/ClickHouse/pull/60672) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix: expand CTE in alter modify query [#60682](https://github.com/ClickHouse/ClickHouse/pull/60682) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix system.parts for non-Atomic/Ordinary database engine (i.e. Memory) [#60689](https://github.com/ClickHouse/ClickHouse/pull/60689) ([Azat Khuzhin](https://github.com/azat)). +* Fix "Invalid storage definition in metadata file" for parameterized views [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Fix buffer overflow in CompressionCodecMultiple [#60731](https://github.com/ClickHouse/ClickHouse/pull/60731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove nonsense from SQL/JSON [#60738](https://github.com/ClickHouse/ClickHouse/pull/60738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove wrong assertion in aggregate function quantileGK [#60740](https://github.com/ClickHouse/ClickHouse/pull/60740) ([李扬](https://github.com/taiyang-li)). +* Fix insert-select + insert_deduplication_token bug by setting streams to 1 [#60745](https://github.com/ClickHouse/ClickHouse/pull/60745) ([Jordi Villar](https://github.com/jrdi)). +* Prevent setting custom metadata headers on unsupported multipart upload operations [#60748](https://github.com/ClickHouse/ClickHouse/pull/60748) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Fix toStartOfInterval [#60763](https://github.com/ClickHouse/ClickHouse/pull/60763) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix crash in arrayEnumerateRanked [#60764](https://github.com/ClickHouse/ClickHouse/pull/60764) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash when using input() in INSERT SELECT JOIN [#60765](https://github.com/ClickHouse/ClickHouse/pull/60765) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash with different allow_experimental_analyzer value in subqueries [#60770](https://github.com/ClickHouse/ClickHouse/pull/60770) ([Dmitry Novik](https://github.com/novikd)). +* Remove recursion when reading from S3 [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix possible stuck on error in HashedDictionaryParallelLoader [#60926](https://github.com/ClickHouse/ClickHouse/pull/60926) ([vdimir](https://github.com/vdimir)). +* Fix async RESTORE with Replicated database [#60934](https://github.com/ClickHouse/ClickHouse/pull/60934) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix deadlock in async inserts to `Log` tables via native protocol [#61055](https://github.com/ClickHouse/ClickHouse/pull/61055) ([Anton Popov](https://github.com/CurtizJ)). +* Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary [#61196](https://github.com/ClickHouse/ClickHouse/pull/61196) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix multiple bugs in groupArraySorted [#61203](https://github.com/ClickHouse/ClickHouse/pull/61203) ([Raúl Marín](https://github.com/Algunenano)). +* Fix Keeper reconfig for standalone binary [#61233](https://github.com/ClickHouse/ClickHouse/pull/61233) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix usage of session_token in S3 engine [#61234](https://github.com/ClickHouse/ClickHouse/pull/61234) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible incorrect result of aggregate function `uniqExact` [#61257](https://github.com/ClickHouse/ClickHouse/pull/61257) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bugs in show database [#61269](https://github.com/ClickHouse/ClickHouse/pull/61269) ([Raúl Marín](https://github.com/Algunenano)). +* Fix logical error in RabbitMQ storage with MATERIALIZED columns [#61320](https://github.com/ClickHouse/ClickHouse/pull/61320) ([vdimir](https://github.com/vdimir)). +* Fix CREATE OR REPLACE DICTIONARY [#61356](https://github.com/ClickHouse/ClickHouse/pull/61356) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix ATTACH query with external ON CLUSTER [#61365](https://github.com/ClickHouse/ClickHouse/pull/61365) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix consecutive keys optimization for nullable keys [#61393](https://github.com/ClickHouse/ClickHouse/pull/61393) ([Anton Popov](https://github.com/CurtizJ)). +* fix issue of actions dag split [#61458](https://github.com/ClickHouse/ClickHouse/pull/61458) ([Raúl Marín](https://github.com/Algunenano)). +* Fix finishing a failed RESTORE [#61466](https://github.com/ClickHouse/ClickHouse/pull/61466) ([Vitaly Baranov](https://github.com/vitlibar)). +* Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings [#61468](https://github.com/ClickHouse/ClickHouse/pull/61468) ([Raúl Marín](https://github.com/Algunenano)). +* Allow queuing in restore pool [#61475](https://github.com/ClickHouse/ClickHouse/pull/61475) ([Nikita Taranov](https://github.com/nickitat)). +* Fix an inconsistency when reading system.parts using UUID. [#61479](https://github.com/ClickHouse/ClickHouse/pull/61479) ([Dan Wu](https://github.com/wudanzy)). +* Fix ALTER QUERY MODIFY SQL SECURITY [#61480](https://github.com/ClickHouse/ClickHouse/pull/61480) ([pufit](https://github.com/pufit)). +* Fix crash in window view [#61526](https://github.com/ClickHouse/ClickHouse/pull/61526) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `repeat` with non native integers [#61527](https://github.com/ClickHouse/ClickHouse/pull/61527) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix client's `-s` argument [#61530](https://github.com/ClickHouse/ClickHouse/pull/61530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix crash in arrayPartialReverseSort [#61539](https://github.com/ClickHouse/ClickHouse/pull/61539) ([Raúl Marín](https://github.com/Algunenano)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix addDays cause an error when used DateTime64 [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* Disallow LowCardinality input type for JSONExtract [#61617](https://github.com/ClickHouse/ClickHouse/pull/61617) ([Julia Kartseva](https://github.com/jkartseva)). +* Fix `system.part_log` for async insert with deduplication [#61620](https://github.com/ClickHouse/ClickHouse/pull/61620) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a `Non-ready set` exception for system.parts. [#61666](https://github.com/ClickHouse/ClickHouse/pull/61666) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix actual_part_name for REPLACE_RANGE (`Entry actual part isn't empty yet`) [#61675](https://github.com/ClickHouse/ClickHouse/pull/61675) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix a sanitizer report in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). +* Fix an observation that the RANGE frame is not supported for Nullable columns. [#61766](https://github.com/ClickHouse/ClickHouse/pull/61766) ([YuanLiu](https://github.com/ditgittube)). + ### ClickHouse release 24.2, 2024-02-29 #### Backward Incompatible Change From 385f6661723f04a6ee8d37ab70eb6fb5088e2888 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 08:30:23 +0100 Subject: [PATCH 097/154] Update Changelog --- CHANGELOG.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9117289028d..14a0ff94a04 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -81,7 +81,7 @@ * Consider lightweight deleted rows when selecting parts to merge, so the disk size of the resulting part will be estimated better. [#58223](https://github.com/ClickHouse/ClickHouse/pull/58223) ([Zhuo Qiu](https://github.com/jewelzqiu)). * Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. [#59016](https://github.com/ClickHouse/ClickHouse/pull/59016) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like `_part_offset`. [#59033](https://github.com/ClickHouse/ClickHouse/pull/59033) ([Amos Bird](https://github.com/amosbird)). Improved overall usability of virtual columns. Now it is allowed to use virtual columns in `PREWHERE` (it's worthwhile for non-const virtual columns like `_part_offset`). Now a builtin documentation is available for virtual columns as a comment of column in `DESCRIBE` query with enabled setting `describe_include_virtual_columns`. [#60205](https://github.com/ClickHouse/ClickHouse/pull/60205) ([Anton Popov](https://github.com/CurtizJ)). -* Instead using a constant key, now object storage generates key for determining remove objects capability. [#59495](https://github.com/ClickHouse/ClickHouse/pull/59495) ([Sema Checherinda](https://github.com/CheSema)). +* Instead of using a constant key, now object storage generates key for determining remove objects capability. [#59495](https://github.com/ClickHouse/ClickHouse/pull/59495) ([Sema Checherinda](https://github.com/CheSema)). * Allow "local" as object storage type instead of "local_blob_storage". [#60165](https://github.com/ClickHouse/ClickHouse/pull/60165) ([Kseniia Sumarokova](https://github.com/kssenii)). * Parallel flush of pending INSERT blocks of Distributed engine on `DETACH`/server shutdown and `SYSTEM FLUSH DISTRIBUTED` (Parallelism will work only if you have multi-disk policy for a table (like everything in the Distributed engine right now)). [#60225](https://github.com/ClickHouse/ClickHouse/pull/60225) ([Azat Khuzhin](https://github.com/azat)). * Add a setting to force read-through cache for merges. [#60308](https://github.com/ClickHouse/ClickHouse/pull/60308) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -148,7 +148,7 @@ * Fix crash with different allow_experimental_analyzer value in subqueries [#60770](https://github.com/ClickHouse/ClickHouse/pull/60770) ([Dmitry Novik](https://github.com/novikd)). * Remove recursion when reading from S3 [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). * Fix possible stuck on error in HashedDictionaryParallelLoader [#60926](https://github.com/ClickHouse/ClickHouse/pull/60926) ([vdimir](https://github.com/vdimir)). -* Fix async RESTORE with Replicated database [#60934](https://github.com/ClickHouse/ClickHouse/pull/60934) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix async RESTORE with Replicated database (experimental feature) [#60934](https://github.com/ClickHouse/ClickHouse/pull/60934) ([Antonio Andelic](https://github.com/antonio2368)). * Fix deadlock in async inserts to `Log` tables via native protocol [#61055](https://github.com/ClickHouse/ClickHouse/pull/61055) ([Anton Popov](https://github.com/CurtizJ)). * Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary [#61196](https://github.com/ClickHouse/ClickHouse/pull/61196) ([Kruglov Pavel](https://github.com/Avogar)). * Fix multiple bugs in groupArraySorted [#61203](https://github.com/ClickHouse/ClickHouse/pull/61203) ([Raúl Marín](https://github.com/Algunenano)). @@ -166,8 +166,8 @@ * Allow queuing in restore pool [#61475](https://github.com/ClickHouse/ClickHouse/pull/61475) ([Nikita Taranov](https://github.com/nickitat)). * Fix an inconsistency when reading system.parts using UUID. [#61479](https://github.com/ClickHouse/ClickHouse/pull/61479) ([Dan Wu](https://github.com/wudanzy)). * Fix ALTER QUERY MODIFY SQL SECURITY [#61480](https://github.com/ClickHouse/ClickHouse/pull/61480) ([pufit](https://github.com/pufit)). -* Fix crash in window view [#61526](https://github.com/ClickHouse/ClickHouse/pull/61526) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix `repeat` with non native integers [#61527](https://github.com/ClickHouse/ClickHouse/pull/61527) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a crash in window view (experimental feature) [#61526](https://github.com/ClickHouse/ClickHouse/pull/61526) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `repeat` with non-native integers [#61527](https://github.com/ClickHouse/ClickHouse/pull/61527) ([Antonio Andelic](https://github.com/antonio2368)). * Fix client's `-s` argument [#61530](https://github.com/ClickHouse/ClickHouse/pull/61530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix crash in arrayPartialReverseSort [#61539](https://github.com/ClickHouse/ClickHouse/pull/61539) ([Raúl Marín](https://github.com/Algunenano)). * Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). From c525436c76cf983dcae5b00d387b43aed563983d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 07:53:46 +0000 Subject: [PATCH 098/154] Update version_date.tsv and changelogs after v23.3.22.3-lts --- docs/changelogs/v23.3.22.3-lts.md | 13 +++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 2 files changed, 17 insertions(+) create mode 100644 docs/changelogs/v23.3.22.3-lts.md diff --git a/docs/changelogs/v23.3.22.3-lts.md b/docs/changelogs/v23.3.22.3-lts.md new file mode 100644 index 00000000000..2900480e12d --- /dev/null +++ b/docs/changelogs/v23.3.22.3-lts.md @@ -0,0 +1,13 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.3.22.3-lts (04075bf96a1) FIXME as compared to v23.3.21.26-lts (d9672a3731f) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix crash in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 32fbfee8274..e4482998fd6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 +v24.1.8.22-stable 2024-03-26 v24.1.7.18-stable 2024-03-15 v24.1.6.52-stable 2024-03-07 v24.1.5.6-stable 2024-02-14 @@ -7,6 +8,7 @@ v24.1.4.20-stable 2024-02-14 v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.6.19-stable 2024-03-26 v23.12.5.81-stable 2024-03-15 v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 @@ -29,6 +31,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.12.13-lts 2024-03-26 v23.8.11.28-lts 2024-03-15 v23.8.10.43-lts 2024-03-05 v23.8.9.54-lts 2024-01-05 @@ -60,6 +63,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.22.3-lts 2024-03-26 v23.3.21.26-lts 2024-03-15 v23.3.20.27-lts 2024-03-05 v23.3.19.32-lts 2024-01-05 From bb83b17c6bdf6b36113ebdea08237cf561c0352c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 07:55:23 +0000 Subject: [PATCH 099/154] Update version_date.tsv and changelogs after v23.8.12.13-lts --- docs/changelogs/v23.8.12.13-lts.md | 20 ++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 2 files changed, 24 insertions(+) create mode 100644 docs/changelogs/v23.8.12.13-lts.md diff --git a/docs/changelogs/v23.8.12.13-lts.md b/docs/changelogs/v23.8.12.13-lts.md new file mode 100644 index 00000000000..dbb36fdc00e --- /dev/null +++ b/docs/changelogs/v23.8.12.13-lts.md @@ -0,0 +1,20 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.12.13-lts (bdbd0d87e5d) FIXME as compared to v23.8.11.28-lts (31879d2ab4c) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Improve isolation of query cache entries under re-created users or role switches [#58611](https://github.com/ClickHouse/ClickHouse/pull/58611) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix crash in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#61428](https://github.com/ClickHouse/ClickHouse/issues/61428):. [#61374](https://github.com/ClickHouse/ClickHouse/pull/61374) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#61484](https://github.com/ClickHouse/ClickHouse/issues/61484): ... [#61441](https://github.com/ClickHouse/ClickHouse/pull/61441) ([Max K.](https://github.com/maxknv)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 32fbfee8274..e4482998fd6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 +v24.1.8.22-stable 2024-03-26 v24.1.7.18-stable 2024-03-15 v24.1.6.52-stable 2024-03-07 v24.1.5.6-stable 2024-02-14 @@ -7,6 +8,7 @@ v24.1.4.20-stable 2024-02-14 v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.6.19-stable 2024-03-26 v23.12.5.81-stable 2024-03-15 v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 @@ -29,6 +31,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.12.13-lts 2024-03-26 v23.8.11.28-lts 2024-03-15 v23.8.10.43-lts 2024-03-05 v23.8.9.54-lts 2024-01-05 @@ -60,6 +63,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.22.3-lts 2024-03-26 v23.3.21.26-lts 2024-03-15 v23.3.20.27-lts 2024-03-05 v23.3.19.32-lts 2024-01-05 From 9b052d059d9c5ea3ff4f0ef9483b1b7298a4b4df Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 09:00:20 +0100 Subject: [PATCH 100/154] No "please" --- src/Client/Suggest.cpp | 1 - src/Interpreters/Context.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index eb98c3a5740..03df582de10 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 086ed333366..7030522dd2a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1219,7 +1219,7 @@ void Context::addWarningMessageAboutDatabaseOrdinary(const String & database_nam /// We don't use getFlagsPath method, because it takes a shared lock. auto convert_databases_flag = fs::path(shared->flags_path) / "convert_ordinary_to_atomic"; auto message = fmt::format("Server has databases (for example `{}`) with Ordinary engine, which was deprecated. " - "To convert this database to a new Atomic engine, please create a forcing flag {} and make sure that ClickHouse has write permission for it. " + "To convert this database to a new Atomic engine, create a flag {} and make sure that ClickHouse has write permission for it. " "Example: sudo touch '{}' && sudo chmod 666 '{}'", database_name, convert_databases_flag.string(), convert_databases_flag.string(), convert_databases_flag.string()); From 6236d35ee783d76b8c3a814e2dc56af611ab7369 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 08:03:44 +0000 Subject: [PATCH 101/154] Update version_date.tsv and changelogs after v23.12.6.19-stable --- docs/changelogs/v23.12.6.19-stable.md | 24 ++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 2 files changed, 28 insertions(+) create mode 100644 docs/changelogs/v23.12.6.19-stable.md diff --git a/docs/changelogs/v23.12.6.19-stable.md b/docs/changelogs/v23.12.6.19-stable.md new file mode 100644 index 00000000000..4659532d3de --- /dev/null +++ b/docs/changelogs/v23.12.6.19-stable.md @@ -0,0 +1,24 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.12.6.19-stable (40080a3c2a4) FIXME as compared to v23.12.5.81-stable (a0fbe3ae813) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Improve isolation of query cache entries under re-created users or role switches [#58611](https://github.com/ClickHouse/ClickHouse/pull/58611) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix possible incorrect result of aggregate function `uniqExact` [#61257](https://github.com/ClickHouse/ClickHouse/pull/61257) ([Anton Popov](https://github.com/CurtizJ)). +* Fix consecutive keys optimization for nullable keys [#61393](https://github.com/ClickHouse/ClickHouse/pull/61393) ([Anton Popov](https://github.com/CurtizJ)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix crash in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#61429](https://github.com/ClickHouse/ClickHouse/issues/61429):. [#61374](https://github.com/ClickHouse/ClickHouse/pull/61374) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#61486](https://github.com/ClickHouse/ClickHouse/issues/61486): ... [#61441](https://github.com/ClickHouse/ClickHouse/pull/61441) ([Max K.](https://github.com/maxknv)). +* Backported in [#61641](https://github.com/ClickHouse/ClickHouse/issues/61641):. [#61592](https://github.com/ClickHouse/ClickHouse/pull/61592) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#61811](https://github.com/ClickHouse/ClickHouse/issues/61811): ![Screenshot_20240323_025055](https://github.com/ClickHouse/ClickHouse/assets/18581488/ccaab212-a1d3-4dfb-8d56-b1991760b6bf). [#61801](https://github.com/ClickHouse/ClickHouse/pull/61801) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 32fbfee8274..e4482998fd6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 +v24.1.8.22-stable 2024-03-26 v24.1.7.18-stable 2024-03-15 v24.1.6.52-stable 2024-03-07 v24.1.5.6-stable 2024-02-14 @@ -7,6 +8,7 @@ v24.1.4.20-stable 2024-02-14 v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.6.19-stable 2024-03-26 v23.12.5.81-stable 2024-03-15 v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 @@ -29,6 +31,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.12.13-lts 2024-03-26 v23.8.11.28-lts 2024-03-15 v23.8.10.43-lts 2024-03-05 v23.8.9.54-lts 2024-01-05 @@ -60,6 +63,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.22.3-lts 2024-03-26 v23.3.21.26-lts 2024-03-15 v23.3.20.27-lts 2024-03-05 v23.3.19.32-lts 2024-01-05 From 88665a1b5b665ccf6feeaf44778c86f0254554a5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 26 Mar 2024 08:05:23 +0000 Subject: [PATCH 102/154] Update version_date.tsv and changelogs after v24.1.8.22-stable --- docs/changelogs/v24.1.8.22-stable.md | 32 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++++ 2 files changed, 36 insertions(+) create mode 100644 docs/changelogs/v24.1.8.22-stable.md diff --git a/docs/changelogs/v24.1.8.22-stable.md b/docs/changelogs/v24.1.8.22-stable.md new file mode 100644 index 00000000000..f780de41c40 --- /dev/null +++ b/docs/changelogs/v24.1.8.22-stable.md @@ -0,0 +1,32 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.1.8.22-stable (7fb8f96d3da) FIXME as compared to v24.1.7.18-stable (90925babd78) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix possible incorrect result of aggregate function `uniqExact` [#61257](https://github.com/ClickHouse/ClickHouse/pull/61257) ([Anton Popov](https://github.com/CurtizJ)). +* Fix consecutive keys optimization for nullable keys [#61393](https://github.com/ClickHouse/ClickHouse/pull/61393) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bug when reading system.parts using UUID (issue 61220). [#61479](https://github.com/ClickHouse/ClickHouse/pull/61479) ([Dan Wu](https://github.com/wudanzy)). +* Fix client `-s` argument [#61530](https://github.com/ClickHouse/ClickHouse/pull/61530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix crash in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Backported in [#61431](https://github.com/ClickHouse/ClickHouse/issues/61431):. [#61374](https://github.com/ClickHouse/ClickHouse/pull/61374) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#61488](https://github.com/ClickHouse/ClickHouse/issues/61488): ... [#61441](https://github.com/ClickHouse/ClickHouse/pull/61441) ([Max K.](https://github.com/maxknv)). +* Backported in [#61642](https://github.com/ClickHouse/ClickHouse/issues/61642):. [#61592](https://github.com/ClickHouse/ClickHouse/pull/61592) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#61479](https://github.com/ClickHouse/ClickHouse/issues/61479) to 24.1: Fix bug when reading system.parts using UUID (issue 61220)."'. [#61775](https://github.com/ClickHouse/ClickHouse/pull/61775) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Speed up cctools building [#61011](https://github.com/ClickHouse/ClickHouse/pull/61011) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 32fbfee8274..e4482998fd6 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 +v24.1.8.22-stable 2024-03-26 v24.1.7.18-stable 2024-03-15 v24.1.6.52-stable 2024-03-07 v24.1.5.6-stable 2024-02-14 @@ -7,6 +8,7 @@ v24.1.4.20-stable 2024-02-14 v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.6.19-stable 2024-03-26 v23.12.5.81-stable 2024-03-15 v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 @@ -29,6 +31,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.12.13-lts 2024-03-26 v23.8.11.28-lts 2024-03-15 v23.8.10.43-lts 2024-03-05 v23.8.9.54-lts 2024-01-05 @@ -60,6 +63,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.22.3-lts 2024-03-26 v23.3.21.26-lts 2024-03-15 v23.3.20.27-lts 2024-03-05 v23.3.19.32-lts 2024-01-05 From ef397f9e37e166159cfe80f6397f963075aaf32c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 09:17:36 +0100 Subject: [PATCH 103/154] Update changelog --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 14a0ff94a04..e576fb447c1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -38,7 +38,6 @@ * Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). * 30x faster printing for 256-bit integers. [#61100](https://github.com/ClickHouse/ClickHouse/pull/61100) ([Raúl Marín](https://github.com/Algunenano)). * If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting `primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns` with the value `0.9` by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. [#60255](https://github.com/ClickHouse/ClickHouse/pull/60255) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. [#60420](https://github.com/ClickHouse/ClickHouse/pull/60420) ([豪肥肥](https://github.com/HowePa)). I appreciate if you will continue to write it correctly, e.g., `JSON` 😇, not `Json` 🤮, but we don't mind if you spell it as you prefer. * Improve the performance of serialized aggregation method when involving multiple `Nullable` columns. [#55809](https://github.com/ClickHouse/ClickHouse/pull/55809) ([Amos Bird](https://github.com/amosbird)). * Lazy build JSON's output to improve performance of ALL JOIN. [#58278](https://github.com/ClickHouse/ClickHouse/pull/58278) ([LiuNeng](https://github.com/liuneng1994)). * Make HTTP/HTTPs connections with external services, such as AWS S3 reusable for all uses cases. Even when response is 3xx or 4xx. [#58845](https://github.com/ClickHouse/ClickHouse/pull/58845) ([Sema Checherinda](https://github.com/CheSema)). @@ -72,6 +71,7 @@ * Add ability to override initial INSERT settings via `SYSTEM FLUSH DISTRIBUTED`. [#61832](https://github.com/ClickHouse/ClickHouse/pull/61832) ([Azat Khuzhin](https://github.com/azat)). * Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. [#61096](https://github.com/ClickHouse/ClickHouse/pull/61096) ([Azat Khuzhin](https://github.com/azat)). * Support files without format extension in Filesystem database. [#60795](https://github.com/ClickHouse/ClickHouse/pull/60795) ([Kruglov Pavel](https://github.com/Avogar)). +* Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. [#60420](https://github.com/ClickHouse/ClickHouse/pull/60420) ([豪肥肥](https://github.com/HowePa)). I appreciate if you will continue to write it correctly, e.g., `JSON` 😇, not `Json` 🤮, but we don't mind if you spell it as you prefer. * Added `none_only_active` mode for `distributed_ddl_output_mode` setting. [#60340](https://github.com/ClickHouse/ClickHouse/pull/60340) ([Alexander Tokmakov](https://github.com/tavplubix)). * The advanced dashboard has slightly better colors for multi-line graphs. [#60391](https://github.com/ClickHouse/ClickHouse/pull/60391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. [#60692](https://github.com/ClickHouse/ClickHouse/pull/60692) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From b2945a5ad4f3be386e66f2c53803fada9eb82c72 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 11:25:56 +0100 Subject: [PATCH 104/154] Mark Replicated table as readonly in case of static disk (read-only/write-once) Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d8aefdf5b4c..c472c11e7f8 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -351,7 +351,7 @@ public: bool canUseZeroCopyReplication() const; - bool isTableReadOnly () { return is_readonly; } + bool isTableReadOnly () { return is_readonly || isStaticStorage(); } std::optional hasMetadataInZooKeeper () { return has_metadata_in_zookeeper; } From 77de81f996d244f3e96c12274a7f9328d493cfd0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:04:44 +0100 Subject: [PATCH 105/154] Simpler and less error prone PoolWithFailoverBase::TryResult::reset() Signed-off-by: Azat Khuzhin --- src/Common/PoolWithFailoverBase.h | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index cf270c9dad0..740103e53bc 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -73,13 +73,7 @@ public: { TryResult() = default; - void reset() - { - entry = Entry(); - is_usable = false; - is_up_to_date = false; - delay = 0; - } + void reset() { *this = {}; } Entry entry; /// use isNull() to check if connection is established bool is_usable = false; /// if connection is established, then can be false only with table check From b9469e2729e02c69bf73f7af953b4c5f23f78ba3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 23 Jan 2024 22:22:23 +0100 Subject: [PATCH 106/154] Do not try to INSERT into readonly replicas for Distributed engine Sometimes replica may be readonly for a long time, and this will make some INSERT queries fail, but it does not make sense to INSERT into readonly replica, so let's ignore them. But note, that this will require to extend TableStatus (not extend, but introduce new version), that will have is_readonly field. Also before background INSERT into Distributed does not uses getManyChecked() which means that they do not request TableStatus packet, while now they would, though this is minor (just a note). v2: Add a note about max_replica_delay_for_distributed_queries for INSERT v3: Skip read-only replicas for async INSERT into Distributed v4: Remove extra @insert parameter for ConnectionPool*::get* It make sense only when the table name had passed -- ConnectionPoolWithFailover::getManyChecked() v5: rebase on top LoggerPtr v6: rebase v7: rebase v8: move TryResult::is_readonly into the end Signed-off-by: Azat Khuzhin --- src/Client/ConnectionEstablisher.cpp | 18 ++++++++++++++++-- src/Client/ConnectionEstablisher.h | 3 +++ src/Client/ConnectionPoolWithFailover.cpp | 16 +++++++++------- src/Client/ConnectionPoolWithFailover.h | 10 +++++++--- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Common/PoolWithFailoverBase.h | 1 + src/Common/ProfileEvents.cpp | 1 + src/Core/ProtocolDefines.h | 5 ++++- src/Interpreters/TablesStatus.cpp | 14 +++++++++----- src/Interpreters/TablesStatus.h | 6 ++++-- src/Processors/QueryPlan/ReadFromRemote.cpp | 3 ++- src/QueryPipeline/RemoteQueryExecutor.cpp | 5 +++-- src/Server/TCPHandler.cpp | 1 + .../DistributedAsyncInsertBatch.cpp | 11 +++++++++-- .../DistributedAsyncInsertDirectoryQueue.cpp | 12 ++++++++---- .../DistributedAsyncInsertDirectoryQueue.h | 6 +++--- src/Storages/Distributed/DistributedSink.cpp | 8 ++++---- src/Storages/Distributed/DistributedSink.h | 2 -- src/Storages/StorageDistributed.cpp | 13 ++++--------- src/Storages/StorageDistributed.h | 3 ++- 20 files changed, 91 insertions(+), 49 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index c43aa6d8087..dd87c5a2f9c 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -8,6 +8,7 @@ namespace ProfileEvents extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; + extern const Event DistributedConnectionReadOnlyReplica; } namespace DB @@ -25,9 +26,10 @@ ConnectionEstablisher::ConnectionEstablisher( ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, + bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_) - : pool(std::move(pool_)), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_) + : pool(std::move(pool_)), timeouts(timeouts_), settings(settings_), insert(insert_), log(log_), table_to_check(table_to_check_) { } @@ -70,6 +72,16 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; + bool is_readonly = table_status_it->second.is_readonly; + if (insert && is_readonly) + { + result.is_up_to_date = false; + + LOG_TRACE(log, "Table {}.{} is readonly on server {}", table_to_check->database, table_to_check->table, result.entry->getDescription()); + ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); + return; + } + const UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; if (!max_allowed_delay) { @@ -87,6 +99,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); + return; } } catch (const Exception & e) @@ -111,10 +124,11 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync( ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, + bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_) : AsyncTaskExecutor(std::make_unique(*this)) - , connection_establisher(std::move(pool_), timeouts_, settings_, log_, table_to_check_) + , connection_establisher(std::move(pool_), timeouts_, settings_, insert_, log_, table_to_check_) { epoll.add(timeout_descriptor.getDescriptor()); } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index a3a01e63246..e2194c76c70 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -20,6 +20,7 @@ public: ConnectionEstablisher(ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, + bool insert, LoggerPtr log, const QualifiedTableName * table_to_check = nullptr); @@ -33,6 +34,7 @@ private: ConnectionPoolPtr pool; const ConnectionTimeouts * timeouts; const Settings & settings; + bool insert; LoggerPtr log; const QualifiedTableName * table_to_check; @@ -55,6 +57,7 @@ public: ConnectionEstablisherAsync(ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, + bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_ = nullptr); diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 492fd4ae9e2..ce0f480764d 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -52,7 +52,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts settings.distributed_replica_max_ignored_errors = 0; settings.fallback_to_stale_replicas_for_distributed_queries = true; - return get(timeouts, settings, true); + return get(timeouts, settings, /* force_connected= */ true); } IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts & timeouts, @@ -65,7 +65,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { - return tryGetEntry(pool, timeouts, fail_message, settings, {}); + return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false, {}); }; const size_t offset = settings.load_balancing_first_offset % nested_pools.size(); @@ -119,7 +119,7 @@ std::vector ConnectionPoolWithFailover::getMany( GetPriorityForLoadBalancing::Func priority_func) { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) - { return tryGetEntry(pool, timeouts, fail_message, settings, nullptr, async_callback); }; + { return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false, nullptr, async_callback); }; std::vector results = getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); @@ -137,7 +137,7 @@ std::vector ConnectionPoolWithFailover::g { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { - return tryGetEntry(pool, timeouts, fail_message, settings); + return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false); }; return getManyImpl(settings, pool_mode, try_get_entry); @@ -148,12 +148,13 @@ std::vector ConnectionPoolWithFailover::g const Settings & settings, PoolMode pool_mode, const QualifiedTableName & table_to_check, + bool insert, AsyncCallback async_callback, std::optional skip_unavailable_endpoints, GetPriorityForLoadBalancing::Func priority_func) { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) - { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, async_callback); }; + { return tryGetEntry(pool, timeouts, fail_message, settings, insert, &table_to_check, async_callback); }; return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); } @@ -212,13 +213,14 @@ ConnectionPoolWithFailover::tryGetEntry( const ConnectionTimeouts & timeouts, std::string & fail_message, const Settings & settings, + bool insert, const QualifiedTableName * table_to_check, [[maybe_unused]] AsyncCallback async_callback) { #if defined(OS_LINUX) if (async_callback) { - ConnectionEstablisherAsync connection_establisher_async(pool, &timeouts, settings, log, table_to_check); + ConnectionEstablisherAsync connection_establisher_async(pool, &timeouts, settings, insert, log, table_to_check); while (true) { connection_establisher_async.resume(); @@ -238,7 +240,7 @@ ConnectionPoolWithFailover::tryGetEntry( } #endif - ConnectionEstablisher connection_establisher(pool, &timeouts, settings, log, table_to_check); + ConnectionEstablisher connection_establisher(pool, &timeouts, settings, insert, log, table_to_check); TryResult result; connection_establisher.run(result, fail_message); return result; diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index edfcbe6e4df..2dd4b760395 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -62,18 +62,21 @@ public: /// The same as getMany(), but return std::vector. std::vector getManyForTableFunction(const ConnectionTimeouts & timeouts, - const Settings & settings, PoolMode pool_mode); + const Settings & settings, + PoolMode pool_mode); using Base = PoolWithFailoverBase; using TryResult = Base::TryResult; - /// The same as getMany(), but check that replication delay for table_to_check is acceptable. - /// Delay threshold is taken from settings. + /// The same as getMany(), but check that: + /// - replication delay for table_to_check is acceptable (delay threshold is taken from settings) + /// - replica is not read only in case of @insert == true std::vector getManyChecked( const ConnectionTimeouts & timeouts, const Settings & settings, PoolMode pool_mode, const QualifiedTableName & table_to_check, + bool insert, AsyncCallback async_callback = {}, std::optional skip_unavailable_endpoints = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -117,6 +120,7 @@ private: const ConnectionTimeouts & timeouts, std::string & fail_message, const Settings & settings, + bool insert, const QualifiedTableName * table_to_check = nullptr, AsyncCallback async_callback = {}); diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index f5b074a0257..b9ce31d1efd 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -44,7 +44,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( for (const auto & shuffled_pool : shuffled_pools) replicas.emplace_back( - std::make_unique(shuffled_pool.pool, &timeouts, settings_, log, table_to_check.get())); + std::make_unique(shuffled_pool.pool, &timeouts, settings_, /* insert= */ false, log, table_to_check.get())); } HedgedConnectionsFactory::~HedgedConnectionsFactory() diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 740103e53bc..46c414d34c1 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -81,6 +81,7 @@ public: bool is_up_to_date = false; /// If true, the entry is a connection to up-to-date replica /// Depends on max_replica_delay_for_distributed_queries setting UInt32 delay = 0; /// Helps choosing the "least stale" option when all replicas are stale. + bool is_readonly = false; /// Table is in read-only mode, INSERT can ignore such replicas. }; struct PoolState; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index e91b5adec87..b27c657aeaa 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -156,6 +156,7 @@ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \ M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \ M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \ + M(DistributedConnectionReadOnlyReplica, "Number of times we got a read-only replica for distributed INSERT query.") \ M(DistributedConnectionFailAtAll, "Total count when distributed connection fails after all retries finished.") \ \ M(HedgedRequestsChangeReplica, "Total count when timeout for changing replica expired in hedged requests.") \ diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 058c6fdc903..159a4c28b6d 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -76,6 +76,9 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION = 54465; static constexpr auto DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION = 54466; +/// Send read-only flag for Replicated tables as well +static constexpr auto DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK = 54467; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -83,6 +86,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION = 54466; /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54466; +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54467; } diff --git a/src/Interpreters/TablesStatus.cpp b/src/Interpreters/TablesStatus.cpp index 911a028f813..a830ac16de6 100644 --- a/src/Interpreters/TablesStatus.cpp +++ b/src/Interpreters/TablesStatus.cpp @@ -13,22 +13,26 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void TableStatus::write(WriteBuffer & out) const +void TableStatus::write(WriteBuffer & out, UInt64 client_protocol_revision) const { writeBinary(is_replicated, out); if (is_replicated) { writeVarUInt(absolute_delay, out); + if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK) + writeVarUInt(is_readonly, out); } } -void TableStatus::read(ReadBuffer & in) +void TableStatus::read(ReadBuffer & in, UInt64 server_protocol_revision) { absolute_delay = 0; readBinary(is_replicated, in); if (is_replicated) { readVarUInt(absolute_delay, in); + if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_TABLE_READ_ONLY_CHECK) + readVarUInt(is_readonly, in); } } @@ -71,14 +75,14 @@ void TablesStatusResponse::write(WriteBuffer & out, UInt64 client_protocol_revis throw Exception(ErrorCodes::LOGICAL_ERROR, "method TablesStatusResponse::write is called for unsupported client revision"); writeVarUInt(table_states_by_id.size(), out); - for (const auto & kv: table_states_by_id) + for (const auto & kv : table_states_by_id) { const QualifiedTableName & table_name = kv.first; writeBinary(table_name.database, out); writeBinary(table_name.table, out); const TableStatus & status = kv.second; - status.write(out); + status.write(out, client_protocol_revision); } } @@ -100,7 +104,7 @@ void TablesStatusResponse::read(ReadBuffer & in, UInt64 server_protocol_revision readBinary(table_name.table, in); TableStatus status; - status.read(in); + status.read(in, server_protocol_revision); table_states_by_id.emplace(std::move(table_name), std::move(status)); } } diff --git a/src/Interpreters/TablesStatus.h b/src/Interpreters/TablesStatus.h index 2323e751fc9..ddd0a721701 100644 --- a/src/Interpreters/TablesStatus.h +++ b/src/Interpreters/TablesStatus.h @@ -28,9 +28,11 @@ struct TableStatus { bool is_replicated = false; UInt32 absolute_delay = 0; + /// Used to filter such nodes out for INSERTs + bool is_readonly = false; - void write(WriteBuffer & out) const; - void read(ReadBuffer & in); + void write(WriteBuffer & out, UInt64 client_protocol_revision) const; + void read(ReadBuffer & in, UInt64 server_protocol_revision); }; struct TablesStatusRequest diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 119710d06d8..95e5c2e366a 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -173,7 +173,8 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream else try_results = my_shard.shard_info.pool->getManyChecked( timeouts, current_settings, PoolMode::GET_MANY, - my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName()); + my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName(), + /* insert= */ false); } catch (const Exception & ex) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 1686a101bde..698793fa9f4 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -87,12 +87,12 @@ RemoteQueryExecutor::RemoteQueryExecutor( { auto table_name = main_table.getQualifiedName(); - ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name); + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, /* insert= */ false, log, &table_name); connection_establisher.run(result, fail_message); } else { - ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr); + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, /* insert= */ false, log, nullptr); connection_establisher.run(result, fail_message); } @@ -221,6 +221,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( current_settings, pool_mode, main_table.getQualifiedName(), + /* insert= */ false, std::move(async_callback), skip_unavailable_endpoints, priority_func); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 26d2390f1df..5c08c697434 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1124,6 +1124,7 @@ void TCPHandler::processTablesStatusRequest() { status.is_replicated = true; status.absolute_delay = static_cast(replicated_table->getAbsoluteDelay()); + status.is_readonly = replicated_table->isTableReadOnly(); } else status.is_replicated = false; diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 8d95e49de57..94e7b6a2647 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } /// Can the batch be split and send files from batch one-by-one instead? @@ -232,7 +233,10 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - connection = parent.pool->get(timeouts); + auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName(), /* insert= */ true); + if (result.empty() || result.front().entry.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + connection = std::move(result.front().entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", @@ -289,7 +293,10 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett parent.storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto connection = parent.pool->get(timeouts); + auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName(), /* insert= */ true); + if (result.empty() || result.front().entry.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + auto connection = std::move(result.front().entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; RemoteInserter remote(*connection, timeouts, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 7fed076713d..9f95018c09e 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -101,7 +101,7 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( StorageDistributed & storage_, const DiskPtr & disk_, const std::string & relative_path_, - ConnectionPoolPtr pool_, + ConnectionPoolWithFailoverPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool) : storage(storage_) @@ -237,7 +237,7 @@ void DistributedAsyncInsertDirectoryQueue::run() } -ConnectionPoolPtr DistributedAsyncInsertDirectoryQueue::createPool(const Cluster::Addresses & addresses, const StorageDistributed & storage) +ConnectionPoolWithFailoverPtr DistributedAsyncInsertDirectoryQueue::createPool(const Cluster::Addresses & addresses, const StorageDistributed & storage) { const auto pool_factory = [&storage] (const Cluster::Address & address) -> ConnectionPoolPtr { @@ -284,7 +284,7 @@ ConnectionPoolPtr DistributedAsyncInsertDirectoryQueue::createPool(const Cluster auto pools = createPoolsForAddresses(addresses, pool_factory, storage.log); const auto settings = storage.getContext()->getSettings(); - return pools.size() == 1 ? pools.front() : std::make_shared(pools, + return std::make_shared(std::move(pools), settings.load_balancing, settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap); @@ -412,7 +412,11 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto connection = pool->get(timeouts, insert_settings); + auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName(), /* insert= */ true); + if (result.empty() || result.front().entry.isNull()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + auto connection = std::move(result.front().entry); + LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", file_path, connection->getDescription(), diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index 4d6afe31d61..ba4d264f967 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -50,13 +50,13 @@ public: StorageDistributed & storage_, const DiskPtr & disk_, const std::string & relative_path_, - ConnectionPoolPtr pool_, + ConnectionPoolWithFailoverPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool); ~DistributedAsyncInsertDirectoryQueue(); - static ConnectionPoolPtr createPool(const Cluster::Addresses & addresses, const StorageDistributed & storage); + static ConnectionPoolWithFailoverPtr createPool(const Cluster::Addresses & addresses, const StorageDistributed & storage); void updatePath(const std::string & new_relative_path); @@ -111,7 +111,7 @@ private: std::string getLoggerName() const; StorageDistributed & storage; - const ConnectionPoolPtr pool; + const ConnectionPoolWithFailoverPtr pool; DiskPtr disk; std::string relative_path; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 1efa98d0c13..d19cd4a7ba7 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -112,19 +112,17 @@ DistributedSink::DistributedSink( const ClusterPtr & cluster_, bool insert_sync_, UInt64 insert_timeout_, - StorageID main_table_, const Names & columns_to_send_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , context(Context::createCopy(context_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) - , query_ast(createInsertToRemoteTableQuery(main_table_.database_name, main_table_.table_name, columns_to_send_)) + , query_ast(createInsertToRemoteTableQuery(storage.remote_storage.database_name, storage.remote_storage.table_name, columns_to_send_)) , query_string(queryToString(query_ast)) , cluster(cluster_) , insert_sync(insert_sync_) , allow_materialized(context->getSettingsRef().insert_allow_materialized_columns) , insert_timeout(insert_timeout_) - , main_table(main_table_) , columns_to_send(columns_to_send_.begin(), columns_to_send_.end()) , log(getLogger("DistributedSink")) { @@ -372,7 +370,9 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si throw Exception(ErrorCodes::LOGICAL_ERROR, "There are several writing job for an automatically replicated shard"); /// TODO: it make sense to rewrite skip_unavailable_shards and max_parallel_replicas here - auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, main_table.getQualifiedName()); + /// NOTE: INSERT will also take into account max_replica_delay_for_distributed_queries + /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) + auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName(), /* insert= */ true); if (results.empty() || results.front().entry.isNull()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection for shard {}", toString(job.shard_index)); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 654c1db354f..7a9e89c9e94 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -46,7 +46,6 @@ public: const ClusterPtr & cluster_, bool insert_sync_, UInt64 insert_timeout_, - StorageID main_table_, const Names & columns_to_send_); String getName() const override { return "DistributedSink"; } @@ -108,7 +107,6 @@ private: /// Sync-related stuff UInt64 insert_timeout; // in seconds - StorageID main_table; NameSet columns_to_send; Stopwatch watch; Stopwatch watch_current_block; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1ee7c6fc6a5..12c2ad331ad 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -334,6 +334,7 @@ StorageDistributed::StorageDistributed( , remote_database(remote_database_) , remote_table(remote_table_) , remote_table_function_ptr(remote_table_function_ptr_) + , remote_storage(remote_table_function_ptr ? StorageID::createEmpty() : StorageID{remote_database, remote_table}) , log(getLogger("StorageDistributed (" + id_.table_name + ")")) , owned_cluster(std::move(owned_cluster_)) , cluster_name(getContext()->getMacros()->expand(cluster_name_)) @@ -896,10 +897,6 @@ void StorageDistributed::read( return; } - StorageID main_table = StorageID::createEmpty(); - if (!remote_table_function_ptr) - main_table = StorageID{remote_database, remote_table}; - const auto & snapshot_data = assert_cast(*storage_snapshot->data); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( @@ -932,7 +929,7 @@ void StorageDistributed::read( query_plan, header, processed_stage, - main_table, + remote_storage, remote_table_function_ptr, select_stream_factory, log, @@ -978,10 +975,8 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata else columns_to_send = metadata_snapshot->getSampleBlockNonMaterialized().getNames(); - /// DistributedSink will not own cluster - return std::make_shared( - local_context, *this, metadata_snapshot, cluster, insert_sync, timeout, - StorageID{remote_database, remote_table}, columns_to_send); + /// DistributedSink will not own cluster, but will own ConnectionPools of the cluster + return std::make_shared(local_context, *this, metadata_snapshot, cluster, insert_sync, timeout, columns_to_send); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 6709b1a2d8c..3a7e63aef50 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -241,6 +241,7 @@ private: String remote_database; String remote_table; ASTPtr remote_table_function_ptr; + StorageID remote_storage; LoggerPtr log; @@ -275,7 +276,7 @@ private: struct ClusterNodeData { std::shared_ptr directory_queue; - ConnectionPoolPtr connection_pool; + ConnectionPoolWithFailoverPtr connection_pool; Cluster::Addresses addresses; size_t clusters_version; }; From 5b872aba7bcb2e35104e4d2c116bcf298e40a06e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 21:41:30 +0100 Subject: [PATCH 107/154] tests: adjust 02537_distributed_loosing_files_after_exception Now, since there is connection probing, the error changed from UNKNOWN_TABLE to ALL_CONNECTION_TRIES_FAILED, though I'm not sure that this good... Signed-off-by: Azat Khuzhin --- .../02537_distributed_loosing_files_after_exception.sql.j2 | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 index e7f7126d6bf..3eabbfb82e7 100644 --- a/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 +++ b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 @@ -12,9 +12,9 @@ SYSTEM STOP DISTRIBUTED SENDS dist; INSERT INTO dist SETTINGS prefer_localhost_replica=0, max_threads=1 VALUES (1); INSERT INTO dist SETTINGS prefer_localhost_replica=0, max_threads=2 VALUES (1); -SYSTEM FLUSH DISTRIBUTED dist; -- { serverError UNKNOWN_TABLE } +SYSTEM FLUSH DISTRIBUTED dist; -- { serverError ALL_CONNECTION_TRIES_FAILED } -- check the second since after using queue it may got lost from it -SYSTEM FLUSH DISTRIBUTED dist; -- { serverError UNKNOWN_TABLE } +SYSTEM FLUSH DISTRIBUTED dist; -- { serverError ALL_CONNECTION_TRIES_FAILED } SELECT is_blocked, data_files FROM system.distribution_queue WHERE database = currentDatabase() AND table = 'dist'; From a3c3bff5df5d7862a18d7e151ca86383892f77fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Jan 2024 11:43:05 +0100 Subject: [PATCH 108/154] tests: cover filtering out read-only replicas during INSERT into Distributed v2: replace without s3_plain disk v3: add check for DistributedConnectionReadOnlyReplica v4: set insert_keeper_max_retries=0 v5: do not set insert_keeper_max_retries=0 Signed-off-by: Azat Khuzhin --- tests/config/config.d/clusters.xml | 15 +++++ ...980_dist_insert_readonly_replica.reference | 3 + .../02980_dist_insert_readonly_replica.sql.j2 | 66 +++++++++++++++++++ 3 files changed, 84 insertions(+) create mode 100644 tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference create mode 100644 tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index 9490f98d6d8..af26565d7e6 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -67,6 +67,21 @@ + + + true + + shard_0 + localhost + 9000 + + + shard_1 + localhost + 9000 + + + 123457 diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference new file mode 100644 index 00000000000..6b27a3b185c --- /dev/null +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference @@ -0,0 +1,3 @@ +shard_0.data 0 +shard_1.data 1 +DistributedConnectionReadOnlyReplica 1 1 diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 new file mode 100644 index 00000000000..2c6d53d88fb --- /dev/null +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -0,0 +1,66 @@ +-- Tags: no-parallel, no-fasttest +-- Tag no-parallel - due to static databases +-- Tag no-fasttest - S3 is required + +drop database if exists shard_0; +drop database if exists shard_1; +create database shard_0; +create database shard_1; + +drop table if exists dist; +drop table if exists dist_batch; +drop table if exists shard_0.data; +drop table if exists shard_1.data; + +-- normal replica +create table shard_0.data (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'write') order by key; +-- create read-only replica +create table shard_1.data (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'read') order by key; + +-- broke replica and leave it read-only +detach table shard_1.data; +insert into system.zookeeper (path, name, value) values ('/tables/shard_1/data/replicas/read', 'columns', ''); +attach table shard_1.data; +select database||'.'||table, is_readonly from system.replicas where database in ('shard_0', 'shard_1') and table = 'data'; + +create table dist as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', '', 'data'); +system stop distributed sends dist; + +create table dist_batch as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', '', 'data') settings background_insert_batch=1; +system stop distributed sends dist_batch; + +set prefer_localhost_replica=0; +set insert_deduplicate=0; + +-- min_insert_block_size_rows is not enough, since replica will be selected +-- before, so we need to perform INSERT multiple times to ensure that at least +-- once read-only replica should be selected (if it wasn't not filtered out) +{% for i in range(1, 30) %} +insert into dist settings distributed_foreground_insert=1 values ({{ i }}); +{% endfor %} +-- cannot check for background inserts, so only here +system flush logs; +select + 'DistributedConnectionReadOnlyReplica', count() > 0, sum(ProfileEvents['DistributedConnectionReadOnlyReplica']) > 0 +from system.query_log +where + event_date >= yesterday() + and current_database = currentDatabase() + and query_kind = 'Insert' + and type != 'QueryStart' + and Settings['distributed_foreground_insert'] = '1'; + +{% for i in range(1, 30) %} +insert into dist settings distributed_foreground_insert=0 values ({{ i }}); +system flush distributed dist; +{% endfor %} + +{% for i in range(1, 30) %} +insert into dist_batch settings distributed_foreground_insert=0 values ({{ i }}); +system flush distributed dist_batch; +{% endfor %} + +drop database shard_0; +drop database shard_1; + +-- vim: ft=sql From 0e99781686183859f2ac4345cfa8dbe79f2b3186 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Feb 2024 22:54:53 +0100 Subject: [PATCH 109/154] Move insert flag from ConnectionEstablisher to TryResult Signed-off-by: Azat Khuzhin --- src/Client/ConnectionEstablisher.cpp | 13 +++----- src/Client/ConnectionEstablisher.h | 3 -- src/Client/ConnectionPoolWithFailover.cpp | 22 ++++++------- src/Client/ConnectionPoolWithFailover.h | 2 +- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Common/PoolWithFailoverBase.h | 38 +++++++++++++++++------ src/QueryPipeline/RemoteQueryExecutor.cpp | 4 +-- 7 files changed, 47 insertions(+), 37 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index dd87c5a2f9c..853fea90f23 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -26,10 +26,9 @@ ConnectionEstablisher::ConnectionEstablisher( ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_) - : pool(std::move(pool_)), timeouts(timeouts_), settings(settings_), insert(insert_), log(log_), table_to_check(table_to_check_) + : pool(std::move(pool_)), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_) { } @@ -72,14 +71,11 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: ProfileEvents::increment(ProfileEvents::DistributedConnectionUsable); result.is_usable = true; - bool is_readonly = table_status_it->second.is_readonly; - if (insert && is_readonly) + if (table_status_it->second.is_readonly) { - result.is_up_to_date = false; - + result.is_readonly = true; LOG_TRACE(log, "Table {}.{} is readonly on server {}", table_to_check->database, table_to_check->table, result.entry->getDescription()); ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); - return; } const UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; @@ -124,11 +120,10 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync( ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_) : AsyncTaskExecutor(std::make_unique(*this)) - , connection_establisher(std::move(pool_), timeouts_, settings_, insert_, log_, table_to_check_) + , connection_establisher(std::move(pool_), timeouts_, settings_, log_, table_to_check_) { epoll.add(timeout_descriptor.getDescriptor()); } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index e2194c76c70..a3a01e63246 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -20,7 +20,6 @@ public: ConnectionEstablisher(ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - bool insert, LoggerPtr log, const QualifiedTableName * table_to_check = nullptr); @@ -34,7 +33,6 @@ private: ConnectionPoolPtr pool; const ConnectionTimeouts * timeouts; const Settings & settings; - bool insert; LoggerPtr log; const QualifiedTableName * table_to_check; @@ -57,7 +55,6 @@ public: ConnectionEstablisherAsync(ConnectionPoolPtr pool_, const ConnectionTimeouts * timeouts_, const Settings & settings_, - bool insert_, LoggerPtr log_, const QualifiedTableName * table_to_check_ = nullptr); diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index ce0f480764d..0c87c52bf68 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -65,7 +65,7 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { - return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false, {}); + return tryGetEntry(pool, timeouts, fail_message, settings); }; const size_t offset = settings.load_balancing_first_offset % nested_pools.size(); @@ -119,9 +119,9 @@ std::vector ConnectionPoolWithFailover::getMany( GetPriorityForLoadBalancing::Func priority_func) { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) - { return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false, nullptr, async_callback); }; + { return tryGetEntry(pool, timeouts, fail_message, settings, nullptr, async_callback); }; - std::vector results = getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); + std::vector results = getManyImpl(settings, pool_mode, try_get_entry, /* insert= */ false, skip_unavailable_endpoints, priority_func); std::vector entries; entries.reserve(results.size()); @@ -137,10 +137,10 @@ std::vector ConnectionPoolWithFailover::g { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { - return tryGetEntry(pool, timeouts, fail_message, settings, /* insert= */ false); + return tryGetEntry(pool, timeouts, fail_message, settings); }; - return getManyImpl(settings, pool_mode, try_get_entry); + return getManyImpl(settings, pool_mode, try_get_entry, /* insert= */ false); } std::vector ConnectionPoolWithFailover::getManyChecked( @@ -154,9 +154,9 @@ std::vector ConnectionPoolWithFailover::g GetPriorityForLoadBalancing::Func priority_func) { TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) - { return tryGetEntry(pool, timeouts, fail_message, settings, insert, &table_to_check, async_callback); }; + { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, async_callback); }; - return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); + return getManyImpl(settings, pool_mode, try_get_entry, insert, skip_unavailable_endpoints, priority_func); } ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings & settings) @@ -171,6 +171,7 @@ std::vector ConnectionPoolWithFailover::g const Settings & settings, PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, + bool insert, std::optional skip_unavailable_endpoints, GetPriorityForLoadBalancing::Func priority_func) { @@ -204,7 +205,7 @@ std::vector ConnectionPoolWithFailover::g UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value; bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value; - return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func); + return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func, insert); } ConnectionPoolWithFailover::TryResult @@ -213,14 +214,13 @@ ConnectionPoolWithFailover::tryGetEntry( const ConnectionTimeouts & timeouts, std::string & fail_message, const Settings & settings, - bool insert, const QualifiedTableName * table_to_check, [[maybe_unused]] AsyncCallback async_callback) { #if defined(OS_LINUX) if (async_callback) { - ConnectionEstablisherAsync connection_establisher_async(pool, &timeouts, settings, insert, log, table_to_check); + ConnectionEstablisherAsync connection_establisher_async(pool, &timeouts, settings, log, table_to_check); while (true) { connection_establisher_async.resume(); @@ -240,7 +240,7 @@ ConnectionPoolWithFailover::tryGetEntry( } #endif - ConnectionEstablisher connection_establisher(pool, &timeouts, settings, insert, log, table_to_check); + ConnectionEstablisher connection_establisher(pool, &timeouts, settings, log, table_to_check); TryResult result; connection_establisher.run(result, fail_message); return result; diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 2dd4b760395..84bc488ab3f 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -109,6 +109,7 @@ private: const Settings & settings, PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, + bool insert, std::optional skip_unavailable_endpoints = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -120,7 +121,6 @@ private: const ConnectionTimeouts & timeouts, std::string & fail_message, const Settings & settings, - bool insert, const QualifiedTableName * table_to_check = nullptr, AsyncCallback async_callback = {}); diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index b9ce31d1efd..f5b074a0257 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -44,7 +44,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( for (const auto & shuffled_pool : shuffled_pools) replicas.emplace_back( - std::make_unique(shuffled_pool.pool, &timeouts, settings_, /* insert= */ false, log, table_to_check.get())); + std::make_unique(shuffled_pool.pool, &timeouts, settings_, log, table_to_check.get())); } HedgedConnectionsFactory::~HedgedConnectionsFactory() diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 46c414d34c1..51ff80be3be 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -108,12 +108,14 @@ public: /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. + /// If insert is true then it will take into account replica read-only flag (non-read-only replicas will be preferred) std::vector getMany( size_t min_entries, size_t max_entries, size_t max_tries, size_t max_ignored_errors, bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority); + const GetPriorityFunc & get_priority, + bool insert); size_t getPoolSize() const { return nested_pools.size(); } @@ -202,7 +204,8 @@ PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_ std::vector results = getMany( 1 /* min entries */, 1 /* max entries */, 1 /* max tries */, max_ignored_errors, fallback_to_stale_replicas, - try_get_entry, get_priority); + try_get_entry, get_priority, + /* insert= */ false); if (results.empty() || results[0].entry.isNull()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "PoolWithFailoverBase::getMany() returned less than min_entries entries."); @@ -216,7 +219,8 @@ PoolWithFailoverBase::getMany( size_t max_ignored_errors, bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority) + const GetPriorityFunc & get_priority, + bool insert) { std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); @@ -294,13 +298,27 @@ PoolWithFailoverBase::getMany( std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); /// Sort so that preferred items are near the beginning. - std::stable_sort( - try_results.begin(), try_results.end(), - [](const TryResult & left, const TryResult & right) - { - return std::forward_as_tuple(!left.is_up_to_date, left.delay) - < std::forward_as_tuple(!right.is_up_to_date, right.delay); - }); + if (insert) + { + /// In case of connections had been requested for INSERT we take into + /// account replica read-only flag. Note, that they are not filtered out, + /// since they may be still non-readonly when the INSERT will be performed. + auto results_comparator = [&](const TryResult & left, const TryResult & right) + { + return std::forward_as_tuple(left.is_readonly, !left.is_up_to_date, left.delay) + < std::forward_as_tuple(right.is_readonly, !right.is_up_to_date, right.delay); + }; + std::stable_sort(try_results.begin(), try_results.end(), results_comparator); + } + else + { + auto results_comparator = [&](const TryResult & left, const TryResult & right) + { + return std::forward_as_tuple(!left.is_up_to_date, left.delay) + < std::forward_as_tuple(!right.is_up_to_date, right.delay); + }; + std::stable_sort(try_results.begin(), try_results.end(), results_comparator); + } if (fallback_to_stale_replicas) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 698793fa9f4..1a3ae8da02a 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -87,12 +87,12 @@ RemoteQueryExecutor::RemoteQueryExecutor( { auto table_name = main_table.getQualifiedName(); - ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, /* insert= */ false, log, &table_name); + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name); connection_establisher.run(result, fail_message); } else { - ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, /* insert= */ false, log, nullptr); + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr); connection_establisher.run(result, fail_message); } From d922bfb1b0d0bc7df9f52e25fb8ae475d11bb2c2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 16:58:58 +0100 Subject: [PATCH 110/154] Introduce distributed_insert_prefer_non_readonly_replica setting Signed-off-by: Azat Khuzhin --- src/Client/ConnectionPoolWithFailover.cpp | 3 ++- src/Common/PoolWithFailoverBase.h | 10 +++++----- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 0c87c52bf68..f9356e28cc8 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -205,7 +205,8 @@ std::vector ConnectionPoolWithFailover::g UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value; bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value; - return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func, insert); + bool prefer_non_read_only_replica = insert && settings.distributed_insert_prefer_non_readonly_replica; + return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func, prefer_non_read_only_replica); } ConnectionPoolWithFailover::TryResult diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 51ff80be3be..7a9edc0bdce 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -108,14 +108,14 @@ public: /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. - /// If insert is true then it will take into account replica read-only flag (non-read-only replicas will be preferred) + /// If prefer_non_read_only_replica is true then it will take into account replica read-only flag (non-read-only replicas will be preferred) std::vector getMany( size_t min_entries, size_t max_entries, size_t max_tries, size_t max_ignored_errors, bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority, - bool insert); + bool prefer_non_read_only_replica); size_t getPoolSize() const { return nested_pools.size(); } @@ -205,7 +205,7 @@ PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_ 1 /* min entries */, 1 /* max entries */, 1 /* max tries */, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, get_priority, - /* insert= */ false); + /* prefer_non_read_only_replica= */ false); if (results.empty() || results[0].entry.isNull()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "PoolWithFailoverBase::getMany() returned less than min_entries entries."); @@ -220,7 +220,7 @@ PoolWithFailoverBase::getMany( bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority, - bool insert) + bool prefer_non_read_only_replica) { std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); @@ -298,7 +298,7 @@ PoolWithFailoverBase::getMany( std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); /// Sort so that preferred items are near the beginning. - if (insert) + if (prefer_non_read_only_replica) { /// In case of connections had been requested for INSERT we take into /// account replica read-only flag. Note, that they are not filtered out, diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 170fd4e9ca0..3d0d28f59c0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -136,6 +136,7 @@ class IColumn; M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ \ + M(Bool, distributed_insert_prefer_non_readonly_replica, true, "If setting is enabled, INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely", 0) \ M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \ M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \ M(Milliseconds, distributed_background_insert_sleep_time_ms, 100, "Sleep time for background INSERTs into Distributed, in case of any errors delay grows exponentially.", 0) ALIAS(distributed_directory_monitor_sleep_time_ms) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6b31e9cd249..28e42559e2a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -136,6 +136,7 @@ static std::map sett {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, + {"distributed_insert_prefer_non_readonly_replica", false, true, "If setting is enabled, INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely"}, {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, From dbdf08010848ed7e79e040fb9db540416f7dfbc0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:13:02 +0100 Subject: [PATCH 111/154] Remove insert argument for ConnectionPoolWithFailover Signed-off-by: Azat Khuzhin --- src/Client/ConnectionPoolWithFailover.cpp | 21 +++++++---- src/Client/ConnectionPoolWithFailover.h | 5 ++- src/Common/PoolWithFailoverBase.h | 36 +++++-------------- src/Processors/QueryPlan/ReadFromRemote.cpp | 3 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 1 - .../DistributedAsyncInsertBatch.cpp | 10 ++++-- .../DistributedAsyncInsertDirectoryQueue.cpp | 6 +++- src/Storages/Distributed/DistributedSink.cpp | 5 ++- 8 files changed, 42 insertions(+), 45 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index f9356e28cc8..dddb3f45ba3 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -121,7 +121,7 @@ std::vector ConnectionPoolWithFailover::getMany( TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings, nullptr, async_callback); }; - std::vector results = getManyImpl(settings, pool_mode, try_get_entry, /* insert= */ false, skip_unavailable_endpoints, priority_func); + std::vector results = getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); std::vector entries; entries.reserve(results.size()); @@ -140,7 +140,7 @@ std::vector ConnectionPoolWithFailover::g return tryGetEntry(pool, timeouts, fail_message, settings); }; - return getManyImpl(settings, pool_mode, try_get_entry, /* insert= */ false); + return getManyImpl(settings, pool_mode, try_get_entry); } std::vector ConnectionPoolWithFailover::getManyChecked( @@ -148,7 +148,6 @@ std::vector ConnectionPoolWithFailover::g const Settings & settings, PoolMode pool_mode, const QualifiedTableName & table_to_check, - bool insert, AsyncCallback async_callback, std::optional skip_unavailable_endpoints, GetPriorityForLoadBalancing::Func priority_func) @@ -156,7 +155,7 @@ std::vector ConnectionPoolWithFailover::g TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, async_callback); }; - return getManyImpl(settings, pool_mode, try_get_entry, insert, skip_unavailable_endpoints, priority_func); + return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); } ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings & settings) @@ -171,7 +170,6 @@ std::vector ConnectionPoolWithFailover::g const Settings & settings, PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, - bool insert, std::optional skip_unavailable_endpoints, GetPriorityForLoadBalancing::Func priority_func) { @@ -205,8 +203,7 @@ std::vector ConnectionPoolWithFailover::g UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value; bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value; - bool prefer_non_read_only_replica = insert && settings.distributed_insert_prefer_non_readonly_replica; - return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func, prefer_non_read_only_replica); + return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func); } ConnectionPoolWithFailover::TryResult @@ -257,4 +254,14 @@ ConnectionPoolWithFailover::getShuffledPools(const Settings & settings, GetPrior return Base::getShuffledPools(max_ignored_errors, priority_func, use_slowdown_count); } +void sortConnectionPoolByNonReadOnlyReplicas(std::vector & results) +{ + auto comparator = [&](const auto & left, const auto & right) + { + return std::forward_as_tuple(left.is_readonly, !left.is_up_to_date, left.delay) + < std::forward_as_tuple(right.is_readonly, !right.is_up_to_date, right.delay); + }; + std::stable_sort(results.begin(), results.end(), comparator); +} + } diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 84bc488ab3f..0f47270e964 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -70,13 +70,11 @@ public: /// The same as getMany(), but check that: /// - replication delay for table_to_check is acceptable (delay threshold is taken from settings) - /// - replica is not read only in case of @insert == true std::vector getManyChecked( const ConnectionTimeouts & timeouts, const Settings & settings, PoolMode pool_mode, const QualifiedTableName & table_to_check, - bool insert, AsyncCallback async_callback = {}, std::optional skip_unavailable_endpoints = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -109,7 +107,6 @@ private: const Settings & settings, PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, - bool insert, std::optional skip_unavailable_endpoints = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -129,6 +126,8 @@ private: GetPriorityForLoadBalancing get_priority_load_balancing; }; +void sortConnectionPoolByNonReadOnlyReplicas(std::vector & results); + using ConnectionPoolWithFailoverPtr = std::shared_ptr; using ConnectionPoolWithFailoverPtrs = std::vector; diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 7a9edc0bdce..ccb2354c312 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -108,14 +108,12 @@ public: /// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool). /// The method will throw if it is unable to get min_entries alive connections or /// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas. - /// If prefer_non_read_only_replica is true then it will take into account replica read-only flag (non-read-only replicas will be preferred) std::vector getMany( size_t min_entries, size_t max_entries, size_t max_tries, size_t max_ignored_errors, bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority, - bool prefer_non_read_only_replica); + const GetPriorityFunc & get_priority); size_t getPoolSize() const { return nested_pools.size(); } @@ -204,8 +202,7 @@ PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_ std::vector results = getMany( 1 /* min entries */, 1 /* max entries */, 1 /* max tries */, max_ignored_errors, fallback_to_stale_replicas, - try_get_entry, get_priority, - /* prefer_non_read_only_replica= */ false); + try_get_entry, get_priority); if (results.empty() || results[0].entry.isNull()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "PoolWithFailoverBase::getMany() returned less than min_entries entries."); @@ -219,8 +216,7 @@ PoolWithFailoverBase::getMany( size_t max_ignored_errors, bool fallback_to_stale_replicas, const TryGetEntryFunc & try_get_entry, - const GetPriorityFunc & get_priority, - bool prefer_non_read_only_replica) + const GetPriorityFunc & get_priority) { std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); @@ -297,28 +293,12 @@ PoolWithFailoverBase::getMany( std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); - /// Sort so that preferred items are near the beginning. - if (prefer_non_read_only_replica) + auto comparator = [&](const TryResult & left, const TryResult & right) { - /// In case of connections had been requested for INSERT we take into - /// account replica read-only flag. Note, that they are not filtered out, - /// since they may be still non-readonly when the INSERT will be performed. - auto results_comparator = [&](const TryResult & left, const TryResult & right) - { - return std::forward_as_tuple(left.is_readonly, !left.is_up_to_date, left.delay) - < std::forward_as_tuple(right.is_readonly, !right.is_up_to_date, right.delay); - }; - std::stable_sort(try_results.begin(), try_results.end(), results_comparator); - } - else - { - auto results_comparator = [&](const TryResult & left, const TryResult & right) - { - return std::forward_as_tuple(!left.is_up_to_date, left.delay) - < std::forward_as_tuple(!right.is_up_to_date, right.delay); - }; - std::stable_sort(try_results.begin(), try_results.end(), results_comparator); - } + return std::forward_as_tuple(!left.is_up_to_date, left.delay) + < std::forward_as_tuple(!right.is_up_to_date, right.delay); + }; + std::stable_sort(try_results.begin(), try_results.end(), comparator); if (fallback_to_stale_replicas) { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 95e5c2e366a..119710d06d8 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -173,8 +173,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream else try_results = my_shard.shard_info.pool->getManyChecked( timeouts, current_settings, PoolMode::GET_MANY, - my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName(), - /* insert= */ false); + my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName()); } catch (const Exception & ex) { diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 1a3ae8da02a..1686a101bde 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -221,7 +221,6 @@ RemoteQueryExecutor::RemoteQueryExecutor( current_settings, pool_mode, main_table.getQualifiedName(), - /* insert= */ false, std::move(async_callback), skip_unavailable_endpoints, priority_func); diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 94e7b6a2647..0b258654f4a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -233,9 +233,12 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName(), /* insert= */ true); + auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); if (result.empty() || result.front().entry.isNull()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + sortConnectionPoolByNonReadOnlyReplicas(result); + connection = std::move(result.front().entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; @@ -293,9 +296,12 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett parent.storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName(), /* insert= */ true); + auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); if (result.empty() || result.front().entry.isNull()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + sortConnectionPoolByNonReadOnlyReplicas(result); + auto connection = std::move(result.front().entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 9f95018c09e..27fbace0482 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -412,9 +412,13 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName(), /* insert= */ true); + auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); if (result.empty() || result.front().entry.isNull()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); + + if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + sortConnectionPoolByNonReadOnlyReplicas(result); + auto connection = std::move(result.front().entry); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index d19cd4a7ba7..4f58b61b0ee 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -372,7 +372,10 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// TODO: it make sense to rewrite skip_unavailable_shards and max_parallel_replicas here /// NOTE: INSERT will also take into account max_replica_delay_for_distributed_queries /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) - auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName(), /* insert= */ true); + auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); + if (settings.distributed_insert_prefer_non_readonly_replica) + sortConnectionPoolByNonReadOnlyReplicas(results); + if (results.empty() || results.front().entry.isNull()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection for shard {}", toString(job.shard_index)); From eb334d717a07203057370a2afcebd169e05d625e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:19:16 +0100 Subject: [PATCH 112/154] Remove superior check for empty pools Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 5 ----- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 3 --- src/Storages/Distributed/DistributedSink.cpp | 4 ---- 3 files changed, 12 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 0b258654f4a..b5a14691756 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; - extern const int LOGICAL_ERROR; } /// Can the batch be split and send files from batch one-by-one instead? @@ -234,8 +233,6 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (result.empty() || result.front().entry.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) sortConnectionPoolByNonReadOnlyReplicas(result); @@ -297,8 +294,6 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (result.empty() || result.front().entry.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) sortConnectionPoolByNonReadOnlyReplicas(result); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 27fbace0482..e041caa2249 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -413,9 +413,6 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); - if (result.empty() || result.front().entry.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection"); - if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) sortConnectionPoolByNonReadOnlyReplicas(result); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 4f58b61b0ee..f41591a52ff 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -375,10 +375,6 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); if (settings.distributed_insert_prefer_non_readonly_replica) sortConnectionPoolByNonReadOnlyReplicas(results); - - if (results.empty() || results.front().entry.isNull()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected exactly one connection for shard {}", toString(job.shard_index)); - job.connection_entry = std::move(results.front().entry); } else From fe613ddd06a6a5e9dac7596e4df33e1bd7d010df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:24:59 +0100 Subject: [PATCH 113/154] Convert distributed_insert_replicas_preferences into enum Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Core/SettingsEnums.cpp | 4 ++++ src/Core/SettingsEnums.h | 8 ++++++++ src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 4 ++-- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/Distributed/DistributedSink.cpp | 2 +- 7 files changed, 18 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3d0d28f59c0..e6d79209f66 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -136,7 +136,7 @@ class IColumn; M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ \ - M(Bool, distributed_insert_prefer_non_readonly_replica, true, "If setting is enabled, INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely", 0) \ + M(ReplicasPreferences, distributed_insert_replicas_preferences, ReplicasPreferences::PREFER_NON_READ_ONLY, "PREFER_NON_READ_ONLY - INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely", 0) \ M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \ M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \ M(Milliseconds, distributed_background_insert_sleep_time_ms, 100, "Sleep time for background INSERTs into Distributed, in case of any errors delay grows exponentially.", 0) ALIAS(distributed_directory_monitor_sleep_time_ms) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 28e42559e2a..3245da584a1 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -136,7 +136,7 @@ static std::map sett {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"distributed_insert_prefer_non_readonly_replica", false, true, "If setting is enabled, INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely"}, + {"distributed_insert_replicas_preferences", "no_preferences", "prefer_non_read_only", "Control preferred replicas for INSERT into Distributed"}, {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index c3f0715ad68..47dadf3c36b 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DistributedProductMode, ErrorCodes::UNKNOWN_DISTRIBUTED_P {"global", DistributedProductMode::GLOBAL}, {"allow", DistributedProductMode::ALLOW}}) +IMPLEMENT_SETTING_ENUM(ReplicasPreferences, ErrorCodes::BAD_ARGUMENTS, + {{"no_preferences", ReplicasPreferences::NO_PREFERENCES}, + {"prefer_non_read_only", ReplicasPreferences::PREFER_NON_READ_ONLY}}) + IMPLEMENT_SETTING_ENUM(QueryCacheNondeterministicFunctionHandling, ErrorCodes::BAD_ARGUMENTS, {{"throw", QueryCacheNondeterministicFunctionHandling::Throw}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 0aa8216bb85..d51271ef16e 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -174,6 +174,14 @@ enum class DistributedProductMode DECLARE_SETTING_ENUM(DistributedProductMode) +enum class ReplicasPreferences +{ + NO_PREFERENCES = 0, + /// Prefer non read-only replicas (they will not be excluded completely). + PREFER_NON_READ_ONLY, +}; +DECLARE_SETTING_ENUM(ReplicasPreferences) + /// How the query cache handles queries with non-deterministic functions, e.g. now() enum class QueryCacheNondeterministicFunctionHandling { diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index b5a14691756..0eed9eacac3 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -233,7 +233,7 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) sortConnectionPoolByNonReadOnlyReplicas(result); connection = std::move(result.front().entry); @@ -294,7 +294,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) sortConnectionPoolByNonReadOnlyReplicas(result); auto connection = std::move(result.front().entry); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index e041caa2249..f5135c6c80a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -413,7 +413,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_prefer_non_readonly_replica) + if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) sortConnectionPoolByNonReadOnlyReplicas(result); auto connection = std::move(result.front().entry); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index f41591a52ff..50017803913 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -373,7 +373,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// NOTE: INSERT will also take into account max_replica_delay_for_distributed_queries /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); - if (settings.distributed_insert_prefer_non_readonly_replica) + if (settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) sortConnectionPoolByNonReadOnlyReplicas(results); job.connection_entry = std::move(results.front().entry); } From a19b27bd7c53764b226da32f62a26758151581d4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:28:26 +0100 Subject: [PATCH 114/154] Revert extra changed lines That was the changes that looks OK for the initial patchset version, but not now, after all review comments. Signed-off-by: Azat Khuzhin --- src/Client/ConnectionEstablisher.cpp | 1 - src/Client/ConnectionPoolWithFailover.h | 7 +++---- src/Common/PoolWithFailoverBase.h | 14 ++++++++------ 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 853fea90f23..6b1b5aa968e 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -95,7 +95,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - return; } } catch (const Exception & e) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 0f47270e964..7bb4291e339 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -62,14 +62,13 @@ public: /// The same as getMany(), but return std::vector. std::vector getManyForTableFunction(const ConnectionTimeouts & timeouts, - const Settings & settings, - PoolMode pool_mode); + const Settings & settings, PoolMode pool_mode); using Base = PoolWithFailoverBase; using TryResult = Base::TryResult; - /// The same as getMany(), but check that: - /// - replication delay for table_to_check is acceptable (delay threshold is taken from settings) + /// The same as getMany(), but check that replication delay for table_to_check is acceptable. + /// Delay threshold is taken from settings. std::vector getManyChecked( const ConnectionTimeouts & timeouts, const Settings & settings, diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index ccb2354c312..46c414d34c1 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -293,12 +293,14 @@ PoolWithFailoverBase::getMany( std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); - auto comparator = [&](const TryResult & left, const TryResult & right) - { - return std::forward_as_tuple(!left.is_up_to_date, left.delay) - < std::forward_as_tuple(!right.is_up_to_date, right.delay); - }; - std::stable_sort(try_results.begin(), try_results.end(), comparator); + /// Sort so that preferred items are near the beginning. + std::stable_sort( + try_results.begin(), try_results.end(), + [](const TryResult & left, const TryResult & right) + { + return std::forward_as_tuple(!left.is_up_to_date, left.delay) + < std::forward_as_tuple(!right.is_up_to_date, right.delay); + }); if (fallback_to_stale_replicas) { From 63f92b2c1658631b5bf1cbdf1d33fa7623182f5c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 22 Feb 2024 17:58:55 +0100 Subject: [PATCH 115/154] Add documentation for distributed_insert_replicas_preferences Signed-off-by: Azat Khuzhin --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6c427442e49..9f790cdeb8e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2817,6 +2817,17 @@ Possible values: Default value: 0. +## distributed_insert_replicas_preferences {#distributed_insert_replicas_preferences} + +Adds ability to change replica preferences for INSERT queries. + +Possible values: + +- `no_preferences` - no preferences +- `prefer_non_read_only` - prefer non read-only replicas for INSERT into Distributed (but note, that they will not be excluded completely, so if you have only read-only replicas it will still try to INSERT into them, with respect to `distributed_replica_max_ignored_errors`). + +Default value: `prefer_non_read_only` + ## distributed_foreground_insert {#distributed_foreground_insert} Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table. From c7b919b19e7597298b42e483e12a63818e7ac47d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Mar 2024 15:12:46 +0100 Subject: [PATCH 116/154] Replace distributed_insert_replicas_preferences with distributed_insert_skip_read_only_replicas v2: fix test Signed-off-by: Azat Khuzhin --- docs/en/operations/settings/settings.md | 10 +++---- src/Client/ConnectionEstablisher.cpp | 2 -- src/Client/ConnectionPoolWithFailover.cpp | 30 +++++++++++-------- src/Client/ConnectionPoolWithFailover.h | 11 +++++-- src/Common/PoolWithFailoverBase.h | 24 +++++++++++---- src/Common/ProfileEvents.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Core/SettingsEnums.cpp | 4 --- src/Core/SettingsEnums.h | 8 ----- .../DistributedAsyncInsertBatch.cpp | 10 ++----- .../DistributedAsyncInsertDirectoryQueue.cpp | 5 +--- src/Storages/Distributed/DistributedSink.cpp | 4 +-- .../02980_dist_insert_readonly_replica.sql.j2 | 24 +++++++++++++-- 14 files changed, 77 insertions(+), 61 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9f790cdeb8e..665dd17eabe 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2817,16 +2817,16 @@ Possible values: Default value: 0. -## distributed_insert_replicas_preferences {#distributed_insert_replicas_preferences} +## distributed_insert_skip_read_only_replicas {#distributed_insert_skip_read_only_replicas} -Adds ability to change replica preferences for INSERT queries. +Enables skipping read-only replicas for INSERT queries into Distributed. Possible values: -- `no_preferences` - no preferences -- `prefer_non_read_only` - prefer non read-only replicas for INSERT into Distributed (but note, that they will not be excluded completely, so if you have only read-only replicas it will still try to INSERT into them, with respect to `distributed_replica_max_ignored_errors`). +- 0 — INSERT was as usual, if it will go to read-only replica it will fail +- 1 — Initiator will skip read-only replicas before sending data to shards. -Default value: `prefer_non_read_only` +Default value: `0` ## distributed_foreground_insert {#distributed_foreground_insert} diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 6b1b5aa968e..303105751ad 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -8,7 +8,6 @@ namespace ProfileEvents extern const Event DistributedConnectionUsable; extern const Event DistributedConnectionMissingTable; extern const Event DistributedConnectionStaleReplica; - extern const Event DistributedConnectionReadOnlyReplica; } namespace DB @@ -75,7 +74,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: { result.is_readonly = true; LOG_TRACE(log, "Table {}.{} is readonly on server {}", table_to_check->database, table_to_check->table, result.entry->getDescription()); - ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); } const UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index dddb3f45ba3..11bdb6108ca 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -158,6 +158,21 @@ std::vector ConnectionPoolWithFailover::g return getManyImpl(settings, pool_mode, try_get_entry, skip_unavailable_endpoints, priority_func); } +std::vector ConnectionPoolWithFailover::getManyCheckedForInsert( + const ConnectionTimeouts & timeouts, + const Settings & settings, + PoolMode pool_mode, + const QualifiedTableName & table_to_check) +{ + TryGetEntryFunc try_get_entry = [&](const NestedPoolPtr & pool, std::string & fail_message) + { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check, /*async_callback=*/ {}); }; + + return getManyImpl(settings, pool_mode, try_get_entry, + /*skip_unavailable_endpoints=*/ std::nullopt, + /*priority_func=*/ {}, + settings.distributed_insert_skip_read_only_replicas); +} + ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings & settings) { const size_t offset = settings.load_balancing_first_offset % nested_pools.size(); @@ -171,7 +186,8 @@ std::vector ConnectionPoolWithFailover::g PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, std::optional skip_unavailable_endpoints, - GetPriorityForLoadBalancing::Func priority_func) + GetPriorityForLoadBalancing::Func priority_func, + bool skip_read_only_replicas) { if (nested_pools.empty()) throw DB::Exception( @@ -203,7 +219,7 @@ std::vector ConnectionPoolWithFailover::g UInt64 max_ignored_errors = settings.distributed_replica_max_ignored_errors.value; bool fallback_to_stale_replicas = settings.fallback_to_stale_replicas_for_distributed_queries.value; - return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, try_get_entry, priority_func); + return Base::getMany(min_entries, max_entries, max_tries, max_ignored_errors, fallback_to_stale_replicas, skip_read_only_replicas, try_get_entry, priority_func); } ConnectionPoolWithFailover::TryResult @@ -254,14 +270,4 @@ ConnectionPoolWithFailover::getShuffledPools(const Settings & settings, GetPrior return Base::getShuffledPools(max_ignored_errors, priority_func, use_slowdown_count); } -void sortConnectionPoolByNonReadOnlyReplicas(std::vector & results) -{ - auto comparator = [&](const auto & left, const auto & right) - { - return std::forward_as_tuple(left.is_readonly, !left.is_up_to_date, left.delay) - < std::forward_as_tuple(right.is_readonly, !right.is_up_to_date, right.delay); - }; - std::stable_sort(results.begin(), results.end(), comparator); -} - } diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 7bb4291e339..a363a50244e 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -77,6 +77,12 @@ public: AsyncCallback async_callback = {}, std::optional skip_unavailable_endpoints = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); + /// The same as getManyChecked(), but respects distributed_insert_skip_read_only_replicas setting. + std::vector getManyCheckedForInsert( + const ConnectionTimeouts & timeouts, + const Settings & settings, + PoolMode pool_mode, + const QualifiedTableName & table_to_check); struct NestedPoolStatus { @@ -107,7 +113,8 @@ private: PoolMode pool_mode, const TryGetEntryFunc & try_get_entry, std::optional skip_unavailable_endpoints = std::nullopt, - GetPriorityForLoadBalancing::Func priority_func = {}); + GetPriorityForLoadBalancing::Func priority_func = {}, + bool skip_read_only_replicas = false); /// Try to get a connection from the pool and check that it is good. /// If table_to_check is not null and the check is enabled in settings, check that replication delay @@ -125,8 +132,6 @@ private: GetPriorityForLoadBalancing get_priority_load_balancing; }; -void sortConnectionPoolByNonReadOnlyReplicas(std::vector & results); - using ConnectionPoolWithFailoverPtr = std::shared_ptr; using ConnectionPoolWithFailoverPtrs = std::vector; diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 46c414d34c1..157b8f37b50 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -30,6 +30,7 @@ namespace ProfileEvents { extern const Event DistributedConnectionFailTry; extern const Event DistributedConnectionFailAtAll; + extern const Event DistributedConnectionReadOnlyReplica; } /// This class provides a pool with fault tolerance. It is used for pooling of connections to replicated DB. @@ -112,6 +113,7 @@ public: size_t min_entries, size_t max_entries, size_t max_tries, size_t max_ignored_errors, bool fallback_to_stale_replicas, + bool skip_read_only_replicas, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority); @@ -200,8 +202,12 @@ PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_ const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { std::vector results = getMany( - 1 /* min entries */, 1 /* max entries */, 1 /* max tries */, - max_ignored_errors, fallback_to_stale_replicas, + /* min_entries= */ 1, + /* max_entries= */ 1, + /* max_tries= */ 1, + max_ignored_errors, + fallback_to_stale_replicas, + /* skip_read_only_replicas= */ false, try_get_entry, get_priority); if (results.empty() || results[0].entry.isNull()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, @@ -215,6 +221,7 @@ PoolWithFailoverBase::getMany( size_t min_entries, size_t max_entries, size_t max_tries, size_t max_ignored_errors, bool fallback_to_stale_replicas, + bool skip_read_only_replicas, const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { @@ -266,9 +273,14 @@ PoolWithFailoverBase::getMany( ++entries_count; if (result.is_usable) { - ++usable_count; - if (result.is_up_to_date) - ++up_to_date_count; + if (!skip_read_only_replicas || !result.is_readonly) + { + ++usable_count; + if (result.is_up_to_date) + ++up_to_date_count; + } + else + ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); } } else @@ -291,7 +303,7 @@ PoolWithFailoverBase::getMany( throw DB::NetException(DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED, "All connection tries failed. Log: \n\n{}\n", fail_messages); - std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); + std::erase_if(try_results, [&](const TryResult & r) { return r.entry.isNull() || !r.is_usable || (skip_read_only_replicas && r.is_readonly); }); /// Sort so that preferred items are near the beginning. std::stable_sort( diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index b27c657aeaa..d8b3f01decc 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -156,7 +156,7 @@ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \ M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \ M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \ - M(DistributedConnectionReadOnlyReplica, "Number of times we got a read-only replica for distributed INSERT query.") \ + M(DistributedConnectionReadOnlyReplica, "Number of times INSERT into Distributed prefer different replica due to current was read-only") \ M(DistributedConnectionFailAtAll, "Total count when distributed connection fails after all retries finished.") \ \ M(HedgedRequestsChangeReplica, "Total count when timeout for changing replica expired in hedged requests.") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6d79209f66..04d30c5111d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -136,7 +136,7 @@ class IColumn; M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ \ - M(ReplicasPreferences, distributed_insert_replicas_preferences, ReplicasPreferences::PREFER_NON_READ_ONLY, "PREFER_NON_READ_ONLY - INSERT query into Distributed will try to prefer non-read-only replicas. Note, that this is only about the order, read-only replicas will not be excluded completely", 0) \ + M(Bool, distributed_insert_skip_read_only_replicas, false, "If true, INSERT into Distributed will skip read-only replicas.", 0) \ M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \ M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \ M(Milliseconds, distributed_background_insert_sleep_time_ms, 100, "Sleep time for background INSERTs into Distributed, in case of any errors delay grows exponentially.", 0) ALIAS(distributed_directory_monitor_sleep_time_ms) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3245da584a1..b7f2d453ba6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -136,7 +136,7 @@ static std::map sett {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"distributed_insert_replicas_preferences", "no_preferences", "prefer_non_read_only", "Control preferred replicas for INSERT into Distributed"}, + {"distributed_insert_skip_read_only_replicas", false, true, "If true, INSERT into Distributed will skip read-only replicas"}, {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 47dadf3c36b..c3f0715ad68 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,10 +79,6 @@ IMPLEMENT_SETTING_ENUM(DistributedProductMode, ErrorCodes::UNKNOWN_DISTRIBUTED_P {"global", DistributedProductMode::GLOBAL}, {"allow", DistributedProductMode::ALLOW}}) -IMPLEMENT_SETTING_ENUM(ReplicasPreferences, ErrorCodes::BAD_ARGUMENTS, - {{"no_preferences", ReplicasPreferences::NO_PREFERENCES}, - {"prefer_non_read_only", ReplicasPreferences::PREFER_NON_READ_ONLY}}) - IMPLEMENT_SETTING_ENUM(QueryCacheNondeterministicFunctionHandling, ErrorCodes::BAD_ARGUMENTS, {{"throw", QueryCacheNondeterministicFunctionHandling::Throw}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index d51271ef16e..0aa8216bb85 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -174,14 +174,6 @@ enum class DistributedProductMode DECLARE_SETTING_ENUM(DistributedProductMode) -enum class ReplicasPreferences -{ - NO_PREFERENCES = 0, - /// Prefer non read-only replicas (they will not be excluded completely). - PREFER_NON_READ_ONLY, -}; -DECLARE_SETTING_ENUM(ReplicasPreferences) - /// How the query cache handles queries with non-deterministic functions, e.g. now() enum class QueryCacheNondeterministicFunctionHandling { diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 0eed9eacac3..21c79597aea 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -232,10 +232,7 @@ void DistributedAsyncInsertBatch::sendBatch(const SettingsChanges & settings_cha insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) - sortConnectionPoolByNonReadOnlyReplicas(result); - + auto result = parent.pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); connection = std::move(result.front().entry); compression_expected = connection->getCompression() == Protocol::Compression::Enable; @@ -293,10 +290,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles(const SettingsChanges & sett parent.storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = parent.pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) - sortConnectionPoolByNonReadOnlyReplicas(result); - + auto result = parent.pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, parent.storage.remote_storage.getQualifiedName()); auto connection = std::move(result.front().entry); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index f5135c6c80a..1ee77611191 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -412,10 +412,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path, insert_settings.applyChanges(settings_changes); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(insert_settings); - auto result = pool->getManyChecked(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); - if (distributed_header.insert_settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) - sortConnectionPoolByNonReadOnlyReplicas(result); - + auto result = pool->getManyCheckedForInsert(timeouts, insert_settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); auto connection = std::move(result.front().entry); LOG_DEBUG(log, "Sending `{}` to {} ({} rows, {} bytes)", diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 50017803913..ddbcc6d473f 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -372,9 +372,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// TODO: it make sense to rewrite skip_unavailable_shards and max_parallel_replicas here /// NOTE: INSERT will also take into account max_replica_delay_for_distributed_queries /// (anyway fallback_to_stale_replicas_for_distributed_queries=true by default) - auto results = shard_info.pool->getManyChecked(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); - if (settings.distributed_insert_replicas_preferences == ReplicasPreferences::PREFER_NON_READ_ONLY) - sortConnectionPoolByNonReadOnlyReplicas(results); + auto results = shard_info.pool->getManyCheckedForInsert(timeouts, settings, PoolMode::GET_ONE, storage.remote_storage.getQualifiedName()); job.connection_entry = std::move(results.front().entry); } else diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 2c6d53d88fb..0d96db4c62f 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -29,14 +29,32 @@ system stop distributed sends dist; create table dist_batch as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', '', 'data') settings background_insert_batch=1; system stop distributed sends dist_batch; +create table dist_single_no_internal_replication as shard_0.data engine=Distributed('default', 'shard_1', 'data'); +system stop distributed sends dist_single_no_internal_replication; + +create table dist_single as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', 'shard_1', 'data'); +system stop distributed sends dist_single; + set prefer_localhost_replica=0; set insert_deduplicate=0; +-- replica is readonly, avoid too many retries +set insert_keeper_max_retries=3; +-- and disable the fault injection to avoid failures +set insert_keeper_fault_injection_probability=0; + +-- for internal_replicatio==false, distributed_insert_skip_read_only_replicas does not changes anything +insert into dist_single_no_internal_replication settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } +insert into dist_single_no_internal_replication settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError TABLE_IS_READ_ONLY } + +-- for internal_replicatio==true, it does +insert into dist_single settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } +insert into dist_single settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError ALL_CONNECTION_TRIES_FAILED } -- min_insert_block_size_rows is not enough, since replica will be selected -- before, so we need to perform INSERT multiple times to ensure that at least -- once read-only replica should be selected (if it wasn't not filtered out) {% for i in range(1, 30) %} -insert into dist settings distributed_foreground_insert=1 values ({{ i }}); +insert into dist settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values ({{ i }}); {% endfor %} -- cannot check for background inserts, so only here system flush logs; @@ -51,12 +69,12 @@ where and Settings['distributed_foreground_insert'] = '1'; {% for i in range(1, 30) %} -insert into dist settings distributed_foreground_insert=0 values ({{ i }}); +insert into dist settings distributed_foreground_insert=0, distributed_insert_skip_read_only_replicas=1 values ({{ i }}); system flush distributed dist; {% endfor %} {% for i in range(1, 30) %} -insert into dist_batch settings distributed_foreground_insert=0 values ({{ i }}); +insert into dist_batch settings distributed_foreground_insert=0, distributed_insert_skip_read_only_replicas=1 values ({{ i }}); system flush distributed dist_batch; {% endfor %} From ac882c1e70041c5cc4ea12ab04da4c1cc58ccea4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Mar 2024 16:13:45 -0400 Subject: [PATCH 117/154] Simplify condition in PoolWithFailoverBase Signed-off-by: Azat Khuzhin --- src/Common/PoolWithFailoverBase.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 157b8f37b50..df180ec27ae 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -273,14 +273,14 @@ PoolWithFailoverBase::getMany( ++entries_count; if (result.is_usable) { - if (!skip_read_only_replicas || !result.is_readonly) + if (skip_read_only_replicas && result.is_readonly) + ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); + else { ++usable_count; if (result.is_up_to_date) ++up_to_date_count; } - else - ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); } } else From f86bb1e9e9fcfeacb61fc1124fe74f086cb2db65 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Mar 2024 16:15:20 -0400 Subject: [PATCH 118/154] Rename to DistributedConnectionSkipReadOnlyReplica Signed-off-by: Azat Khuzhin --- src/Common/PoolWithFailoverBase.h | 4 ++-- src/Common/ProfileEvents.cpp | 2 +- .../0_stateless/02980_dist_insert_readonly_replica.sql.j2 | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index df180ec27ae..8365e818840 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -30,7 +30,7 @@ namespace ProfileEvents { extern const Event DistributedConnectionFailTry; extern const Event DistributedConnectionFailAtAll; - extern const Event DistributedConnectionReadOnlyReplica; + extern const Event DistributedConnectionSkipReadOnlyReplica; } /// This class provides a pool with fault tolerance. It is used for pooling of connections to replicated DB. @@ -274,7 +274,7 @@ PoolWithFailoverBase::getMany( if (result.is_usable) { if (skip_read_only_replicas && result.is_readonly) - ProfileEvents::increment(ProfileEvents::DistributedConnectionReadOnlyReplica); + ProfileEvents::increment(ProfileEvents::DistributedConnectionSkipReadOnlyReplica); else { ++usable_count; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index d8b3f01decc..f69718fdee7 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -156,7 +156,7 @@ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.") \ M(DistributedConnectionMissingTable, "Number of times we rejected a replica from a distributed query, because it did not contain a table needed for the query.") \ M(DistributedConnectionStaleReplica, "Number of times we rejected a replica from a distributed query, because some table needed for a query had replication lag higher than the configured threshold.") \ - M(DistributedConnectionReadOnlyReplica, "Number of times INSERT into Distributed prefer different replica due to current was read-only") \ + M(DistributedConnectionSkipReadOnlyReplica, "Number of replicas skipped during INSERT into Distributed table due to replicas being read-only") \ M(DistributedConnectionFailAtAll, "Total count when distributed connection fails after all retries finished.") \ \ M(HedgedRequestsChangeReplica, "Total count when timeout for changing replica expired in hedged requests.") \ diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 0d96db4c62f..2da8acba713 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -59,7 +59,7 @@ insert into dist settings distributed_foreground_insert=1, distributed_insert_sk -- cannot check for background inserts, so only here system flush logs; select - 'DistributedConnectionReadOnlyReplica', count() > 0, sum(ProfileEvents['DistributedConnectionReadOnlyReplica']) > 0 + 'DistributedConnectionSkipReadOnlyReplica', count() > 0, sum(ProfileEvents['DistributedConnectionSkipReadOnlyReplica']) > 0 from system.query_log where event_date >= yesterday() From 1bb9cffb9b6d3698651e960ab90d5459bfe5bd41 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Mar 2024 16:18:58 -0400 Subject: [PATCH 119/154] Remove distributed_insert_skip_read_only_replicas from settings history Signed-off-by: Azat Khuzhin --- src/Core/SettingsChangesHistory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b7f2d453ba6..6b31e9cd249 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -136,7 +136,6 @@ static std::map sett {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"distributed_insert_skip_read_only_replicas", false, true, "If true, INSERT into Distributed will skip read-only replicas"}, {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, From a90d64abf3728919fd2f93f8c322f58bb41290c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 19 Mar 2024 16:22:08 -0400 Subject: [PATCH 120/154] Cleanup test Signed-off-by: Azat Khuzhin --- .../02980_dist_insert_readonly_replica.reference | 2 +- .../02980_dist_insert_readonly_replica.sql.j2 | 10 ++++------ 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference index 6b27a3b185c..2ee76705e90 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.reference @@ -1,3 +1,3 @@ shard_0.data 0 shard_1.data 1 -DistributedConnectionReadOnlyReplica 1 1 +DistributedConnectionSkipReadOnlyReplica 1 1 diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 2da8acba713..3ed8f439dcc 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -9,8 +9,6 @@ create database shard_1; drop table if exists dist; drop table if exists dist_batch; -drop table if exists shard_0.data; -drop table if exists shard_1.data; -- normal replica create table shard_0.data (key Int) engine=ReplicatedMergeTree('/tables/{database}/data', 'write') order by key; @@ -29,8 +27,8 @@ system stop distributed sends dist; create table dist_batch as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', '', 'data') settings background_insert_batch=1; system stop distributed sends dist_batch; -create table dist_single_no_internal_replication as shard_0.data engine=Distributed('default', 'shard_1', 'data'); -system stop distributed sends dist_single_no_internal_replication; +create table dist_single_no_internal_replication_read_only as shard_0.data engine=Distributed('default', 'shard_1', 'data'); +system stop distributed sends dist_single_no_internal_replication_read_only; create table dist_single as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', 'shard_1', 'data'); system stop distributed sends dist_single; @@ -43,8 +41,8 @@ set insert_keeper_max_retries=3; set insert_keeper_fault_injection_probability=0; -- for internal_replicatio==false, distributed_insert_skip_read_only_replicas does not changes anything -insert into dist_single_no_internal_replication settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } -insert into dist_single_no_internal_replication settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError TABLE_IS_READ_ONLY } +insert into dist_single_no_internal_replication_read_only settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } +insert into dist_single_no_internal_replication_read_only settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError TABLE_IS_READ_ONLY } -- for internal_replicatio==true, it does insert into dist_single settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } From 3481fd5191015e9209379a700201cccaecf25228 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 Mar 2024 16:46:56 +0100 Subject: [PATCH 121/154] Fix settings history for distributed_insert_skip_read_only_replicas CI: https://s3.amazonaws.com/clickhouse-test-reports/59176/fa0c47a42f4187e8284e19d1cd008a0a76c38caf/fast_test.html Signed-off-by: Azat Khuzhin --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6b31e9cd249..247af4d5d16 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -101,6 +101,7 @@ static std::map sett {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, + {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, From 0900c4af607617e1fe4ceac16710455cafd75c42 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Mar 2024 11:31:48 +0100 Subject: [PATCH 122/154] Add comments for 02980_dist_insert_readonly_replica Signed-off-by: Azat Khuzhin --- .../0_stateless/02980_dist_insert_readonly_replica.sql.j2 | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 3ed8f439dcc..45140ca6e01 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -40,11 +40,12 @@ set insert_keeper_max_retries=3; -- and disable the fault injection to avoid failures set insert_keeper_fault_injection_probability=0; --- for internal_replicatio==false, distributed_insert_skip_read_only_replicas does not changes anything +-- for internal_replication==false, distributed_insert_skip_read_only_replicas does not changes anything +-- because in case of internal_replication=false it cannot skip any replicas, since it need to duplicate the data to all replicas on the initiator insert into dist_single_no_internal_replication_read_only settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } insert into dist_single_no_internal_replication_read_only settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError TABLE_IS_READ_ONLY } --- for internal_replicatio==true, it does +-- for internal_replication==true, it does insert into dist_single settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=0 values (0); -- { serverError TABLE_IS_READ_ONLY } insert into dist_single settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values (0); -- { serverError ALL_CONNECTION_TRIES_FAILED } From 05792ee9cad88186f9204f3f1e38dc4fd6ac050e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Mar 2024 11:34:26 +0100 Subject: [PATCH 123/154] Make 02980_dist_insert_readonly_replica test more stable (by using round_robin LB) Signed-off-by: Azat Khuzhin --- .../0_stateless/02980_dist_insert_readonly_replica.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 45140ca6e01..4e71fb41c89 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -53,7 +53,7 @@ insert into dist_single settings distributed_foreground_insert=1, distributed_in -- before, so we need to perform INSERT multiple times to ensure that at least -- once read-only replica should be selected (if it wasn't not filtered out) {% for i in range(1, 30) %} -insert into dist settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1 values ({{ i }}); +insert into dist settings distributed_foreground_insert=1, distributed_insert_skip_read_only_replicas=1, load_balancing='round_robin' values ({{ i }}); {% endfor %} -- cannot check for background inserts, so only here system flush logs; From 4eafb8653573c8efd1d64e834a37b8a2fdb9b304 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Mar 2024 11:35:32 +0100 Subject: [PATCH 124/154] Do not use default (removed) cluster in 02980_dist_insert_readonly_replica Signed-off-by: Azat Khuzhin --- .../0_stateless/02980_dist_insert_readonly_replica.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 index 4e71fb41c89..5bf40f34f5c 100644 --- a/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 +++ b/tests/queries/0_stateless/02980_dist_insert_readonly_replica.sql.j2 @@ -27,7 +27,7 @@ system stop distributed sends dist; create table dist_batch as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', '', 'data') settings background_insert_batch=1; system stop distributed sends dist_batch; -create table dist_single_no_internal_replication_read_only as shard_0.data engine=Distributed('default', 'shard_1', 'data'); +create table dist_single_no_internal_replication_read_only as shard_0.data engine=Distributed('test_shard_localhost', 'shard_1', 'data'); system stop distributed sends dist_single_no_internal_replication_read_only; create table dist_single as shard_0.data engine=Distributed('test_cluster_two_replicas_different_databases_internal_replication', 'shard_1', 'data'); From 2ed53e309bdd07e8292d351b801ffdc1509795ec Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 26 Mar 2024 07:44:55 -0300 Subject: [PATCH 125/154] Reset part level upon attach from disk on MergeTree (#61536) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * hackish change to check how CI reacts * remove unnecessary renameTo * introduce separate metho * remove unused arg * rename meth od * add simple test * add current database * set mutations sync * Update tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.sql --------- Co-authored-by: János Benjamin Antal Co-authored-by: János Benjamin Antal --- src/Storages/StorageMergeTree.cpp | 10 ++++++++- src/Storages/StorageMergeTree.h | 1 + ...vel_is_reset_attach_from_disk_mt.reference | 22 +++++++++++++++++++ ...art_level_is_reset_attach_from_disk_mt.sql | 21 ++++++++++++++++++ 4 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.reference create mode 100644 tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.sql diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8c78ac10d27..aad4fc36a1b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2042,7 +2042,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( MergeTreeData::Transaction transaction(*this, local_context->getCurrentTransaction().get()); { auto lock = lockParts(); - fillNewPartName(loaded_parts[i], lock); + fillNewPartNameAndResetLevel(loaded_parts[i], lock); renameTempPartAndAdd(loaded_parts[i], transaction, lock); transaction.commit(&lock); } @@ -2482,4 +2482,12 @@ void StorageMergeTree::fillNewPartName(MutableDataPartPtr & part, DataPartsLock part->setName(part->getNewName(part->info)); } +void StorageMergeTree::fillNewPartNameAndResetLevel(MutableDataPartPtr & part, DataPartsLock &) +{ + part->info.min_block = part->info.max_block = increment.get(); + part->info.mutation = 0; + part->info.level = 0; + part->setName(part->getNewName(part->info)); +} + } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c384a391291..d864b3e626c 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -260,6 +260,7 @@ private: std::set * mutation_ids = nullptr, bool from_another_mutation = false) const; void fillNewPartName(MutableDataPartPtr & part, DataPartsLock & lock); + void fillNewPartNameAndResetLevel(MutableDataPartPtr & part, DataPartsLock & lock); void startBackgroundMovesIfNeeded() override; diff --git a/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.reference b/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.reference new file mode 100644 index 00000000000..a3cb6a511e5 --- /dev/null +++ b/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.reference @@ -0,0 +1,22 @@ +-- {echoOn} +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test VALUES (1), (2), (3); +OPTIMIZE TABLE test FINAL; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +all_1_1_1 +ALTER TABLE test DETACH PART 'all_1_1_1'; +ALTER TABLE test ATTACH PART 'all_1_1_1'; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +all_2_2_0 +-- Same as above, but with attach partition (different code path, should be tested as well) +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test VALUES (1), (2), (3); +OPTIMIZE TABLE test FINAL; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +all_1_1_1 +ALTER TABLE test DETACH PART 'all_1_1_1'; +ALTER TABLE test ATTACH PARTITION tuple(); +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +all_2_2_0 diff --git a/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.sql b/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.sql new file mode 100644 index 00000000000..eb05dfea802 --- /dev/null +++ b/tests/queries/0_stateless/03013_test_part_level_is_reset_attach_from_disk_mt.sql @@ -0,0 +1,21 @@ +-- Tags: no-shared-merge-tree +SET alter_sync = 2; +-- {echoOn} +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test VALUES (1), (2), (3); +OPTIMIZE TABLE test FINAL; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +ALTER TABLE test DETACH PART 'all_1_1_1'; +ALTER TABLE test ATTACH PART 'all_1_1_1'; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); + +-- Same as above, but with attach partition (different code path, should be tested as well) +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test VALUES (1), (2), (3); +OPTIMIZE TABLE test FINAL; +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); +ALTER TABLE test DETACH PART 'all_1_1_1'; +ALTER TABLE test ATTACH PARTITION tuple(); +SELECT part_name FROM system.parts where table='test' and active and database = currentDatabase(); From 14c60dea8e3f8698e27947e311dd743f9bd53d28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Mar 2024 12:18:44 +0100 Subject: [PATCH 126/154] Fix flaky test_broken_projestions/test.py::test_broken_ignored_replicated --- .../test_broken_projections/test.py | 42 +++++++++++-------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 4a4690a5d0a..271b9ec5a18 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -223,14 +223,17 @@ def check(node, table, check_result, expect_broken_part="", expected_error=""): query_id = node.query( f"SELECT queryID() FROM (SELECT c FROM '{table}' WHERE d == 12 ORDER BY c)" ).strip() - node.query("SYSTEM FLUSH LOGS") - res = node.query( - f""" - SELECT query, splitByChar('.', arrayJoin(projections))[-1] - FROM system.query_log - WHERE query_id='{query_id}' AND type='QueryFinish' - """ - ) + for _ in range(10): + node.query("SYSTEM FLUSH LOGS") + res = node.query( + f""" + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE query_id='{query_id}' AND type='QueryFinish' + """ + ) + if res != "": + break if res == "": res = node.query( """ @@ -238,7 +241,7 @@ def check(node, table, check_result, expect_broken_part="", expected_error=""): FROM system.query_log ORDER BY query_start_time_microseconds DESC """ ) - print(f"LOG: {res}") + print(f"Looked for query id {query_id}, but to no avail: {res}") assert False assert "proj1" in res @@ -250,14 +253,17 @@ def check(node, table, check_result, expect_broken_part="", expected_error=""): query_id = node.query( f"SELECT queryID() FROM (SELECT d FROM '{table}' WHERE c == 12 ORDER BY d)" ).strip() - node.query("SYSTEM FLUSH LOGS") - res = node.query( - f""" - SELECT query, splitByChar('.', arrayJoin(projections))[-1] - FROM system.query_log - WHERE query_id='{query_id}' AND type='QueryFinish' - """ - ) + for _ in range(10): + node.query("SYSTEM FLUSH LOGS") + res = node.query( + f""" + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE query_id='{query_id}' AND type='QueryFinish' + """ + ) + if res != "": + break if res == "": res = node.query( """ @@ -265,7 +271,7 @@ def check(node, table, check_result, expect_broken_part="", expected_error=""): FROM system.query_log ORDER BY query_start_time_microseconds DESC """ ) - print(f"LOG: {res}") + print(f"Looked for query id {query_id}, but to no avail: {res}") assert False assert "proj2" in res From c7775a10eaecc02bf5e7829fce89aa5d3d656ec3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 26 Mar 2024 13:39:54 +0100 Subject: [PATCH 127/154] Do not run the test with SharedMergeTree --- .../queries/0_stateless/03010_read_system_parts_table_test.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03010_read_system_parts_table_test.sql b/tests/queries/0_stateless/03010_read_system_parts_table_test.sql index a50005b2312..1f72c2a3d6a 100644 --- a/tests/queries/0_stateless/03010_read_system_parts_table_test.sql +++ b/tests/queries/0_stateless/03010_read_system_parts_table_test.sql @@ -1,3 +1,5 @@ +-- Tags: no-shared-merge-tree +-- The parts might named differently with SMT DROP TABLE IF EXISTS users; CREATE TABLE users (uid Int16, name String, age Int16) ENGINE=MergeTree ORDER BY uid PARTITION BY uid; From e9262536641bbbf724a9c0452ef763d04e4a9f49 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Mar 2024 14:47:01 +0100 Subject: [PATCH 128/154] Check is Rust avaiable for build, if not, suggest a way to disable Rust support Rust requires internet connection, so let's detect if it works, and if not, suggest building with -DENABLE_RUST=OFF, here is an example of the output in case of error: $ docker run --network=none ... -- Copy skim to /root/rust/skim CMake Error at rust/CMakeLists.txt:112 (message): Rust (/rust/rustup/toolchains/nightly-x86_64-unknown-linux-gnu/bin/cargo) support is not available (likely there is no internet connectivity): Updating git repository `https://github.com/azat-rust/tuikit.git` warning: spurious network error (3 tries remaining): failed to resolve address for github.com: Temporary failure in name resolution; class=Net (12) warning: spurious network error (2 tries remaining): failed to resolve address for github.com: Temporary failure in name resolution; class=Net (12) warning: spurious network error (1 tries remaining): failed to resolve address for github.com: Temporary failure in name resolution; class=Net (12) error: failed to load source for dependency `tuikit` Caused by: Unable to update https://github.com/azat-rust/tuikit.git?rev=e1994c0e03ff02c49cf1471f0cc3cbf185ce0104 Caused by: failed to clone into: /rust/cargo/git/db/tuikit-c3ca927b4dbcf00d Caused by: network failure seems to have happened if a proxy or similar is necessary `net.git-fetch-with-cli` may help here https://doc.rust-lang.org/cargo/reference/config.html#netgit-fetch-with-cli Caused by: failed to resolve address for github.com: Temporary failure in name resolution; class=Net (12) You can disable Rust support with -DENABLE_RUST=OFF Call Stack (most recent call first): rust/CMakeLists.txt:129 (add_rust_subdirectory) -- Configuring incomplete, errors occurred! Signed-off-by: Azat Khuzhin --- rust/CMakeLists.txt | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 6715a54221a..cebfd36a24a 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -99,6 +99,19 @@ function(add_rust_subdirectory src) message(STATUS "Copy ${src} to ${dst}") file(COPY "${src}" DESTINATION "${CMAKE_CURRENT_BINARY_DIR}" PATTERN target EXCLUDE) + + # Check is Rust available or not. + # + # `cargo update --dry-run` will not update anything, but will check the internet connectivity. + execute_process(COMMAND ${Rust_CARGO_CACHED} update --dry-run + WORKING_DIRECTORY "${dst}" + RESULT_VARIABLE CARGO_UPDATE_RESULT + OUTPUT_VARIABLE CARGO_UPDATE_STDOUT + ERROR_VARIABLE CARGO_UPDATE_STDERR) + if (CARGO_UPDATE_RESULT) + message(FATAL_ERROR "Rust (${Rust_CARGO_CACHED}) support is not available (likely there is no internet connectivity):\n${CARGO_UPDATE_STDERR}\nYou can disable Rust support with -DENABLE_RUST=OFF") + endif() + add_subdirectory("${dst}" "${dst}") # cmake -E copy* do now know how to exclude files From 1f0938f66990b933d1f94f8255872f473f9262fc Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Tue, 26 Mar 2024 14:38:51 +0000 Subject: [PATCH 129/154] add a test --- .../0_stateless/02935_date_trunc_case_unsensitiveness.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql index ecf6877d477..06cda16fb01 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -7,3 +7,6 @@ SELECT dateTrunc('Week', toDate('2022-03-01')); SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('week', toDate('2022-03-01')); +SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError 36 } +SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError 36 } +SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError 36 } From e4d9a7405758a6d6aee36429b1db6871b1a2d10a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 26 Mar 2024 14:43:53 +0000 Subject: [PATCH 130/154] CI: create CIOptions class to handle ci user config CI: create CIOptions class to handle ci user config --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- tests/ci/ci.py | 311 +++++++++++++++++++------------ 2 files changed, 197 insertions(+), 116 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index fede3fe519d..6110ea19426 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -44,7 +44,7 @@ At a minimum, the following information should be added (but add more as needed) --- ### Modify your CI run: **NOTE:** If your merge the PR with modified CI you **MUST KNOW** what you are doing -**NOTE:** Set desired options before CI starts or re-push after updates +**NOTE:** You can update this config before RunConfig job is started to have it applied in CI #### Run only: - [ ] Integration tests diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 514189a8b8a..bd1582261a8 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -12,7 +12,7 @@ from copy import deepcopy from dataclasses import asdict, dataclass from enum import Enum from pathlib import Path -from typing import Any, Dict, List, Optional, Sequence, Set, Union +from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union import docker_images_helper import upload_result_helper @@ -733,6 +733,183 @@ class CiCache: return await_finished +@dataclass +class CiOptions: + # job will be included in the run if any keyword from the list matches job name + include_keywords: Optional[List[str]] = None + # job will be excluded in the run if any keyword from the list matches job name + exclude_keywords: Optional[List[str]] = None + + # list of specified preconfigured ci sets to run + ci_sets: Optional[List[str]] = None + # list of specified jobs to run + ci_jobs: Optional[List[str]] = None + + # btaches to run for all multi-batch jobs + job_batches: Optional[List[int]] = None + + do_not_test: bool = False + no_ci_cache: bool = False + no_merge_commit: bool = False + + def as_dict(self) -> Dict[str, Any]: + return asdict(self) + + @staticmethod + def create_from_run_config(run_config: Dict[str, Any]) -> "CiOptions": + return CiOptions(**run_config["ci_options"]) + + @staticmethod + def create_from_pr_message(commit_message: Optional[str]) -> "CiOptions": + """ + Creates CiOptions instance based on tags found in PR body and/or commit message + @commit_message - may be provided directly for debugging purposes, otherwise it will be retrieved from git. + """ + res = CiOptions() + pr_info = PRInfo() + if not pr_info.is_pr(): + # CI options can be configured in PRs only + return res + + message = commit_message or GitRunner(set_cwd_to_git_root=True).run( + f"{GIT_PREFIX} log {pr_info.sha} --format=%B -n 1" + ) + + pattern = r"(#|- \[x\] + Integration tests -- [ ] Integration tests (arm64) -- [ ] Stateless tests (release) -- [ ] Stateless tests (asan) -- [ ] Stateful tests (release) -- [ ] Stateful tests (asan) -- [ ] No sanitizers -- [ ] Tests with analyzer -- [ ] Fast tests -- [ ] Only package_debug build -- [ ] Add your CI variant description here +#### Include tests (required builds will be added automatically): +- [ ] Fast test +- [ ] Integration Tests +- [ ] Stateless tests +- [ ] Stateful tests +- [ ] Unit tests +- [ ] Performance tests +- [ ] All with ASAN +- [ ] All with TSAN +- [ ] All with Analyzer +- [ ] Add your option here -#### CI options: +#### Exclude tests: +- [ ] Fast test +- [ ] Integration Tests +- [ ] Stateless tests +- [ ] Stateful tests +- [ ] Performance tests +- [ ] All with ASAN +- [ ] All with TSAN +- [ ] All with MSAN +- [ ] All with UBSAN +- [ ] All with Coverage +- [ ] All with Aarch64 +- [ ] Add your option here + +#### Extra options: - [ ] do not test (only style check) - [ ] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index bd1582261a8..a3214784d29 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -760,18 +760,21 @@ class CiOptions: return CiOptions(**run_config["ci_options"]) @staticmethod - def create_from_pr_message(commit_message: Optional[str]) -> "CiOptions": + def create_from_pr_message( + debug_message: Optional[str], update_from_api: bool + ) -> "CiOptions": """ Creates CiOptions instance based on tags found in PR body and/or commit message @commit_message - may be provided directly for debugging purposes, otherwise it will be retrieved from git. """ res = CiOptions() pr_info = PRInfo() - if not pr_info.is_pr(): + if ( + not pr_info.is_pr() and not debug_message + ): # if commit_message is provided it's test/debug scenario - do not return # CI options can be configured in PRs only return res - - message = commit_message or GitRunner(set_cwd_to_git_root=True).run( + message = debug_message or GitRunner(set_cwd_to_git_root=True).run( f"{GIT_PREFIX} log {pr_info.sha} --format=%B -n 1" ) @@ -779,10 +782,13 @@ class CiOptions: matches = [match[-1] for match in re.findall(pattern, message)] print(f"CI tags from commit message: [{matches}]") - pr_info = PRInfo(pr_event_from_api=True) # Fetch updated PR body from GH API - matches_pr = [match[-1] for match in re.findall(pattern, pr_info.body)] - print(f"CI tags from PR body: [{matches_pr}]") - matches = set(matches + matches_pr) + if not debug_message: # to be skipped if debug/test + pr_info = PRInfo( + pr_event_from_api=update_from_api + ) # Fetch updated PR body from GH API + matches_pr = [match[-1] for match in re.findall(pattern, pr_info.body)] + print(f"CI tags from PR body: [{matches_pr}]") + matches = list(set(matches + matches_pr)) if "do not test" in pr_info.labels: # do_not_test could be set in GH labels @@ -800,11 +806,15 @@ class CiOptions: elif match.startswith("ci_include_"): if not res.include_keywords: res.include_keywords = [] - res.include_keywords.append(match.removeprefix("ci_include_")) + res.include_keywords.append( + normalize_check_name(match.removeprefix("ci_include_")) + ) elif match.startswith("ci_exclude_"): if not res.exclude_keywords: res.exclude_keywords = [] - res.exclude_keywords.append(match.removeprefix("ci_exclude_")) + res.exclude_keywords.append( + normalize_check_name(match.removeprefix("ci_exclude_")) + ) elif match == Labels.NO_CI_CACHE: res.no_ci_cache = True print("NOTE: CI Cache will be disabled") @@ -845,6 +855,44 @@ class CiOptions: """ jobs_to_do_requested = [] # type: List[str] + # -1. Handle "ci_exclude_" tags if any + if self.exclude_keywords: + new_jobs_to_do = list(jobs_to_do) + for job in jobs_to_do: + found = False + for keyword in self.exclude_keywords: + if keyword in normalize_check_name(job): + print( + f"Job [{job}] matches Exclude keyword [{keyword}] - remove" + ) + found = True + break + if found: + new_jobs_to_do.remove(job) + jobs_to_do = new_jobs_to_do + + # 0. Handle "ci_include_" tags if any + if self.include_keywords: + for job in jobs_to_do: + found = False + for keyword in self.include_keywords: + if keyword in normalize_check_name(job): + print(f"Job [{job}] matches Include keyword [{keyword}] - add") + found = True + break + if found: + job_with_parents = CI_CONFIG.get_job_with_parents(job) + for job in job_with_parents: + if job in jobs_to_do and job not in jobs_to_do_requested: + jobs_to_do_requested.append(job) + assert ( + jobs_to_do_requested + ), "Include tags are set but now job configured - Invalid tags, probably [{self.include_keywords}]" + if JobNames.STYLE_CHECK not in jobs_to_do_requested: + # Style check must not be omitted + jobs_to_do_requested.append(JobNames.STYLE_CHECK) + + # FIXME: to be removed in favor of include/exclude # 1. Handle "ci_set_" tags if any if self.ci_sets: for tag in self.ci_sets: @@ -855,6 +903,7 @@ class CiOptions: ) jobs_to_do_requested += label_config.run_jobs + # FIXME: to be removed in favor of include/exclude # 2. Handle "job_" tags if any if self.ci_jobs: for tag in self.ci_jobs: @@ -870,6 +919,8 @@ class CiOptions: # 3. Handle "do not test" if self.do_not_test: + label_config = CI_CONFIG.get_label_config(Labels.DO_NOT_TEST_LABEL) + assert label_config print( f"NOTE: CI 'do not test' setting applied, set jobs: [{label_config.run_jobs}]" ) @@ -877,8 +928,7 @@ class CiOptions: print( "WARNING: 'do not test' is used alongside with other CI modifying tags - 'do not test' prevails" ) - label_config = CI_CONFIG.get_label_config(Labels.DO_NOT_TEST_LABEL) - jobs_to_do_requested = label_config.run_jobs + jobs_to_do_requested = list(label_config.run_jobs) if jobs_to_do_requested: jobs_to_do_requested = list(set(jobs_to_do_requested)) @@ -1794,7 +1844,9 @@ def main() -> int: ### CONFIGURE action: start if args.configure: - ci_options = CiOptions.create_from_pr_message(args.commit_message or None) + ci_options = CiOptions.create_from_pr_message( + args.commit_message or None, update_from_api=True + ) # tokens = _fetch_commit_tokens(message, pr_info) if ci_options.no_merge_commit and CI: diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py new file mode 100644 index 00000000000..61aeec59581 --- /dev/null +++ b/tests/ci/test_ci_options.py @@ -0,0 +1,121 @@ +#!/usr/bin/env python3 + +# type: ignore + +import unittest +from ci import CiOptions +from ci_config import JobNames + +_TEST_BODY_1 = """ +#### Run only: +- [x] Integration tests +- [ ] Integration tests (arm64) +- [x] Integration tests +- [x] Integration tests +- [ ] Integration tests +- [x] some invalid mask - should be skipped +- [x] Integration tests +- [ ] Integration tests + +#### CI options: +- [ ] do not test (only style check) +- [x] disable merge-commit (no merge from master before tests) +- [ ] disable CI cache (job reuse) + +#### Only specified batches in multi-batch jobs: +- [x] 1 +- [ ] 2 +""" + +_TEST_BODY_2 = """ +- [x] MUST include integration tests +- [x] MUST include stateless tests +- [x] no action must be applied +- [ ] no action must be applied +- [x] MUST exclude tsan +- [x] MUST exclude aarch64 +- [x] MUST exclude test with analazer +- [ ] no action applied +- [x] Must exclude statless test with s3 storage +- [x] Must exclude tests on coverage build +""" + +_TEST_BODY_3 = """ +- [x] Must include all tests for analyzer +""" + + +class TestCIOptions(unittest.TestCase): + def test_pr_body_parsing(self): + ci_options = CiOptions.create_from_pr_message( + _TEST_BODY_1, update_from_api=False + ) + self.assertFalse(ci_options.do_not_test) + self.assertFalse(ci_options.no_ci_cache) + self.assertTrue(ci_options.no_merge_commit) + self.assertEqual(ci_options.ci_sets, ["ci_set_integration"]) + self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"]) + self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"]) + + def test_options_applied(self): + self.maxDiff = None + ci_options = CiOptions.create_from_pr_message( + _TEST_BODY_2, update_from_api=False + ) + self.assertCountEqual( + ci_options.include_keywords, ["integration", "foo_bar", "stateless"] + ) + self.assertCountEqual( + ci_options.exclude_keywords, + ["tsan", "aarch64", "analyzer", "s3_storage", "coverage"], + ) + jobs_to_do = list(JobNames) + jobs_to_skip = [] + job_params = {} + jobs_to_do, jobs_to_skip, job_params = ci_options.apply( + jobs_to_do, jobs_to_skip, job_params + ) + self.assertCountEqual( + jobs_to_do, + [ + "Style check", + "package_release", + "package_asan", + "package_ubsan", + "package_debug", + "package_msan", + "Stateless tests (asan)", + "Stateless tests flaky check (asan)", + "Stateless tests (msan)", + "Stateless tests (ubsan)", + "Stateless tests (debug)", + "Stateless tests (release)", + "Integration tests (release)", + "Integration tests (asan)", + "Integration tests flaky check (asan)", + ], + ) + + def test_options_applied_2(self): + self.maxDiff = None + ci_options = CiOptions.create_from_pr_message( + _TEST_BODY_3, update_from_api=False + ) + self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) + self.assertIsNone(ci_options.exclude_keywords) + jobs_to_do = list(JobNames) + jobs_to_skip = [] + job_params = {} + jobs_to_do, jobs_to_skip, job_params = ci_options.apply( + jobs_to_do, jobs_to_skip, job_params + ) + self.assertCountEqual( + jobs_to_do, + [ + "Style check", + "Integration tests (asan, analyzer)", + "package_release", + "Stateless tests (release, analyzer, s3, DatabaseReplicated)", + "package_asan", + ], + ) From 3bc91476290e46f140daff1af40043b794fef88a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 26 Mar 2024 20:56:39 +0100 Subject: [PATCH 139/154] Update 02916_move_partition_inactive_replica.sql --- .../0_stateless/02916_move_partition_inactive_replica.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql index ca153eea221..b0699539ac1 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -8,8 +8,8 @@ drop table if exists shard_1.from_1; drop table if exists shard_0.to; drop table if exists shard_1.to; -create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; -create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; +create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, shared_merge_tree_disable_merges_and_mutations_assignment=1; +create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, shared_merge_tree_disable_merges_and_mutations_assignment=1; system stop merges shard_0.from_1; system stop merges shard_1.from_1; From e5ba8e82a9fd705d0913e89848fa3e44880a2e80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 23:25:22 +0100 Subject: [PATCH 140/154] Tomorrow --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e576fb447c1..84e51c1efdf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,7 +6,7 @@ # 2024 Changelog -### ClickHouse release 24.3 LTS, 2024-03-26 +### ClickHouse release 24.3 LTS, 2024-03-27 #### Upgrade Notes * The setting `allow_experimental_analyzer` is enabled by default and it switches the query analysis to a new implementation, which has better compatibility and feature completeness. The feature "analyzer" is considered beta instead of experimental. You can turn the old behavior by setting the `compatibility` to `24.2` or disabling the `allow_experimental_analyzer` setting. Watch the [video on YouTube](https://www.youtube.com/watch?v=zhrOYQpgvkk). From 4069ff6cb04a77690f8d9272171864e50b47e08c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 26 Mar 2024 23:21:13 +0000 Subject: [PATCH 141/154] impl --- .../01193_metadata_loading.reference | 5 -- .../0_stateless/01193_metadata_loading.sh | 53 ------------------- 2 files changed, 58 deletions(-) delete mode 100644 tests/queries/0_stateless/01193_metadata_loading.reference delete mode 100755 tests/queries/0_stateless/01193_metadata_loading.sh diff --git a/tests/queries/0_stateless/01193_metadata_loading.reference b/tests/queries/0_stateless/01193_metadata_loading.reference deleted file mode 100644 index 9789cbf33ba..00000000000 --- a/tests/queries/0_stateless/01193_metadata_loading.reference +++ /dev/null @@ -1,5 +0,0 @@ -1000 0 2020-06-25 hello [1,2] [3,4] -1000 1 2020-06-26 word [10,20] [30,40] -ok -8000 0 2020-06-25 hello [1,2] [3,4] -8000 1 2020-06-26 word [10,20] [30,40] diff --git a/tests/queries/0_stateless/01193_metadata_loading.sh b/tests/queries/0_stateless/01193_metadata_loading.sh deleted file mode 100755 index 69178a93d42..00000000000 --- a/tests/queries/0_stateless/01193_metadata_loading.sh +++ /dev/null @@ -1,53 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-parallel, no-fasttest, no-s3-storage, no-sanitize-coverage - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Check that attaching a database with a large number of tables is not too slow. -# it is the worst way of making performance test, nevertheless it can detect significant slowdown and some other issues, that usually found by stress test - -db="test_01193_$RANDOM_$RANDOM_$RANDOM_$RANDOM" -tables=1000 -threads=10 -count_multiplier=1 -max_time_ms=1500 - -create_tables() { - $CLICKHOUSE_CLIENT -q "WITH - 'CREATE TABLE $db.table_$1_' AS create1, - ' (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=' AS create2, - ['Memory', 'File(CSV)', 'Log', 'StripeLog', 'MergeTree ORDER BY i'] AS engines, - 'INSERT INTO $db.table_$1_' AS insert1, - ' VALUES (0, ''2020-06-25'', ''hello'', [1, 2], [3, 4]), (1, ''2020-06-26'', ''word'', [10, 20], [30, 40])' AS insert2 - SELECT arrayStringConcat( - groupArray( - create1 || toString(number) || create2 || engines[1 + number % length(engines)] || ';\n' || - insert1 || toString(number) || insert2 - ), ';\n') FROM numbers($tables) SETTINGS max_bytes_before_external_group_by = 0 FORMAT TSVRaw;" | $CLICKHOUSE_CLIENT -nm -} - -$CLICKHOUSE_CLIENT -q "CREATE DATABASE $db" - -for i in $(seq 1 $threads); do - create_tables "$i" & -done -wait - -$CLICKHOUSE_CLIENT -q "CREATE TABLE $db.table_merge (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=Merge('$db', '^table_')" -$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM merge('$db', '^table_9') GROUP BY i, d, s, n.i, n.f ORDER BY i" - -for i in {1..50}; do - $CLICKHOUSE_CLIENT -q "DETACH DATABASE $db" - $CLICKHOUSE_CLIENT --query_profiler_real_time_period_ns=100000000 --query_profiler_cpu_time_period_ns=100000000 -q "ATTACH DATABASE $db" --query_id="$db-$i"; -done - -$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" -durations=$($CLICKHOUSE_CLIENT -q "SELECT groupArray(query_duration_ms) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2") -$CLICKHOUSE_CLIENT -q "SELECT 'durations', '$db', $durations FORMAT Null" -$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(arrayJoin($durations)) < $max_time_ms, 'ok', toString($durations))" - -$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i" - -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS $db" From 6732636222b3eedd6d0fe70358ba83083f84753b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 27 Mar 2024 11:29:24 +0100 Subject: [PATCH 142/154] Add tests to the broken list --- tests/analyzer_tech_debt.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 1b65e3f5e38..89af93f581b 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -3,3 +3,8 @@ 02354_vector_search_queries 02901_parallel_replicas_rollup 02999_scalar_subqueries_bug_2 +# Flaky list +01825_type_json_in_array +01414_mutations_and_errors_zookeeper +# Check after ConstantNode refactoring +02154_parser_backtracking From 104774c646e17ac38e6affb0b92262fa284ff1d5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 27 Mar 2024 11:39:33 +0100 Subject: [PATCH 143/154] Fix CI config --- tests/ci/test_ci_options.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 61aeec59581..7c87144a84d 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -113,9 +113,9 @@ class TestCIOptions(unittest.TestCase): jobs_to_do, [ "Style check", - "Integration tests (asan, analyzer)", + "Integration tests (asan, old analyzer)", "package_release", - "Stateless tests (release, analyzer, s3, DatabaseReplicated)", + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)", "package_asan", ], ) From 414bccc52763ced3ee8da50e45798400223a3523 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 27 Mar 2024 12:06:50 +0100 Subject: [PATCH 144/154] Revert "disallow LowCardinality input type for JSONExtract" (#61960) --- src/Functions/FunctionsJSON.h | 13 ++++++++++--- .../0_stateless/00918_json_functions.reference | 1 - tests/queries/0_stateless/00918_json_functions.sql | 2 -- .../02474_extract_fixedstring_from_json.reference | 7 ------- .../02474_extract_fixedstring_from_json.sql | 7 ------- 5 files changed, 10 insertions(+), 20 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 8a193785f87..2539fa1aeb4 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -348,7 +348,6 @@ public: String getName() const override { return Name::name; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { @@ -470,6 +469,9 @@ public: else return_type = json_return_type; + /// Top-level LowCardinality columns are processed outside JSON parser. + json_return_type = removeLowCardinality(json_return_type); + DataTypes argument_types; argument_types.reserve(arguments.size()); for (const auto & argument : arguments) @@ -865,9 +867,11 @@ struct JSONExtractTree explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } bool insertResultToColumn(IColumn & dest, const Element & element) override { - // For types other than string, delegate the insertion to JSONExtractRawImpl. - if (!element.isString()) + // If element is an object we delegate the insertion to JSONExtractRawImpl + if (element.isObject()) return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); + else if (!element.isString()) + return false; auto str = element.getString(); if (str.size() > fixed_length) @@ -1482,6 +1486,9 @@ public: // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) { + if (element.getObject().size() > fixed_length) + return false; + ColumnFixedString::Chars chars; WriteBufferFromVector buf(chars, AppendModeTag()); traverse(element, buf); diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 7b725111755..43b15ded93d 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -69,7 +69,6 @@ hello (3333.6,'test') (3333.6333333333,'test') (3333.6333333333,'test') -\N 123456.1234 Decimal(20, 4) 123456.1234 Decimal(20, 4) 123456789012345.12 Decimal(30, 4) diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index 61fcb21fcbd..e19dd17670e 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -81,7 +81,6 @@ SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Dec SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(10,1), b LowCardinality(String))'); SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); -SELECT JSONExtract(materialize('{"string_value":null}'), materialize('string_value'), 'LowCardinality(Nullable(String))'); SELECT JSONExtract('{"a":123456.123456}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456.123456"}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456789012345.12"}', 'a', 'Decimal(30, 4)') as a, toTypeName(a); @@ -327,4 +326,3 @@ SELECT JSONExtract('[]', JSONExtract('0', 'UInt256'), 'UInt256'); -- { serverErr SELECT '--show error: key of map type should be String'; SELECT JSONExtract('{"a": [100.0, 200], "b": [-100, 200.0, 300]}', 'Map(Int64, Array(Float64))'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT JSONExtract(materialize(toLowCardinality('{"string_value":null}')), materialize('string_value'), 'LowCardinality(Nullable(String))'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference index 21ddf5d3512..783d12fcf1a 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference @@ -8,10 +8,3 @@ \0\0\0\0\0 131231 131231 -1234 -1234 -{"b":131231} -\0\0\0\0 -1234567890 -18446744073709551615 --9223372036854775807 diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql index bbb9f55062b..cfc47e00cba 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql @@ -6,10 +6,3 @@ SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(5)'); SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(6)'); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(5))') FROM numbers(2); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(6))') FROM numbers(2); -SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))'); -SELECT JSONExtract(materialize('{"a": 131231, "b": "1234"}'), 'b', 'LowCardinality(FixedString(4))'); -SELECT JSONExtract(materialize('{"a": {"b": 131231} }'), 'a', 'LowCardinality(FixedString(12))'); -SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(4))'); -SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(10))'); -SELECT JSONExtract(materialize('{"a": 18446744073709551615}'), 'a', 'LowCardinality(FixedString(20))'); -SELECT JSONExtract(materialize('{"a": -9223372036854775807}'), 'a', 'LowCardinality(FixedString(20))'); From 5a40a871c80a18d387f44ecc63b110502fe44529 Mon Sep 17 00:00:00 2001 From: Geoff Genz Date: Wed, 27 Mar 2024 05:42:22 -0600 Subject: [PATCH 145/154] Fix Arrow format settings anchor --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a76bb01ce9e..14db25a31e4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2356,7 +2356,7 @@ You can select data from a ClickHouse table and save them into some file in the $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filename.arrow} ``` -### Arrow format settings {#parquet-format-settings} +### Arrow format settings {#arrow-format-settings} - [output_format_arrow_low_cardinality_as_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_low_cardinality_as_dictionary) - enable output ClickHouse LowCardinality type as Dictionary Arrow type. Default value - `false`. - [output_format_arrow_use_64_bit_indexes_for_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_use_64_bit_indexes_for_dictionary) - use 64-bit integer type for Dictionary indexes. Default value - `false`. From 4ba1078314ae109683d43c48d03366964fcd74e4 Mon Sep 17 00:00:00 2001 From: ILya Limarenko <67974432+I-Limar@users.noreply.github.com> Date: Wed, 27 Mar 2024 13:10:14 +0100 Subject: [PATCH 146/154] Update url-functions.md suggestion to fix the example for the cutToFirstSignificantSubdomainWithWWW function --- docs/en/sql-reference/functions/url-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index ac81815b47f..a0b0170721c 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -128,9 +128,9 @@ Returns the part of the domain that includes top-level subdomains up to the “f For example: -- `cutToFirstSignificantSubdomain('https://news.clickhouse.com.tr/') = 'clickhouse.com.tr'`. -- `cutToFirstSignificantSubdomain('www.tr') = 'www.tr'`. -- `cutToFirstSignificantSubdomain('tr') = ''`. +- `cutToFirstSignificantSubdomainWithWWW('https://news.clickhouse.com.tr/') = 'clickhouse.com.tr'`. +- `cutToFirstSignificantSubdomainWithWWW('www.tr') = 'www.tr'`. +- `cutToFirstSignificantSubdomainWithWWW('tr') = ''`. ### cutToFirstSignificantSubdomainCustom From 2daaa660c53be9ee2ab683fcfa477505ff8fe44d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 27 Mar 2024 13:43:49 +0100 Subject: [PATCH 147/154] Update input_format_parquet_allow_missing_columns default value Default value for input_format_parquet_allow_missing_columns was modified from 0 to 1, but the documentation was never updated. --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 477fbf94625..a865a956b82 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1367,7 +1367,7 @@ Default value: `1'000'000`. While importing data, when column is not found in schema default value will be used instead of error. -Disabled by default. +Enabled by default. ### input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference {#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference} From f7bdbaaacff2bd9b3b010cef207efcd9b08f3d0f Mon Sep 17 00:00:00 2001 From: Arnaud Rocher Date: Wed, 27 Mar 2024 13:46:58 +0100 Subject: [PATCH 148/154] Fix section link casing --- docs/en/sql-reference/table-functions/generate.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/generate.md b/docs/en/sql-reference/table-functions/generate.md index 3b9b077af49..a78015e9830 100644 --- a/docs/en/sql-reference/table-functions/generate.md +++ b/docs/en/sql-reference/table-functions/generate.md @@ -53,7 +53,7 @@ SELECT * FROM random; └──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘ ``` -In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure): +In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generaterandomstructure): ```sql SELECT * FROM generateRandom(generateRandomStructure(4, 101), 101) LIMIT 3; From beecfec1b4fc67fae3f5062bef8dbe07a5b78fc8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 27 Mar 2024 15:17:16 +0100 Subject: [PATCH 149/154] Update url-functions.md From d7187a172fec7ab691f7e7b05b2bcd55874aaea6 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 27 Mar 2024 15:24:16 +0000 Subject: [PATCH 150/154] CI: failover for job_url request from gh --- tests/ci/ci.py | 6 +++--- tests/ci/upload_result_helper.py | 8 +++++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 17552377971..3ba4e4845d7 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -798,7 +798,7 @@ class CiOptions: if match.startswith("job_"): if not res.ci_jobs: res.ci_jobs = [] - res.ci_jobs.append(match) + res.ci_jobs.append(match.removeprefix("job_")) elif match.startswith("ci_set_") and match in Labels: if not res.ci_sets: res.ci_sets = [] @@ -906,10 +906,10 @@ class CiOptions: # FIXME: to be removed in favor of include/exclude # 2. Handle "job_" tags if any if self.ci_jobs: - for tag in self.ci_jobs: + for job in self.ci_jobs: job_with_parents = CI_CONFIG.get_job_with_parents(job) print( - f"NOTE: CI Job's tag: [#job_{tag}], add jobs: [{job_with_parents}]" + f"NOTE: CI Job's tag: [#job_{job}], add jobs: [{job_with_parents}]" ) # always add requested job itself, even if it could be skipped jobs_to_do_requested.append(job_with_parents[0]) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index 9dca3fae1dc..7404ebc6335 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -92,13 +92,19 @@ def upload_results( else: raw_log_url = GITHUB_JOB_URL() + try: + job_url = GITHUB_JOB_URL() + except Exception: + print("ERROR: Failed to get job URL from GH API, job report will use run URL instead.") + job_url = GITHUB_RUN_URL + if test_results or not ready_report_url: html_report = create_test_html_report( check_name, test_results, raw_log_url, GITHUB_RUN_URL, - GITHUB_JOB_URL(), + job_url, branch_url, branch_name, commit_url, From a45439ed528fa557f77abf7cd250ddd44fdd364a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 27 Mar 2024 15:34:55 +0000 Subject: [PATCH 151/154] Automatic style fix --- tests/ci/upload_result_helper.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index 7404ebc6335..cb745131e0d 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -95,7 +95,9 @@ def upload_results( try: job_url = GITHUB_JOB_URL() except Exception: - print("ERROR: Failed to get job URL from GH API, job report will use run URL instead.") + print( + "ERROR: Failed to get job URL from GH API, job report will use run URL instead." + ) job_url = GITHUB_RUN_URL if test_results or not ready_report_url: From c9749ae586b5778e1ae1f06bed5d79e5df6d3e41 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Wed, 27 Mar 2024 17:07:59 +0000 Subject: [PATCH 152/154] CI: remove unnecessary job url for Mark release ready --- tests/ci/mark_release_ready.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py index 31415fef9c0..7ffb3c9a89b 100755 --- a/tests/ci/mark_release_ready.py +++ b/tests/ci/mark_release_ready.py @@ -5,7 +5,6 @@ import logging import os from commit_status_helper import get_commit, post_commit_status -from env_helper import GITHUB_JOB_URL from get_robot_token import get_best_robot_token from git_helper import commit as commit_arg from github_helper import GitHub @@ -33,7 +32,6 @@ def main(): help="if given, used instead of one from PRInfo", ) args = parser.parse_args() - url = "" description = "the release can be created from the commit, manually set" pr_info = None if not args.commit: @@ -41,7 +39,6 @@ def main(): if pr_info.event == pr_info.default_event: raise ValueError("neither launched from the CI nor commit is given") args.commit = pr_info.sha - url = GITHUB_JOB_URL() description = "the release can be created from the commit" args.token = args.token or get_best_robot_token() @@ -52,7 +49,7 @@ def main(): post_commit_status( commit, SUCCESS, - url, + "", description, RELEASE_READY_STATUS, pr_info, From 4250c5fb19e4bb554ba21216c2eae4476f115406 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 27 Mar 2024 18:48:14 +0100 Subject: [PATCH 153/154] Update autogenerated version to 24.4.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++---- .../StorageSystemContributors.generated.cpp | 30 +++++++++++++++++++ 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 2929c64ded8..26cb0eb23c6 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54484) +SET(VERSION_REVISION 54485) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 3) +SET(VERSION_MINOR 4) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 891689a41506d00aa169548f5b4a8774351242c4) -SET(VERSION_DESCRIBE v24.3.1.1-testing) -SET(VERSION_STRING 24.3.1.1) +SET(VERSION_GITHASH 2c5c589a882ceec35439650337b92db3e76f0081) +SET(VERSION_DESCRIBE v24.4.1.1-testing) +SET(VERSION_STRING 24.4.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 0ad22082863..6a66d8a7e19 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -245,6 +245,7 @@ const char * auto_contributors[] { "Brendan Cox", "Brett Hoerner", "Brian Hunter", + "Brokenice0415", "Bulat Gaifullin", "Camden Cheek", "Camilo Sierra", @@ -286,6 +287,7 @@ const char * auto_contributors[] { "Dale Mcdiarmid", "Dalitso Banda", "Dan Roscigno", + "Dan Wu", "DanRoscigno", "Dani Pozo", "Daniel Bershatsky", @@ -294,6 +296,7 @@ const char * auto_contributors[] { "Daniel Kutenin", "Daniel Pozo Escalona", "Daniel Qin", + "Daniil Ivanik", "Daniil Rubin", "Danila Kutenin", "Daniël van Eeden", @@ -634,6 +637,7 @@ const char * auto_contributors[] { "LiuCong", "LiuNeng", "LiuYangkuan", + "LiuYuan", "Lloyd-Pottiger", "Lopatin Konstantin", "Lorenzo Mangani", @@ -668,6 +672,7 @@ const char * auto_contributors[] { "Marek Vavruša", "Marek Vavruša", "Mariano Benítez Mulet", + "Marina Fathouat", "Mark Andreev", "Mark Frost", "Mark Needham", @@ -767,6 +772,7 @@ const char * auto_contributors[] { "N. Kolotov", "NIKITA MIKHAILOV", "Narek Galstyan", + "Nataly Merezhuk", "Natalya Chizhonkova", "Natasha Murashkina", "NeZeD [Mac Pro]", @@ -787,6 +793,7 @@ const char * auto_contributors[] { "Nikhil Raman", "Nikifor Seriakov", "Nikita", + "Nikita Fomichev", "Nikita Keba", "Nikita Lapkov", "Nikita Mikhailov", @@ -804,10 +811,12 @@ const char * auto_contributors[] { "Nikolay Degterinsky", "Nikolay Edigaryev", "Nikolay Kirsh", + "Nikolay Monkov", "Nikolay Semyachkin", "Nikolay Shcheglov", "Nikolay Vasiliev", "Nikolay Volosatov", + "Nikolay Yankin", "Nir Peled", "Nityananda Gohain", "Niu Zhaojie", @@ -831,11 +840,13 @@ const char * auto_contributors[] { "Orkhan Zeynalli", "Oskar Wojciski", "OuO", + "Oxide Computer Company", "PHO", "Pablo Alegre", "Pablo Marcos", "Pablo Musa", "Palash Goel", + "PapaToemmsn", "Paramtamtam", "Patrick Zippenfenig", "Paul Loyd", @@ -859,7 +870,9 @@ const char * auto_contributors[] { "Persiyanov Dmitriy Andreevich", "Pervakov Grigorii", "Pervakov Grigory", + "Peter", "Petr Vasilev", + "Pham Anh Tuan", "Philip Hallstrom", "Philippe Ombredanne", "PigInCloud", @@ -973,11 +986,14 @@ const char * auto_contributors[] { "SevaCode", "Seyed Mehrshad Hosseini", "Shane Andrade", + "Shanfeng Pang", "Shani Elharrar", "Shaun Struwig", "Sherry Wang", "Shoh Jahon", "Shri Bodas", + "Shuai li", + "Shubham Ranjan", "Sichen Zhao", "SiderZhang", "Sidorov Pavel", @@ -1139,6 +1155,7 @@ const char * auto_contributors[] { "Wangyang Guo", "Waterkin", "Weiqing Xu", + "William Schoeffel", "William Shallum", "Winter Zhang", "Xbitz29", @@ -1252,6 +1269,7 @@ const char * auto_contributors[] { "awesomeleo", "bakam412", "bbkas", + "beetelbrox", "benamazing", "benbiti", "bgranvea", @@ -1261,6 +1279,7 @@ const char * auto_contributors[] { "bkuschel", "blazerer", "bluebirddm", + "bluikko", "bo zeng", "bobrovskij artemij", "booknouse", @@ -1309,6 +1328,7 @@ const char * auto_contributors[] { "d.v.semenov", "dalei2019", "damozhaeva", + "danila-ermakov", "dankondr", "daoready", "darkkeks", @@ -1324,6 +1344,7 @@ const char * auto_contributors[] { "dheerajathrey", "dimarub2000", "dinosaur", + "divanik", "divanorama", "dkxiaohei", "dmi-feo", @@ -1454,6 +1475,7 @@ const char * auto_contributors[] { "joelynch", "johanngan", "johnnymatthews", + "josh-hildred", "jsc0218", "jthmath", "jun won", @@ -1595,6 +1617,7 @@ const char * auto_contributors[] { "nautaa", "ndchikin", "nellicus", + "nemonlou", "neng.liu", "never lee", "ni1l", @@ -1637,6 +1660,7 @@ const char * auto_contributors[] { "pufit", "pyos", "pzhdfy", + "qaziqarta", "qianlixiang", "qianmoQ", "qieqieplus", @@ -1684,8 +1708,10 @@ const char * auto_contributors[] { "sev7e0", "sevirov", "sfod", + "shabroo", "shangshujie", "shedx", + "shuai-xu", "shuchaome", "shuyang", "sichenzhao", @@ -1710,6 +1736,7 @@ const char * auto_contributors[] { "sundy-li", "sundyli", "sunlisheng", + "sunny", "sunny19930321", "svladykin", "tai", @@ -1733,6 +1760,7 @@ const char * auto_contributors[] { "tiger.yan", "timfursov", "tison", + "tomershafir", "tomtana", "topvisor", "tpanetti", @@ -1740,6 +1768,7 @@ const char * auto_contributors[] { "tyrionhuang", "ubuntu", "una", + "unashi", "unbyte", "unegare", "unknown", @@ -1882,6 +1911,7 @@ const char * auto_contributors[] { "董海镔", "袁焊忠", "谢磊", + "豪肥肥", "贾顺名(Jarvis)", "郭小龙", "陈小玉", From 52417e3d42f5ef2cc51680435c63eaca6a4da47c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 27 Mar 2024 18:02:02 +0000 Subject: [PATCH 154/154] Update version_date.tsv and changelogs after v24.3.1.2672-lts --- SECURITY.md | 8 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.3.1.2672-lts.md | 537 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 544 insertions(+), 8 deletions(-) create mode 100644 docs/changelogs/v24.3.1.2672-lts.md diff --git a/SECURITY.md b/SECURITY.md index 86578b188d8..4701f2ec70b 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,18 +13,16 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 24.3 | ✔️ | | 24.2 | ✔️ | | 24.1 | ✔️ | -| 23.12 | ✔️ | -| 23.11 | ❌ | -| 23.10 | ❌ | -| 23.9 | ❌ | +| 23.* | ❌ | | 23.8 | ✔️ | | 23.7 | ❌ | | 23.6 | ❌ | | 23.5 | ❌ | | 23.4 | ❌ | -| 23.3 | ✔️ | +| 23.3 | ❌ | | 23.2 | ❌ | | 23.1 | ❌ | | 22.* | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 17eee6d4287..19d90fa509e 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.2.2.71" +ARG VERSION="24.3.1.2672" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index bd5fa313adc..ee93b6eb8f0 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.2.2.71" +ARG VERSION="24.3.1.2672" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 256dcdc029f..e92823b686a 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.2.2.71" +ARG VERSION="24.3.1.2672" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v24.3.1.2672-lts.md b/docs/changelogs/v24.3.1.2672-lts.md new file mode 100644 index 00000000000..e5d008680a8 --- /dev/null +++ b/docs/changelogs/v24.3.1.2672-lts.md @@ -0,0 +1,537 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.3.1.2672-lts (2c5c589a882) FIXME as compared to v24.2.1.2248-stable (891689a4150) + +#### Backward Incompatible Change +* Don't allow to set max_parallel_replicas to 0 as it doesn't make sense. Setting it to 0 could lead to unexpected logical errors. Closes [#60140](https://github.com/ClickHouse/ClickHouse/issues/60140). [#60430](https://github.com/ClickHouse/ClickHouse/pull/60430) ([Kruglov Pavel](https://github.com/Avogar)). +* Change the column name from `duration_ms` to `duration_microseconds` in the `system.zookeeper` table to reflect the reality that the duration is in the microsecond resolution. [#60774](https://github.com/ClickHouse/ClickHouse/pull/60774) ([Duc Canh Le](https://github.com/canhld94)). +* Reject incoming INSERT queries in case when query-level settings `async_insert` and `deduplicate_blocks_in_dependent_materialized_views` are enabled together. This behaviour is controlled by a setting `throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert` and enabled by default. This is a continuation of https://github.com/ClickHouse/ClickHouse/pull/59699 needed to unblock https://github.com/ClickHouse/ClickHouse/pull/59915. [#60888](https://github.com/ClickHouse/ClickHouse/pull/60888) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Utility `clickhouse-copier` is moved to a separate repository on GitHub: https://github.com/ClickHouse/copier. It is no longer included in the bundle but is still available as a separate download. This closes: [#60734](https://github.com/ClickHouse/ClickHouse/issues/60734) This closes: [#60540](https://github.com/ClickHouse/ClickHouse/issues/60540) This closes: [#60250](https://github.com/ClickHouse/ClickHouse/issues/60250) This closes: [#52917](https://github.com/ClickHouse/ClickHouse/issues/52917) This closes: [#51140](https://github.com/ClickHouse/ClickHouse/issues/51140) This closes: [#47517](https://github.com/ClickHouse/ClickHouse/issues/47517) This closes: [#47189](https://github.com/ClickHouse/ClickHouse/issues/47189) This closes: [#46598](https://github.com/ClickHouse/ClickHouse/issues/46598) This closes: [#40257](https://github.com/ClickHouse/ClickHouse/issues/40257) This closes: [#36504](https://github.com/ClickHouse/ClickHouse/issues/36504) This closes: [#35485](https://github.com/ClickHouse/ClickHouse/issues/35485) This closes: [#33702](https://github.com/ClickHouse/ClickHouse/issues/33702) This closes: [#26702](https://github.com/ClickHouse/ClickHouse/issues/26702) ### Documentation entry for user-facing changes. [#61058](https://github.com/ClickHouse/ClickHouse/pull/61058) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* To increase compatibility with MySQL, function `locate` now accepts arguments `(needle, haystack[, start_pos])` by default. The previous behavior `(haystack, needle, [, start_pos])` can be restored by setting `function_locate_has_mysql_compatible_argument_order = 0`. [#61092](https://github.com/ClickHouse/ClickHouse/pull/61092) ([Robert Schulze](https://github.com/rschu1ze)). +* The obsolete in-memory data parts have been deprecated since version 23.5 and have not been supported since version 23.10. Now the remaining code is removed. Continuation of [#55186](https://github.com/ClickHouse/ClickHouse/issues/55186) and [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). It is unlikely that you have used in-memory data parts because they were available only before version 23.5 and only when you enabled them manually by specifying the corresponding SETTINGS for a MergeTree table. To check if you have in-memory data parts, run the following query: `SELECT part_type, count() FROM system.parts GROUP BY part_type ORDER BY part_type`. To disable the usage of in-memory data parts, do `ALTER TABLE ... MODIFY SETTING min_bytes_for_compact_part = DEFAULT, min_rows_for_compact_part = DEFAULT`. Before upgrading from old ClickHouse releases, first check that you don't have in-memory data parts. If there are in-memory data parts, disable them first, then wait while there are no in-memory data parts and continue the upgrade. [#61127](https://github.com/ClickHouse/ClickHouse/pull/61127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Forbid `SimpleAggregateFunction` in `ORDER BY` of `MergeTree` tables (like `AggregateFunction` is forbidden, but they are forbidden because they are not comparable) by default (use `allow_suspicious_primary_key` to allow them). [#61399](https://github.com/ClickHouse/ClickHouse/pull/61399) ([Azat Khuzhin](https://github.com/azat)). +* ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. This is controlled by the settings, `output_format_parquet_string_as_string`, `output_format_orc_string_as_string`, `output_format_arrow_string_as_string`. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases. Parquet/ORC/Arrow supports many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools lack support for the faster `lz4` compression method, that's why we set `zstd` by default. This is controlled by the settings `output_format_parquet_compression_method`, `output_format_orc_compression_method`, and `output_format_arrow_compression_method`. We changed the default to `zstd` for Parquet and ORC, but not Arrow (it is emphasized for low-level usages). [#61817](https://github.com/ClickHouse/ClickHouse/pull/61817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the new ClickHouse version, the functions `geoDistance`, `greatCircleDistance`, and `greatCircleAngle` will use 64-bit double precision floating point data type for internal calculations and return type if all the arguments are Float64. This closes [#58476](https://github.com/ClickHouse/ClickHouse/issues/58476). In previous versions, the function always used Float32. You can switch to the old behavior by setting `geo_distance_returns_float64_on_float64_arguments` to `false` or setting `compatibility` to `24.2` or earlier. [#61848](https://github.com/ClickHouse/ClickHouse/pull/61848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Topk/topkweighed support mode, which return count of values and it's error. [#54508](https://github.com/ClickHouse/ClickHouse/pull/54508) ([UnamedRus](https://github.com/UnamedRus)). +* Add generate_series as a table function. This function generates table with an arithmetic progression with natural numbers. [#59390](https://github.com/ClickHouse/ClickHouse/pull/59390) ([divanik](https://github.com/divanik)). +* Support reading and writing backups as tar archives. [#59535](https://github.com/ClickHouse/ClickHouse/pull/59535) ([josh-hildred](https://github.com/josh-hildred)). +* Implemented support for S3Express buckets. [#59965](https://github.com/ClickHouse/ClickHouse/pull/59965) ([Nikita Taranov](https://github.com/nickitat)). +* Allow to attach parts from a different disk * attach partition from the table on other disks using copy instead of hard link (such as instant table) * attach partition using copy when the hard link fails even on the same disk. [#60112](https://github.com/ClickHouse/ClickHouse/pull/60112) ([Unalian](https://github.com/Unalian)). +* Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60281](https://github.com/ClickHouse/ClickHouse/pull/60281) ([Shaun Struwig](https://github.com/Blargian)). +* Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. [#60420](https://github.com/ClickHouse/ClickHouse/pull/60420) ([豪肥肥](https://github.com/HowePa)). +* Add four properties to the `StorageMemory` (memory-engine) `min_bytes_to_keep, max_bytes_to_keep, min_rows_to_keep` and `max_rows_to_keep` - Add tests to reflect new changes - Update `memory.md` documentation - Add table `context` property to `MemorySink` to enable access to table parameter bounds. [#60612](https://github.com/ClickHouse/ClickHouse/pull/60612) ([Jake Bamrah](https://github.com/JakeBamrah)). +* Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60649](https://github.com/ClickHouse/ClickHouse/pull/60649) ([Robert Schulze](https://github.com/rschu1ze)). +* Separate limits on number of waiting and executing queries. Added new server setting `max_waiting_queries` that limits the number of queries waiting due to `async_load_databases`. Existing limits on number of executing queries no longer count waiting queries. [#61053](https://github.com/ClickHouse/ClickHouse/pull/61053) ([Sergei Trifonov](https://github.com/serxa)). +* Add support for `ATTACH PARTITION ALL`. [#61107](https://github.com/ClickHouse/ClickHouse/pull/61107) ([Kirill Nikiforov](https://github.com/allmazz)). +* Add a new function, `getClientHTTPHeader`. This closes [#54665](https://github.com/ClickHouse/ClickHouse/issues/54665). Co-authored with @lingtaolf. [#61820](https://github.com/ClickHouse/ClickHouse/pull/61820) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement +* Improve the performance of serialized aggregation method when involving multiple [nullable] columns. This is a general version of [#51399](https://github.com/ClickHouse/ClickHouse/issues/51399) that doesn't compromise on abstraction integrity. [#55809](https://github.com/ClickHouse/ClickHouse/pull/55809) ([Amos Bird](https://github.com/amosbird)). +* Lazy build join output to improve performance of ALL join. [#58278](https://github.com/ClickHouse/ClickHouse/pull/58278) ([LiuNeng](https://github.com/liuneng1994)). +* Improvements to aggregate functions ArgMin / ArgMax / any / anyLast / anyHeavy, as well as `ORDER BY {u8/u16/u32/u64/i8/i16/u32/i64) LIMIT 1` queries. [#58640](https://github.com/ClickHouse/ClickHouse/pull/58640) ([Raúl Marín](https://github.com/Algunenano)). +* Trivial optimize on column filter. Avoid those filter columns whoes underlying data type is not number being filtered with `result_size_hint = -1`. Peak memory can be reduced to 44% of the original in some cases. [#59698](https://github.com/ClickHouse/ClickHouse/pull/59698) ([李扬](https://github.com/taiyang-li)). +* If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting `primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns` with the value `0.9` by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. [#60255](https://github.com/ClickHouse/ClickHouse/pull/60255) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Execute multiIf function columnarly when result_type's underlying type is number. [#60384](https://github.com/ClickHouse/ClickHouse/pull/60384) ([李扬](https://github.com/taiyang-li)). +* Faster (almost 2x) mutexes (was slower due to ThreadFuzzer). [#60823](https://github.com/ClickHouse/ClickHouse/pull/60823) ([Azat Khuzhin](https://github.com/azat)). +* Move connection drain from prepare to work, and drain multiple connections in parallel. [#60845](https://github.com/ClickHouse/ClickHouse/pull/60845) ([lizhuoyu5](https://github.com/lzydmxy)). +* Optimize insertManyFrom of nullable number or nullable string. [#60846](https://github.com/ClickHouse/ClickHouse/pull/60846) ([李扬](https://github.com/taiyang-li)). +* Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). +* Operations with the filesystem cache will suffer less from the lock contention. [#61066](https://github.com/ClickHouse/ClickHouse/pull/61066) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Optimize ColumnString::replicate and prevent memcpySmallAllowReadWriteOverflow15Impl from being optimized to built-in memcpy. Close [#61074](https://github.com/ClickHouse/ClickHouse/issues/61074). ColumnString::replicate speeds up by 2.46x on x86-64. [#61075](https://github.com/ClickHouse/ClickHouse/pull/61075) ([李扬](https://github.com/taiyang-li)). +* 30x faster printing for 256-bit integers. [#61100](https://github.com/ClickHouse/ClickHouse/pull/61100) ([Raúl Marín](https://github.com/Algunenano)). +* If a query with a syntax error contained COLUMNS matcher with a regular expression, the regular expression was compiled each time during the parser's backtracking, instead of being compiled once. This was a fundamental error. The compiled regexp was put to AST. But the letter A in AST means "abstract" which means it should not contain heavyweight objects. Parts of AST can be created and discarded during parsing, including a large number of backtracking. This leads to slowness on the parsing side and consequently allows DoS by a readonly user. But the main problem is that it prevents progress in fuzzers. [#61543](https://github.com/ClickHouse/ClickHouse/pull/61543) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a new analyzer pass to optimize in single value. [#61564](https://github.com/ClickHouse/ClickHouse/pull/61564) ([LiuNeng](https://github.com/liuneng1994)). + +#### Improvement +* While running the MODIFY COLUMN query for materialized views, check the inner table's structure to ensure every column exists. [#47427](https://github.com/ClickHouse/ClickHouse/pull/47427) ([sunny](https://github.com/sunny19930321)). +* Added table `system.keywords` which contains all the keywords from parser. Mostly needed and will be used for better fuzzing and syntax highlighting. [#51808](https://github.com/ClickHouse/ClickHouse/pull/51808) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Ordinary database engine is deprecated. You will receive a warning in clickhouse-client if your server is using it. This closes [#52229](https://github.com/ClickHouse/ClickHouse/issues/52229). [#56942](https://github.com/ClickHouse/ClickHouse/pull/56942) ([shabroo](https://github.com/shabroo)). +* All zero copy locks related to a table have to be dropped when the table is dropped. The directory which contains these locks has to be removed also. [#57575](https://github.com/ClickHouse/ClickHouse/pull/57575) ([Sema Checherinda](https://github.com/CheSema)). +* Allow declaring enum in external table structure. [#57857](https://github.com/ClickHouse/ClickHouse/pull/57857) ([Duc Canh Le](https://github.com/canhld94)). +* Consider lightweight deleted rows when selecting parts to merge. [#58223](https://github.com/ClickHouse/ClickHouse/pull/58223) ([Zhuo Qiu](https://github.com/jewelzqiu)). +* This PR makes http/https connections reusable for all uses cases. Even when response is 3xx or 4xx. [#58845](https://github.com/ClickHouse/ClickHouse/pull/58845) ([Sema Checherinda](https://github.com/CheSema)). +* Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. [#59016](https://github.com/ClickHouse/ClickHouse/pull/59016) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like `_part_offset`. [#59033](https://github.com/ClickHouse/ClickHouse/pull/59033) ([Amos Bird](https://github.com/amosbird)). +* Add ability to skip read-only replicas for INSERT into Distributed engine (Controlled with `distributed_insert_skip_read_only_replicas` setting, by default OFF - backward compatible). [#59176](https://github.com/ClickHouse/ClickHouse/pull/59176) ([Azat Khuzhin](https://github.com/azat)). +* Instead using a constant key, now object storage generates key for determining remove objects capability. [#59495](https://github.com/ClickHouse/ClickHouse/pull/59495) ([Sema Checherinda](https://github.com/CheSema)). +* Add positional pread in libhdfs3. If you want to call positional read in libhdfs3, use the hdfsPread function in hdfs.h as follows. `tSize hdfsPread(hdfsFS fs, hdfsFile file, void * buffer, tSize length, tOffset position);`. [#59624](https://github.com/ClickHouse/ClickHouse/pull/59624) ([M1eyu](https://github.com/M1eyu2018)). +* Add asynchronous WriteBuffer for AzureBlobStorage similar to S3. [#59929](https://github.com/ClickHouse/ClickHouse/pull/59929) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Allow "local" as object storage type instead of "local_blob_storage". [#60165](https://github.com/ClickHouse/ClickHouse/pull/60165) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Improved overall usability of virtual columns. Now it is allowed to use virtual columns in `PREWHERE` (it's worthwhile for non-const virtual columns like `_part_offset`). Now a builtin documentation is available for virtual columns as a comment of column in `DESCRIBE` query with enabled setting `describe_include_virtual_columns`. [#60205](https://github.com/ClickHouse/ClickHouse/pull/60205) ([Anton Popov](https://github.com/CurtizJ)). +* Parallel flush of pending INSERT blocks of Distributed engine on `DETACH`/server shutdown and `SYSTEM FLUSH DISTRIBUTED` (Parallelism will work only if you have multi disk policy for table (like everything in Distributed engine right now)). [#60225](https://github.com/ClickHouse/ClickHouse/pull/60225) ([Azat Khuzhin](https://github.com/azat)). +* Filter setting is improper in `joinRightColumnsSwitchNullability`, resolve [#59625](https://github.com/ClickHouse/ClickHouse/issues/59625). [#60259](https://github.com/ClickHouse/ClickHouse/pull/60259) ([lgbo](https://github.com/lgbo-ustc)). +* Add a setting to force read-through cache for merges. [#60308](https://github.com/ClickHouse/ClickHouse/pull/60308) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Issue [#57598](https://github.com/ClickHouse/ClickHouse/issues/57598) mentions a variant behaviour regarding transaction handling. An issued COMMIT/ROLLBACK when no transaction is active is reported as an error contrary to MySQL behaviour. [#60338](https://github.com/ClickHouse/ClickHouse/pull/60338) ([PapaToemmsn](https://github.com/PapaToemmsn)). +* Added `none_only_active` mode for `distributed_ddl_output_mode` setting. [#60340](https://github.com/ClickHouse/ClickHouse/pull/60340) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make `/` redirect to the Play UI. [#60390](https://github.com/ClickHouse/ClickHouse/pull/60390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The advanced dashboard has slightly better colors for multi-line graphs. [#60391](https://github.com/ClickHouse/ClickHouse/pull/60391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Function `substring` now has a new alias `byteSlice`. [#60494](https://github.com/ClickHouse/ClickHouse/pull/60494) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed server setting `dns_cache_max_size` to `dns_cache_max_entries` to reduce ambiguity. [#60500](https://github.com/ClickHouse/ClickHouse/pull/60500) ([Kirill Nikiforov](https://github.com/allmazz)). +* `SHOW INDEX | INDEXES | INDICES | KEYS` no longer sorts by the primary key columns (which was unintuitive). [#60514](https://github.com/ClickHouse/ClickHouse/pull/60514) ([Robert Schulze](https://github.com/rschu1ze)). +* Keeper improvement: abort during startup if an invalid snapshot is detected to avoid data loss. [#60537](https://github.com/ClickHouse/ClickHouse/pull/60537) ([Antonio Andelic](https://github.com/antonio2368)). +* Added MergeTree read split ranges into intersecting and non intersecting fault injection using `merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_fault_probability` setting. [#60548](https://github.com/ClickHouse/ClickHouse/pull/60548) ([Maksim Kita](https://github.com/kitaisreal)). +* The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. [#60692](https://github.com/ClickHouse/ClickHouse/pull/60692) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* String types and Enums can be used in the same context, such as: arrays, UNION queries, conditional expressions. This closes [#60726](https://github.com/ClickHouse/ClickHouse/issues/60726). [#60727](https://github.com/ClickHouse/ClickHouse/pull/60727) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update tzdata to 2024a. [#60768](https://github.com/ClickHouse/ClickHouse/pull/60768) ([Raúl Marín](https://github.com/Algunenano)). +* Support files without format extension in Filesystem database. [#60795](https://github.com/ClickHouse/ClickHouse/pull/60795) ([Kruglov Pavel](https://github.com/Avogar)). +* Keeper improvement: support `leadership_expiry_ms` in Keeper's settings. [#60806](https://github.com/ClickHouse/ClickHouse/pull/60806) ([Brokenice0415](https://github.com/Brokenice0415)). +* Always infer exponential numbers in JSON formats regardless of the setting `input_format_try_infer_exponent_floats`. Add setting `input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects` that allows to use String type for ambiguous paths instead of an exception during named Tuples inference from JSON objects. [#60808](https://github.com/ClickHouse/ClickHouse/pull/60808) ([Kruglov Pavel](https://github.com/Avogar)). +* Add support for `START TRANSACTION` syntax typically used in MySQL syntax, resolving https://github.com/ClickHouse/ClickHouse/discussions/60865. [#60886](https://github.com/ClickHouse/ClickHouse/pull/60886) ([Zach Naimon](https://github.com/ArctypeZach)). +* Add a flag for SMJ to treat null as biggest/smallest. So the behavior can be compitable with other SQL systems, like Apache Spark. [#60896](https://github.com/ClickHouse/ClickHouse/pull/60896) ([loudongfeng](https://github.com/loudongfeng)). +* Clickhouse version has been added to docker labels. Closes [#54224](https://github.com/ClickHouse/ClickHouse/issues/54224). [#60949](https://github.com/ClickHouse/ClickHouse/pull/60949) ([Nikolay Monkov](https://github.com/nikmonkov)). +* Add a setting `parallel_replicas_allow_in_with_subquery = 1` which allows subqueries for IN work with parallel replicas. [#60950](https://github.com/ClickHouse/ClickHouse/pull/60950) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* DNSResolver shuffles set of resolved IPs. [#60965](https://github.com/ClickHouse/ClickHouse/pull/60965) ([Sema Checherinda](https://github.com/CheSema)). +* Support detect output format by file exctension in `clickhouse-client` and `clickhouse-local`. [#61036](https://github.com/ClickHouse/ClickHouse/pull/61036) ([豪肥肥](https://github.com/HowePa)). +* Check memory limit update periodically. [#61049](https://github.com/ClickHouse/ClickHouse/pull/61049) ([Han Fei](https://github.com/hanfei1991)). +* Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. [#61096](https://github.com/ClickHouse/ClickHouse/pull/61096) ([Azat Khuzhin](https://github.com/azat)). +* Add the function `toUInt128OrZero`, which was missed by mistake (the mistake is related to https://github.com/ClickHouse/ClickHouse/pull/945). The compatibility aliases `FROM_UNIXTIME` and `DATE_FORMAT` (they are not ClickHouse-native and only exist for MySQL compatibility) have been made case insensitive, as expected for SQL-compatibility aliases. [#61114](https://github.com/ClickHouse/ClickHouse/pull/61114) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improvements for the access checks, allowing to revoke of unpossessed rights in case the target user doesn't have the revoking grants either. Example: ```sql GRANT SELECT ON *.* TO user1; REVOKE SELECT ON system.* FROM user1;. [#61115](https://github.com/ClickHouse/ClickHouse/pull/61115) ([pufit](https://github.com/pufit)). +* Fix an error in previeous opt: https://github.com/ClickHouse/ClickHouse/pull/59698: remove break to make sure the first filtered column has minimum size cc @jsc0218. [#61145](https://github.com/ClickHouse/ClickHouse/pull/61145) ([李扬](https://github.com/taiyang-li)). +* Fix `has()` function with `Nullable` column (fixes [#60214](https://github.com/ClickHouse/ClickHouse/issues/60214)). [#61249](https://github.com/ClickHouse/ClickHouse/pull/61249) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Now it's possible to specify attribute `merge="true"` in config substitutions for subtrees ``. In case this attribute specified, clickhouse will merge subtree with existing configuration, otherwise default behavior is append new content to configuration. [#61299](https://github.com/ClickHouse/ClickHouse/pull/61299) ([alesapin](https://github.com/alesapin)). +* Add async metrics for virtual memory mappings: VMMaxMapCount & VMNumMaps. Closes [#60662](https://github.com/ClickHouse/ClickHouse/issues/60662). [#61354](https://github.com/ClickHouse/ClickHouse/pull/61354) ([Tuan Pham Anh](https://github.com/tuanpavn)). +* Use `temporary_files_codec` setting in all places where we create temporary data, for example external memory sorting and external memory GROUP BY. Before it worked only in `partial_merge` JOIN algorithm. [#61456](https://github.com/ClickHouse/ClickHouse/pull/61456) ([Maksim Kita](https://github.com/kitaisreal)). +* Remove duplicated check `containing_part.empty()`, It's already being checked here: https://github.com/ClickHouse/ClickHouse/blob/1296dac3c7e47670872c15e3f5e58f869e0bd2f2/src/Storages/MergeTree/MergeTreeData.cpp#L6141. [#61467](https://github.com/ClickHouse/ClickHouse/pull/61467) ([William Schoeffel](https://github.com/wiledusc)). +* Add a new setting `max_parser_backtracks` which allows to limit the complexity of query parsing. [#61502](https://github.com/ClickHouse/ClickHouse/pull/61502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support parallel reading for azure blob storage. [#61503](https://github.com/ClickHouse/ClickHouse/pull/61503) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Less contention during dynamic resize of filesystem cache. [#61524](https://github.com/ClickHouse/ClickHouse/pull/61524) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disallow sharded mode of StorageS3 queue, because it will be rewritten. [#61537](https://github.com/ClickHouse/ClickHouse/pull/61537) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed typo: from `use_leagcy_max_level` to `use_legacy_max_level`. [#61545](https://github.com/ClickHouse/ClickHouse/pull/61545) ([William Schoeffel](https://github.com/wiledusc)). +* Remove some duplicate entries in blob_storage_log. [#61622](https://github.com/ClickHouse/ClickHouse/pull/61622) ([YenchangChan](https://github.com/YenchangChan)). +* Enable `allow_experimental_analyzer` setting by default. [#61652](https://github.com/ClickHouse/ClickHouse/pull/61652) ([Dmitry Novik](https://github.com/novikd)). +* Added `current_user` function as a compatibility alias for MySQL. [#61770](https://github.com/ClickHouse/ClickHouse/pull/61770) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Use managed identity for backups IO when using Azure Blob Storage. Add a setting to prevent ClickHouse from attempting to create a non-existent container, which requires permissions at the storage account level. [#61785](https://github.com/ClickHouse/ClickHouse/pull/61785) ([Daniel Pozo Escalona](https://github.com/danipozo)). +* Enable `output_format_pretty_row_numbers` by default. It is better for usability. [#61791](https://github.com/ClickHouse/ClickHouse/pull/61791) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the previous version, some numbers in Pretty formats were not pretty enough. [#61794](https://github.com/ClickHouse/ClickHouse/pull/61794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A long value in Pretty formats won't be cut if it is the single value in the resultset, such as in the result of the `SHOW CREATE TABLE` query. [#61795](https://github.com/ClickHouse/ClickHouse/pull/61795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Similarly to `clickhouse-local`, `clickhouse-client` will accept the `--output-format` option as a synonym to the `--format` option. This closes [#59848](https://github.com/ClickHouse/ClickHouse/issues/59848). [#61797](https://github.com/ClickHouse/ClickHouse/pull/61797) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If stdout is a terminal and the output format is not specified, `clickhouse-client` and similar tools will use `PrettyCompact` by default, similarly to the interactive mode. `clickhouse-client` and `clickhouse-local` will handle command line arguments for input and output formats in a unified fashion. This closes [#61272](https://github.com/ClickHouse/ClickHouse/issues/61272). [#61800](https://github.com/ClickHouse/ClickHouse/pull/61800) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Underscore digit groups in Pretty formats for better readability. This is controlled by a new setting, `output_format_pretty_highlight_digit_groups`. [#61802](https://github.com/ClickHouse/ClickHouse/pull/61802) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add ability to override initial INSERT SETTINGS via SYSTEM FLUSH DISTRIBUTED. [#61832](https://github.com/ClickHouse/ClickHouse/pull/61832) ([Azat Khuzhin](https://github.com/azat)). +* Fixed grammar from "a" to "the" in the warning message. There is only one Atomic engine, so it should be "to the new Atomic engine" instead of "to a new Atomic engine". [#61952](https://github.com/ClickHouse/ClickHouse/pull/61952) ([shabroo](https://github.com/shabroo)). + +#### Build/Testing/Packaging Improvement +* Update sccache to the latest version; significantly reduce images size by reshaking the dependency trees; use the latest working odbc driver. [#59953](https://github.com/ClickHouse/ClickHouse/pull/59953) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update python related style checkers. Continue the [#50174](https://github.com/ClickHouse/ClickHouse/issues/50174). [#60408](https://github.com/ClickHouse/ClickHouse/pull/60408) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Upgrade `prqlc` to 0.11.3. [#60616](https://github.com/ClickHouse/ClickHouse/pull/60616) ([Maximilian Roos](https://github.com/max-sixty)). +* Attach gdb to running fuzzer process. [#60654](https://github.com/ClickHouse/ClickHouse/pull/60654) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use explicit template instantiation more aggressively. Get rid of templates in favor of overloaded functions in some places. [#60730](https://github.com/ClickHouse/ClickHouse/pull/60730) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* The real-time query profiler now works on AArch64. In previous versions, it worked only when a program didn't spend time inside a syscall. [#60807](https://github.com/ClickHouse/ClickHouse/pull/60807) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ... Too big translation unit in `Aggregator`. [#61211](https://github.com/ClickHouse/ClickHouse/pull/61211) ([lgbo](https://github.com/lgbo-ustc)). +* Fixed flakiness of 01603_insert_select_too_many_parts test. Closes [#61158](https://github.com/ClickHouse/ClickHouse/issues/61158). [#61259](https://github.com/ClickHouse/ClickHouse/pull/61259) ([Ilya Yatsishin](https://github.com/qoega)). +* Now it possible to use `chassert(expression, comment)` in the codebase. [#61263](https://github.com/ClickHouse/ClickHouse/pull/61263) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Teach the fuzzer to use other numeric types. [#61317](https://github.com/ClickHouse/ClickHouse/pull/61317) ([Raúl Marín](https://github.com/Algunenano)). +* Increase memory limit for coverage builds. [#61405](https://github.com/ClickHouse/ClickHouse/pull/61405) ([Raúl Marín](https://github.com/Algunenano)). +* Add generic query text fuzzer in `clickhouse-local`. [#61508](https://github.com/ClickHouse/ClickHouse/pull/61508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix function execution over const and LowCardinality with GROUP BY const for analyzer [#59986](https://github.com/ClickHouse/ClickHouse/pull/59986) ([Azat Khuzhin](https://github.com/azat)). +* Fix finished_mutations_to_keep=0 for MergeTree (as docs says 0 is to keep everything) [#60031](https://github.com/ClickHouse/ClickHouse/pull/60031) ([Azat Khuzhin](https://github.com/azat)). +* PartsSplitter invalid ranges for the same part [#60041](https://github.com/ClickHouse/ClickHouse/pull/60041) ([Maksim Kita](https://github.com/kitaisreal)). +* Azure Blob Storage : Fix issues endpoint and prefix [#60251](https://github.com/ClickHouse/ClickHouse/pull/60251) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* fix LRUResource Cache bug (Hive cache) [#60262](https://github.com/ClickHouse/ClickHouse/pull/60262) ([shanfengp](https://github.com/Aed-p)). +* Force reanalysis if parallel replicas changed [#60362](https://github.com/ClickHouse/ClickHouse/pull/60362) ([Raúl Marín](https://github.com/Algunenano)). +* Fix usage of plain metadata type with new disks configuration option [#60396](https://github.com/ClickHouse/ClickHouse/pull/60396) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike [#60451](https://github.com/ClickHouse/ClickHouse/pull/60451) ([Kruglov Pavel](https://github.com/Avogar)). +* Try to avoid calculation of scalar subqueries for CREATE TABLE. [#60464](https://github.com/ClickHouse/ClickHouse/pull/60464) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix deadlock in parallel parsing when lots of rows are skipped due to errors [#60516](https://github.com/ClickHouse/ClickHouse/pull/60516) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix_max_query_size_for_kql_compound_operator: [#60534](https://github.com/ClickHouse/ClickHouse/pull/60534) ([Yong Wang](https://github.com/kashwy)). +* Keeper fix: add timeouts when waiting for commit logs [#60544](https://github.com/ClickHouse/ClickHouse/pull/60544) ([Antonio Andelic](https://github.com/antonio2368)). +* Reduce the number of read rows from `system.numbers` [#60546](https://github.com/ClickHouse/ClickHouse/pull/60546) ([JackyWoo](https://github.com/JackyWoo)). +* Don't output number tips for date types [#60577](https://github.com/ClickHouse/ClickHouse/pull/60577) ([Raúl Marín](https://github.com/Algunenano)). +* Fix reading from MergeTree with non-deterministic functions in filter [#60586](https://github.com/ClickHouse/ClickHouse/pull/60586) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix logical error on bad compatibility setting value type [#60596](https://github.com/ClickHouse/ClickHouse/pull/60596) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix inconsistent aggregate function states in mixed x86-64 / ARM clusters [#60610](https://github.com/ClickHouse/ClickHouse/pull/60610) ([Harry Lee](https://github.com/HarryLeeIBM)). +* fix(prql): Robust panic handler [#60615](https://github.com/ClickHouse/ClickHouse/pull/60615) ([Maximilian Roos](https://github.com/max-sixty)). +* Fix `intDiv` for decimal and date arguments [#60672](https://github.com/ClickHouse/ClickHouse/pull/60672) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix: expand CTE in alter modify query [#60682](https://github.com/ClickHouse/ClickHouse/pull/60682) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix system.parts for non-Atomic/Ordinary database engine (i.e. Memory) [#60689](https://github.com/ClickHouse/ClickHouse/pull/60689) ([Azat Khuzhin](https://github.com/azat)). +* Fix "Invalid storage definition in metadata file" for parameterized views [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Fix buffer overflow in CompressionCodecMultiple [#60731](https://github.com/ClickHouse/ClickHouse/pull/60731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove nonsense from SQL/JSON [#60738](https://github.com/ClickHouse/ClickHouse/pull/60738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove wrong sanitize checking in aggregate function quantileGK [#60740](https://github.com/ClickHouse/ClickHouse/pull/60740) ([李扬](https://github.com/taiyang-li)). +* Fix insert-select + insert_deduplication_token bug by setting streams to 1 [#60745](https://github.com/ClickHouse/ClickHouse/pull/60745) ([Jordi Villar](https://github.com/jrdi)). +* Prevent setting custom metadata headers on unsupported multipart upload operations [#60748](https://github.com/ClickHouse/ClickHouse/pull/60748) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Fix toStartOfInterval [#60763](https://github.com/ClickHouse/ClickHouse/pull/60763) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix crash in arrayEnumerateRanked [#60764](https://github.com/ClickHouse/ClickHouse/pull/60764) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash when using input() in INSERT SELECT JOIN [#60765](https://github.com/ClickHouse/ClickHouse/pull/60765) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash with different allow_experimental_analyzer value in subqueries [#60770](https://github.com/ClickHouse/ClickHouse/pull/60770) ([Dmitry Novik](https://github.com/novikd)). +* Remove recursion when reading from S3 [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix possible stuck on error in HashedDictionaryParallelLoader [#60926](https://github.com/ClickHouse/ClickHouse/pull/60926) ([vdimir](https://github.com/vdimir)). +* Fix async RESTORE with Replicated database [#60934](https://github.com/ClickHouse/ClickHouse/pull/60934) ([Antonio Andelic](https://github.com/antonio2368)). +* fix csv format not support tuple [#60994](https://github.com/ClickHouse/ClickHouse/pull/60994) ([shuai.xu](https://github.com/shuai-xu)). +* Fix deadlock in async inserts to `Log` tables via native protocol [#61055](https://github.com/ClickHouse/ClickHouse/pull/61055) ([Anton Popov](https://github.com/CurtizJ)). +* Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary [#61196](https://github.com/ClickHouse/ClickHouse/pull/61196) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix multiple bugs in groupArraySorted [#61203](https://github.com/ClickHouse/ClickHouse/pull/61203) ([Raúl Marín](https://github.com/Algunenano)). +* Fix Keeper reconfig for standalone binary [#61233](https://github.com/ClickHouse/ClickHouse/pull/61233) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix usage of session_token in S3 engine [#61234](https://github.com/ClickHouse/ClickHouse/pull/61234) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible incorrect result of aggregate function `uniqExact` [#61257](https://github.com/ClickHouse/ClickHouse/pull/61257) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bugs in show database [#61269](https://github.com/ClickHouse/ClickHouse/pull/61269) ([Raúl Marín](https://github.com/Algunenano)). +* Fix logical error in RabbitMQ storage with MATERIALIZED columns [#61320](https://github.com/ClickHouse/ClickHouse/pull/61320) ([vdimir](https://github.com/vdimir)). +* Fix CREATE OR REPLACE DICTIONARY [#61356](https://github.com/ClickHouse/ClickHouse/pull/61356) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix crash in ObjectJson parsing array with nulls [#61364](https://github.com/ClickHouse/ClickHouse/pull/61364) ([vdimir](https://github.com/vdimir)). +* Fix ATTACH query with external ON CLUSTER [#61365](https://github.com/ClickHouse/ClickHouse/pull/61365) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix consecutive keys optimization for nullable keys [#61393](https://github.com/ClickHouse/ClickHouse/pull/61393) ([Anton Popov](https://github.com/CurtizJ)). +* fix issue of actions dag split [#61458](https://github.com/ClickHouse/ClickHouse/pull/61458) ([Raúl Marín](https://github.com/Algunenano)). +* Fix finishing a failed RESTORE [#61466](https://github.com/ClickHouse/ClickHouse/pull/61466) ([Vitaly Baranov](https://github.com/vitlibar)). +* Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings [#61468](https://github.com/ClickHouse/ClickHouse/pull/61468) ([Raúl Marín](https://github.com/Algunenano)). +* Allow queuing in restore pool [#61475](https://github.com/ClickHouse/ClickHouse/pull/61475) ([Nikita Taranov](https://github.com/nickitat)). +* Fix bug when reading system.parts using UUID (issue 61220). [#61479](https://github.com/ClickHouse/ClickHouse/pull/61479) ([Dan Wu](https://github.com/wudanzy)). +* Fix ALTER QUERY MODIFY SQL SECURITY [#61480](https://github.com/ClickHouse/ClickHouse/pull/61480) ([pufit](https://github.com/pufit)). +* Fix crash in window view [#61526](https://github.com/ClickHouse/ClickHouse/pull/61526) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `repeat` with non native integers [#61527](https://github.com/ClickHouse/ClickHouse/pull/61527) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix client `-s` argument [#61530](https://github.com/ClickHouse/ClickHouse/pull/61530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Reset part level upon attach from disk on MergeTree [#61536](https://github.com/ClickHouse/ClickHouse/pull/61536) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix crash in arrayPartialReverseSort [#61539](https://github.com/ClickHouse/ClickHouse/pull/61539) ([Raúl Marín](https://github.com/Algunenano)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix addDays cause an error when used datetime64 [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* disallow LowCardinality input type for JSONExtract [#61617](https://github.com/ClickHouse/ClickHouse/pull/61617) ([Julia Kartseva](https://github.com/jkartseva)). +* Fix `system.part_log` for async insert with deduplication [#61620](https://github.com/ClickHouse/ClickHouse/pull/61620) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix Non-ready set for system.parts. [#61666](https://github.com/ClickHouse/ClickHouse/pull/61666) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Don't allow the same expression in ORDER BY with and without WITH FILL [#61667](https://github.com/ClickHouse/ClickHouse/pull/61667) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix actual_part_name for REPLACE_RANGE (`Entry actual part isn't empty yet`) [#61675](https://github.com/ClickHouse/ClickHouse/pull/61675) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix columns after executing MODIFY QUERY for a materialized view with internal table [#61734](https://github.com/ClickHouse/ClickHouse/pull/61734) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix crash in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). +* Fix RANGE frame is not supported for Nullable columns. [#61766](https://github.com/ClickHouse/ClickHouse/pull/61766) ([YuanLiu](https://github.com/ditgittube)). +* Revert "Revert "Fix bug when reading system.parts using UUID (issue 61220)."" [#61779](https://github.com/ClickHouse/ClickHouse/pull/61779) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). + +#### CI Fix or Improvement (changelog entry is not required) + +* Decoupled changes from [#60408](https://github.com/ClickHouse/ClickHouse/issues/60408). [#60553](https://github.com/ClickHouse/ClickHouse/pull/60553) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Eliminates the need to provide input args to docker server jobs to clean yml files. [#60602](https://github.com/ClickHouse/ClickHouse/pull/60602) ([Max K.](https://github.com/maxknv)). +* Debug and fix markreleaseready. [#60611](https://github.com/ClickHouse/ClickHouse/pull/60611) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix build_report job so that it's defined by ci_config only (not yml file). [#60613](https://github.com/ClickHouse/ClickHouse/pull/60613) ([Max K.](https://github.com/maxknv)). +* Do not await ci pending jobs on release branches decrease wait timeout to fit into gh job timeout. [#60652](https://github.com/ClickHouse/ClickHouse/pull/60652) ([Max K.](https://github.com/maxknv)). +* Set limited number of builds for "special build check" report in backports. [#60850](https://github.com/ClickHouse/ClickHouse/pull/60850) ([Max K.](https://github.com/maxknv)). +* ... [#60935](https://github.com/ClickHouse/ClickHouse/pull/60935) ([Max K.](https://github.com/maxknv)). +* ... [#60947](https://github.com/ClickHouse/ClickHouse/pull/60947) ([Max K.](https://github.com/maxknv)). +* ... [#60952](https://github.com/ClickHouse/ClickHouse/pull/60952) ([Max K.](https://github.com/maxknv)). +* ... [#60958](https://github.com/ClickHouse/ClickHouse/pull/60958) ([Max K.](https://github.com/maxknv)). +* ... [#61022](https://github.com/ClickHouse/ClickHouse/pull/61022) ([Max K.](https://github.com/maxknv)). +* Just a preparation for the merge queue support. [#61099](https://github.com/ClickHouse/ClickHouse/pull/61099) ([Max K.](https://github.com/maxknv)). +* ... [#61133](https://github.com/ClickHouse/ClickHouse/pull/61133) ([Max K.](https://github.com/maxknv)). +* In PRs: - run typos, aspell check - always - run pylint, mypy - only if py file(s) changed in PRs - run basic source files style check - only if not all changes in py files. [#61148](https://github.com/ClickHouse/ClickHouse/pull/61148) ([Max K.](https://github.com/maxknv)). +* ... [#61172](https://github.com/ClickHouse/ClickHouse/pull/61172) ([Max K.](https://github.com/maxknv)). +* ... [#61183](https://github.com/ClickHouse/ClickHouse/pull/61183) ([Han Fei](https://github.com/hanfei1991)). +* ... [#61185](https://github.com/ClickHouse/ClickHouse/pull/61185) ([Max K.](https://github.com/maxknv)). +* TBD. [#61197](https://github.com/ClickHouse/ClickHouse/pull/61197) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ... [#61214](https://github.com/ClickHouse/ClickHouse/pull/61214) ([Max K.](https://github.com/maxknv)). +* ... [#61441](https://github.com/ClickHouse/ClickHouse/pull/61441) ([Max K.](https://github.com/maxknv)). +* ![Screenshot_20240323_025055](https://github.com/ClickHouse/ClickHouse/assets/18581488/ccaab212-a1d3-4dfb-8d56-b1991760b6bf). [#61801](https://github.com/ClickHouse/ClickHouse/pull/61801) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ... [#61877](https://github.com/ClickHouse/ClickHouse/pull/61877) ([Max K.](https://github.com/maxknv)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Revert "Use `MergeTree` as a default table engine""'. [#60524](https://github.com/ClickHouse/ClickHouse/pull/60524) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Revert "Support resource request canceling""'. [#60558](https://github.com/ClickHouse/ClickHouse/pull/60558) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Add `toMillisecond` function"'. [#60644](https://github.com/ClickHouse/ClickHouse/pull/60644) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Synchronize parsers"'. [#60759](https://github.com/ClickHouse/ClickHouse/pull/60759) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Fix wacky primary key sorting in `SHOW INDEX`"'. [#60898](https://github.com/ClickHouse/ClickHouse/pull/60898) ([Antonio Andelic](https://github.com/antonio2368)). +* NO CL ENTRY: 'Revert "CI: make style check faster"'. [#61142](https://github.com/ClickHouse/ClickHouse/pull/61142) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Don't allow to set max_parallel_replicas to 0 as it doesn't make sense"'. [#61200](https://github.com/ClickHouse/ClickHouse/pull/61200) ([Kruglov Pavel](https://github.com/Avogar)). +* NO CL ENTRY: 'Revert "Fix usage of session_token in S3 engine"'. [#61359](https://github.com/ClickHouse/ClickHouse/pull/61359) ([Antonio Andelic](https://github.com/antonio2368)). +* NO CL ENTRY: 'Revert "Revert "Fix usage of session_token in S3 engine""'. [#61362](https://github.com/ClickHouse/ClickHouse/pull/61362) ([Kruglov Pavel](https://github.com/Avogar)). +* NO CL ENTRY: 'Reorder hidden and shown checks in comment, change url of Mergeable check'. [#61373](https://github.com/ClickHouse/ClickHouse/pull/61373) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Remove unnecessary layers from clickhouse/cctools'. [#61374](https://github.com/ClickHouse/ClickHouse/pull/61374) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Revert "Updated format settings references in the docs (datetime.md)"'. [#61435](https://github.com/ClickHouse/ClickHouse/pull/61435) ([Kruglov Pavel](https://github.com/Avogar)). +* NO CL ENTRY: 'Revert "CI: ARM integration tests: disable tests with HDFS "'. [#61449](https://github.com/ClickHouse/ClickHouse/pull/61449) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "Analyzer: Fix virtual columns in StorageMerge"'. [#61518](https://github.com/ClickHouse/ClickHouse/pull/61518) ([Antonio Andelic](https://github.com/antonio2368)). +* NO CL ENTRY: 'Revert "Revert "Analyzer: Fix virtual columns in StorageMerge""'. [#61528](https://github.com/ClickHouse/ClickHouse/pull/61528) ([Dmitry Novik](https://github.com/novikd)). +* NO CL ENTRY: 'Improve build_download_helper'. [#61592](https://github.com/ClickHouse/ClickHouse/pull/61592) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* NO CL ENTRY: 'Revert "Un-flake `test_undrop_query`"'. [#61668](https://github.com/ClickHouse/ClickHouse/pull/61668) ([Robert Schulze](https://github.com/rschu1ze)). +* NO CL ENTRY: 'Fix flaky tests (stateless, integration)'. [#61816](https://github.com/ClickHouse/ClickHouse/pull/61816) ([Nikita Fomichev](https://github.com/fm4v)). +* NO CL ENTRY: 'Better usability of "expect" tests: less trouble with running directly'. [#61818](https://github.com/ClickHouse/ClickHouse/pull/61818) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix flaky `02122_parallel_formatting_Template`"'. [#61868](https://github.com/ClickHouse/ClickHouse/pull/61868) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Add --now option to enable and start the service" #job_Install_packages_amd64'. [#61878](https://github.com/ClickHouse/ClickHouse/pull/61878) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "disallow LowCardinality input type for JSONExtract"'. [#61960](https://github.com/ClickHouse/ClickHouse/pull/61960) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Improve query performance in case of very small blocks [#58879](https://github.com/ClickHouse/ClickHouse/pull/58879) ([Azat Khuzhin](https://github.com/azat)). +* Analyzer: fixes for JOIN columns resolution [#59007](https://github.com/ClickHouse/ClickHouse/pull/59007) ([vdimir](https://github.com/vdimir)). +* Fix race on `Context::async_insert_queue` [#59082](https://github.com/ClickHouse/ClickHouse/pull/59082) ([Alexander Tokmakov](https://github.com/tavplubix)). +* CI: support batch specification in commit message [#59738](https://github.com/ClickHouse/ClickHouse/pull/59738) ([Max K.](https://github.com/maxknv)). +* Update storing-data.md [#60024](https://github.com/ClickHouse/ClickHouse/pull/60024) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make max_insert_delayed_streams_for_parallel_write actually work [#60079](https://github.com/ClickHouse/ClickHouse/pull/60079) ([alesapin](https://github.com/alesapin)). +* Analyzer: support join using column from select list [#60182](https://github.com/ClickHouse/ClickHouse/pull/60182) ([vdimir](https://github.com/vdimir)). +* test for [#60223](https://github.com/ClickHouse/ClickHouse/issues/60223) [#60258](https://github.com/ClickHouse/ClickHouse/pull/60258) ([Denny Crane](https://github.com/den-crane)). +* Analyzer: Refactor execution name for ConstantNode [#60313](https://github.com/ClickHouse/ClickHouse/pull/60313) ([Dmitry Novik](https://github.com/novikd)). +* Fix database iterator waiting code [#60314](https://github.com/ClickHouse/ClickHouse/pull/60314) ([Sergei Trifonov](https://github.com/serxa)). +* QueryCache: Don't acquire the query count mutex if not necessary [#60348](https://github.com/ClickHouse/ClickHouse/pull/60348) ([zhongyuankai](https://github.com/zhongyuankai)). +* Fix bugfix check (due to unknown commit_logs_cache_size_threshold) [#60375](https://github.com/ClickHouse/ClickHouse/pull/60375) ([Azat Khuzhin](https://github.com/azat)). +* Enable testing with `io_uring` back [#60383](https://github.com/ClickHouse/ClickHouse/pull/60383) ([Nikita Taranov](https://github.com/nickitat)). +* Analyzer - improve hiding secret arguments. [#60386](https://github.com/ClickHouse/ClickHouse/pull/60386) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* CI: make workflow yml abstract [#60421](https://github.com/ClickHouse/ClickHouse/pull/60421) ([Max K.](https://github.com/maxknv)). +* Improve test test_reload_clusters_config [#60426](https://github.com/ClickHouse/ClickHouse/pull/60426) ([Kruglov Pavel](https://github.com/Avogar)). +* Revert "Revert "Merge pull request [#56864](https://github.com/ClickHouse/ClickHouse/issues/56864) from ClickHouse/broken-projections-better-handling"" [#60452](https://github.com/ClickHouse/ClickHouse/pull/60452) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not check to and from files existence in metadata_storage because it does not see uncommitted changes [#60462](https://github.com/ClickHouse/ClickHouse/pull/60462) ([Alexander Gololobov](https://github.com/davenger)). +* Fix option ambiguous in `clickhouse-local` [#60475](https://github.com/ClickHouse/ClickHouse/pull/60475) ([豪肥肥](https://github.com/HowePa)). +* Fix: test_parallel_replicas_custom_key_load_balancing [#60485](https://github.com/ClickHouse/ClickHouse/pull/60485) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix: progress bar for *Cluster table functions [#60491](https://github.com/ClickHouse/ClickHouse/pull/60491) ([Igor Nikonov](https://github.com/devcrafter)). +* Analyzer: Support different ObjectJSON on shards [#60497](https://github.com/ClickHouse/ClickHouse/pull/60497) ([Dmitry Novik](https://github.com/novikd)). +* Cancel PipelineExecutor properly in case of exception in spawnThreads [#60499](https://github.com/ClickHouse/ClickHouse/pull/60499) ([Kruglov Pavel](https://github.com/Avogar)). +* Refactor StorageSystemOneBlock [#60510](https://github.com/ClickHouse/ClickHouse/pull/60510) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Simple cleanup while fixing progress bar [#60513](https://github.com/ClickHouse/ClickHouse/pull/60513) ([Igor Nikonov](https://github.com/devcrafter)). +* PullingAsyncPipelineExecutor cleanup [#60515](https://github.com/ClickHouse/ClickHouse/pull/60515) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix bad error message [#60518](https://github.com/ClickHouse/ClickHouse/pull/60518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Synchronize Access [#60519](https://github.com/ClickHouse/ClickHouse/pull/60519) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Synchronize metrics and Keeper [#60520](https://github.com/ClickHouse/ClickHouse/pull/60520) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enforce clang-tidy in `programs/` and `utils/` headers [#60521](https://github.com/ClickHouse/ClickHouse/pull/60521) ([Robert Schulze](https://github.com/rschu1ze)). +* Synchronize parsers [#60522](https://github.com/ClickHouse/ClickHouse/pull/60522) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix a bunch of clang-tidy warnings in headers [#60523](https://github.com/ClickHouse/ClickHouse/pull/60523) ([Robert Schulze](https://github.com/rschu1ze)). +* General sanity in function `seriesOutliersDetectTukey` [#60535](https://github.com/ClickHouse/ClickHouse/pull/60535) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update Chinese document for max_query_size, max_parser_depth and optimize_functions_to_subcolumns [#60541](https://github.com/ClickHouse/ClickHouse/pull/60541) ([Alex Cheng](https://github.com/Alex-Cheng)). +* Userspace page cache again [#60552](https://github.com/ClickHouse/ClickHouse/pull/60552) ([Michael Kolupaev](https://github.com/al13n321)). +* Traverse shadow directory for system.remote_data_paths [#60585](https://github.com/ClickHouse/ClickHouse/pull/60585) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Add test for [#58906](https://github.com/ClickHouse/ClickHouse/issues/58906) [#60597](https://github.com/ClickHouse/ClickHouse/pull/60597) ([Raúl Marín](https://github.com/Algunenano)). +* Use python zipfile to have x-platform idempotent lambda packages [#60603](https://github.com/ClickHouse/ClickHouse/pull/60603) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* tests: suppress data-race in librdkafka statistics code [#60604](https://github.com/ClickHouse/ClickHouse/pull/60604) ([Azat Khuzhin](https://github.com/azat)). +* Update version after release [#60605](https://github.com/ClickHouse/ClickHouse/pull/60605) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v24.2.1.2248-stable [#60607](https://github.com/ClickHouse/ClickHouse/pull/60607) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Addition to changelog [#60609](https://github.com/ClickHouse/ClickHouse/pull/60609) ([Anton Popov](https://github.com/CurtizJ)). +* internal: Refine rust prql code [#60617](https://github.com/ClickHouse/ClickHouse/pull/60617) ([Maximilian Roos](https://github.com/max-sixty)). +* fix(rust): Fix skim's panic handler [#60621](https://github.com/ClickHouse/ClickHouse/pull/60621) ([Maximilian Roos](https://github.com/max-sixty)). +* Resubmit "Analyzer: compute ALIAS columns right after reading" [#60641](https://github.com/ClickHouse/ClickHouse/pull/60641) ([vdimir](https://github.com/vdimir)). +* Analyzer: Fix bug with join_use_nulls and PREWHERE [#60655](https://github.com/ClickHouse/ClickHouse/pull/60655) ([vdimir](https://github.com/vdimir)). +* Add test for [#59891](https://github.com/ClickHouse/ClickHouse/issues/59891) [#60657](https://github.com/ClickHouse/ClickHouse/pull/60657) ([Raúl Marín](https://github.com/Algunenano)). +* Fix missed entries in system.part_log in case of fetch preferred over merges/mutations [#60659](https://github.com/ClickHouse/ClickHouse/pull/60659) ([Azat Khuzhin](https://github.com/azat)). +* Always apply first minmax index among available skip indices [#60675](https://github.com/ClickHouse/ClickHouse/pull/60675) ([Igor Nikonov](https://github.com/devcrafter)). +* Remove bad test `02152_http_external_tables_memory_tracking` [#60690](https://github.com/ClickHouse/ClickHouse/pull/60690) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix questionable behavior in the `parseDateTimeBestEffort` function. [#60691](https://github.com/ClickHouse/ClickHouse/pull/60691) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky checks [#60694](https://github.com/ClickHouse/ClickHouse/pull/60694) ([Azat Khuzhin](https://github.com/azat)). +* Resubmit http_external_tables_memory_tracking test [#60695](https://github.com/ClickHouse/ClickHouse/pull/60695) ([Azat Khuzhin](https://github.com/azat)). +* Fix bugfix and upgrade checks (due to "Unknown handler type 'redirect'" error) [#60696](https://github.com/ClickHouse/ClickHouse/pull/60696) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_grant_and_revoke/test.py::test_grant_all_on_table (after syncing with cloud) [#60699](https://github.com/ClickHouse/ClickHouse/pull/60699) ([Azat Khuzhin](https://github.com/azat)). +* Remove unit test for ColumnObject [#60709](https://github.com/ClickHouse/ClickHouse/pull/60709) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve unit tests [#60710](https://github.com/ClickHouse/ClickHouse/pull/60710) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix scheduler fairness test [#60712](https://github.com/ClickHouse/ClickHouse/pull/60712) ([Sergei Trifonov](https://github.com/serxa)). +* Do not retry queries if container is down in integration tests (resubmit) [#60714](https://github.com/ClickHouse/ClickHouse/pull/60714) ([Azat Khuzhin](https://github.com/azat)). +* Mark one setting as obsolete [#60715](https://github.com/ClickHouse/ClickHouse/pull/60715) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix a test with Analyzer [#60723](https://github.com/ClickHouse/ClickHouse/pull/60723) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Two tests are fixed with Analyzer [#60724](https://github.com/ClickHouse/ClickHouse/pull/60724) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove old code [#60728](https://github.com/ClickHouse/ClickHouse/pull/60728) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove more code from LIVE VIEW [#60729](https://github.com/ClickHouse/ClickHouse/pull/60729) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `test_keeper_back_to_back/test.py::test_concurrent_watches` [#60749](https://github.com/ClickHouse/ClickHouse/pull/60749) ([Antonio Andelic](https://github.com/antonio2368)). +* Catch exceptions on finalize in `InterserverIOHTTPHandler` [#60769](https://github.com/ClickHouse/ClickHouse/pull/60769) ([Antonio Andelic](https://github.com/antonio2368)). +* Reduce flakiness of 02932_refreshable_materialized_views [#60771](https://github.com/ClickHouse/ClickHouse/pull/60771) ([Michael Kolupaev](https://github.com/al13n321)). +* Use 64-bit capabilities if available [#60775](https://github.com/ClickHouse/ClickHouse/pull/60775) ([Azat Khuzhin](https://github.com/azat)). +* Include multiline logs in fuzzer fatal.log report [#60796](https://github.com/ClickHouse/ClickHouse/pull/60796) ([Raúl Marín](https://github.com/Algunenano)). +* Add missing clone calls related to compression [#60810](https://github.com/ClickHouse/ClickHouse/pull/60810) ([Raúl Marín](https://github.com/Algunenano)). +* New private runners [#60811](https://github.com/ClickHouse/ClickHouse/pull/60811) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Move userspace page cache settings to the correct section of SettingsChangeHistory.h [#60812](https://github.com/ClickHouse/ClickHouse/pull/60812) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v23.8.10.43-lts [#60851](https://github.com/ClickHouse/ClickHouse/pull/60851) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix fuzzer report [#60853](https://github.com/ClickHouse/ClickHouse/pull/60853) ([Raúl Marín](https://github.com/Algunenano)). +* Update version_date.tsv and changelogs after v23.3.20.27-lts [#60857](https://github.com/ClickHouse/ClickHouse/pull/60857) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Refactor OptimizeDateOrDateTimeConverterWithPreimageVisitor [#60875](https://github.com/ClickHouse/ClickHouse/pull/60875) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Fix race in PageCache [#60878](https://github.com/ClickHouse/ClickHouse/pull/60878) ([Michael Kolupaev](https://github.com/al13n321)). +* Small changes in async inserts code [#60885](https://github.com/ClickHouse/ClickHouse/pull/60885) ([Nikita Taranov](https://github.com/nickitat)). +* Remove useless verbose logging from AWS library [#60921](https://github.com/ClickHouse/ClickHouse/pull/60921) ([alesapin](https://github.com/alesapin)). +* Throw on query timeout in ZooKeeperRetries [#60922](https://github.com/ClickHouse/ClickHouse/pull/60922) ([Antonio Andelic](https://github.com/antonio2368)). +* Bring clickhouse-test changes from private [#60924](https://github.com/ClickHouse/ClickHouse/pull/60924) ([Raúl Marín](https://github.com/Algunenano)). +* Add debug info to exceptions in `IMergeTreeDataPart::checkConsistency()` [#60981](https://github.com/ClickHouse/ClickHouse/pull/60981) ([Nikita Taranov](https://github.com/nickitat)). +* Fix a typo [#60987](https://github.com/ClickHouse/ClickHouse/pull/60987) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Replace some header includes with forward declarations [#61003](https://github.com/ClickHouse/ClickHouse/pull/61003) ([Amos Bird](https://github.com/amosbird)). +* Speed up cctools building [#61011](https://github.com/ClickHouse/ClickHouse/pull/61011) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix ASTRenameQuery::clone [#61013](https://github.com/ClickHouse/ClickHouse/pull/61013) ([vdimir](https://github.com/vdimir)). +* Update README.md [#61021](https://github.com/ClickHouse/ClickHouse/pull/61021) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix TableFunctionExecutable::skipAnalysisForArguments [#61037](https://github.com/ClickHouse/ClickHouse/pull/61037) ([Dmitry Novik](https://github.com/novikd)). +* Fix: parallel replicas with PREWHERE (ubsan) [#61052](https://github.com/ClickHouse/ClickHouse/pull/61052) ([Igor Nikonov](https://github.com/devcrafter)). +* Fast fix tests. [#61056](https://github.com/ClickHouse/ClickHouse/pull/61056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `test_placement_info` [#61057](https://github.com/ClickHouse/ClickHouse/pull/61057) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* Fix: parallel replicas with CTEs, crash in EXPLAIN SYNTAX with analyzer [#61059](https://github.com/ClickHouse/ClickHouse/pull/61059) ([Igor Nikonov](https://github.com/devcrafter)). +* Debug fuzzer failures [#61062](https://github.com/ClickHouse/ClickHouse/pull/61062) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add regression tests for fixed issues [#61076](https://github.com/ClickHouse/ClickHouse/pull/61076) ([Antonio Andelic](https://github.com/antonio2368)). +* Analyzer: Fix 01244_optimize_distributed_group_by_sharding_key [#61089](https://github.com/ClickHouse/ClickHouse/pull/61089) ([Dmitry Novik](https://github.com/novikd)). +* Use global scalars cache with analyzer [#61104](https://github.com/ClickHouse/ClickHouse/pull/61104) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix removing is_active node after re-creation [#61105](https://github.com/ClickHouse/ClickHouse/pull/61105) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update 02962_system_sync_replica_lightweight_from_modifier.sh [#61110](https://github.com/ClickHouse/ClickHouse/pull/61110) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Simplify bridges [#61118](https://github.com/ClickHouse/ClickHouse/pull/61118) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* update cppkafka to v0.4.1 [#61119](https://github.com/ClickHouse/ClickHouse/pull/61119) ([Ilya Golshtein](https://github.com/ilejn)). +* CI: add wf class in ci_config [#61122](https://github.com/ClickHouse/ClickHouse/pull/61122) ([Max K.](https://github.com/maxknv)). +* QueryFuzzer: replace element randomly when AST part buffer is full [#61124](https://github.com/ClickHouse/ClickHouse/pull/61124) ([Tomer Shafir](https://github.com/tomershafir)). +* CI: make style check fast [#61125](https://github.com/ClickHouse/ClickHouse/pull/61125) ([Max K.](https://github.com/maxknv)). +* Better gitignore [#61128](https://github.com/ClickHouse/ClickHouse/pull/61128) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix something strange [#61129](https://github.com/ClickHouse/ClickHouse/pull/61129) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update check-large-objects.sh to be language neutral [#61130](https://github.com/ClickHouse/ClickHouse/pull/61130) ([Dan Wu](https://github.com/wudanzy)). +* Throw memory limit exceptions to avoid OOM in some places [#61132](https://github.com/ClickHouse/ClickHouse/pull/61132) ([alesapin](https://github.com/alesapin)). +* Fix test_distributed_directory_monitor_split_batch_on_failure flakienss [#61136](https://github.com/ClickHouse/ClickHouse/pull/61136) ([Azat Khuzhin](https://github.com/azat)). +* Fix llvm symbolizer on CI [#61147](https://github.com/ClickHouse/ClickHouse/pull/61147) ([Azat Khuzhin](https://github.com/azat)). +* Some clang-tidy fixes [#61150](https://github.com/ClickHouse/ClickHouse/pull/61150) ([Robert Schulze](https://github.com/rschu1ze)). +* Revive "Less contention in the cache, part 2" [#61152](https://github.com/ClickHouse/ClickHouse/pull/61152) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Enable black back [#61159](https://github.com/ClickHouse/ClickHouse/pull/61159) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* CI: fix nightly job issue [#61160](https://github.com/ClickHouse/ClickHouse/pull/61160) ([Max K.](https://github.com/maxknv)). +* Split `RangeHashedDictionary` [#61162](https://github.com/ClickHouse/ClickHouse/pull/61162) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Remove a few templates from Aggregator.cpp [#61171](https://github.com/ClickHouse/ClickHouse/pull/61171) ([Raúl Marín](https://github.com/Algunenano)). +* Avoid some logical errors in experimantal Object type [#61173](https://github.com/ClickHouse/ClickHouse/pull/61173) ([Kruglov Pavel](https://github.com/Avogar)). +* Update ReadSettings.h [#61174](https://github.com/ClickHouse/ClickHouse/pull/61174) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: ARM integration tests: disable tests with HDFS [#61182](https://github.com/ClickHouse/ClickHouse/pull/61182) ([Max K.](https://github.com/maxknv)). +* Disable sanitizers with 02784_parallel_replicas_automatic_decision_join [#61184](https://github.com/ClickHouse/ClickHouse/pull/61184) ([Raúl Marín](https://github.com/Algunenano)). +* Fix `02887_mutations_subcolumns` test flakiness [#61198](https://github.com/ClickHouse/ClickHouse/pull/61198) ([Nikita Taranov](https://github.com/nickitat)). +* Make variant tests a bit faster [#61199](https://github.com/ClickHouse/ClickHouse/pull/61199) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix strange log message [#61206](https://github.com/ClickHouse/ClickHouse/pull/61206) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix 01603_insert_select_too_many_parts flakiness [#61218](https://github.com/ClickHouse/ClickHouse/pull/61218) ([Azat Khuzhin](https://github.com/azat)). +* Make every style-checker runner types scaling-out very quickly [#61231](https://github.com/ClickHouse/ClickHouse/pull/61231) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve `test_failed_mutations` [#61235](https://github.com/ClickHouse/ClickHouse/pull/61235) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix `test_merge_tree_load_parts/test.py::test_merge_tree_load_parts_corrupted` [#61236](https://github.com/ClickHouse/ClickHouse/pull/61236) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* fix `forget_partition` test [#61237](https://github.com/ClickHouse/ClickHouse/pull/61237) ([Sergei Trifonov](https://github.com/serxa)). +* Print more info in `02572_system_logs_materialized_views_ignore_errors` to debug [#61246](https://github.com/ClickHouse/ClickHouse/pull/61246) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix runtime error in AST Fuzzer [#61248](https://github.com/ClickHouse/ClickHouse/pull/61248) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Add retries to `02908_many_requests_to_system_replicas` [#61253](https://github.com/ClickHouse/ClickHouse/pull/61253) ([Nikita Taranov](https://github.com/nickitat)). +* Followup fix ASTRenameQuery::clone [#61254](https://github.com/ClickHouse/ClickHouse/pull/61254) ([vdimir](https://github.com/vdimir)). +* Disable test 02998_primary_key_skip_columns.sql in sanitizer builds as it can be slow [#61256](https://github.com/ClickHouse/ClickHouse/pull/61256) ([Kruglov Pavel](https://github.com/Avogar)). +* Update curl to curl with data race fix [#61264](https://github.com/ClickHouse/ClickHouse/pull/61264) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix `01417_freeze_partition_verbose` [#61266](https://github.com/ClickHouse/ClickHouse/pull/61266) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Free memory earlier in inserts [#61267](https://github.com/ClickHouse/ClickHouse/pull/61267) ([Anton Popov](https://github.com/CurtizJ)). +* Fixing test_build_sets_from_multiple_threads/test.py::test_set [#61286](https://github.com/ClickHouse/ClickHouse/pull/61286) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Analyzer: Fix virtual columns in StorageMerge [#61298](https://github.com/ClickHouse/ClickHouse/pull/61298) ([Dmitry Novik](https://github.com/novikd)). +* Fix 01952_optimize_distributed_group_by_sharding_key with analyzer. [#61301](https://github.com/ClickHouse/ClickHouse/pull/61301) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix data race in poco tcp server [#61309](https://github.com/ClickHouse/ClickHouse/pull/61309) ([Sema Checherinda](https://github.com/CheSema)). +* Don't use default cluster in test test_distibuted_settings [#61314](https://github.com/ClickHouse/ClickHouse/pull/61314) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix false positive assertion in cache [#61319](https://github.com/ClickHouse/ClickHouse/pull/61319) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix test test_input_format_parallel_parsing_memory_tracking [#61322](https://github.com/ClickHouse/ClickHouse/pull/61322) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix 01761_cast_to_enum_nullable with analyzer. [#61323](https://github.com/ClickHouse/ClickHouse/pull/61323) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add zookeeper retries for exists check in forcefullyRemoveBrokenOutdatedPartFromZooKeeper [#61324](https://github.com/ClickHouse/ClickHouse/pull/61324) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Minor changes in stress and fuzzer reports [#61333](https://github.com/ClickHouse/ClickHouse/pull/61333) ([Raúl Marín](https://github.com/Algunenano)). +* Un-flake `test_undrop_query` [#61348](https://github.com/ClickHouse/ClickHouse/pull/61348) ([Robert Schulze](https://github.com/rschu1ze)). +* Tiny improvement for replication.lib [#61361](https://github.com/ClickHouse/ClickHouse/pull/61361) ([alesapin](https://github.com/alesapin)). +* Fix bugfix check (due to "unknown object storage type: azure") [#61363](https://github.com/ClickHouse/ClickHouse/pull/61363) ([Azat Khuzhin](https://github.com/azat)). +* Fix `01599_multiline_input_and_singleline_comments` 3 minute wait [#61371](https://github.com/ClickHouse/ClickHouse/pull/61371) ([Sergei Trifonov](https://github.com/serxa)). +* Terminate EC2 on spot event if runner isn't running [#61377](https://github.com/ClickHouse/ClickHouse/pull/61377) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Try fix docs check [#61378](https://github.com/ClickHouse/ClickHouse/pull/61378) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix `heap-use-after-free` for Merge table with alias [#61380](https://github.com/ClickHouse/ClickHouse/pull/61380) ([Antonio Andelic](https://github.com/antonio2368)). +* Disable `optimize_rewrite_sum_if_to_count_if` if return type is nullable (new analyzer) [#61389](https://github.com/ClickHouse/ClickHouse/pull/61389) ([Antonio Andelic](https://github.com/antonio2368)). +* Analyzer: Fix planner context for subquery in StorageMerge [#61392](https://github.com/ClickHouse/ClickHouse/pull/61392) ([Dmitry Novik](https://github.com/novikd)). +* Fix `test_failed_async_inserts` [#61394](https://github.com/ClickHouse/ClickHouse/pull/61394) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix test test_system_clusters_actual_information flakiness [#61395](https://github.com/ClickHouse/ClickHouse/pull/61395) ([Kruglov Pavel](https://github.com/Avogar)). +* Remove default cluster from default config from test config [#61396](https://github.com/ClickHouse/ClickHouse/pull/61396) ([Raúl Marín](https://github.com/Algunenano)). +* Enable clang-tidy in headers [#61406](https://github.com/ClickHouse/ClickHouse/pull/61406) ([Robert Schulze](https://github.com/rschu1ze)). +* Add sanity check for poll_max_batch_size FileLog setting [#61408](https://github.com/ClickHouse/ClickHouse/pull/61408) ([Kruglov Pavel](https://github.com/Avogar)). +* ThreadFuzzer: randomize sleep time [#61410](https://github.com/ClickHouse/ClickHouse/pull/61410) ([Tomer Shafir](https://github.com/tomershafir)). +* Update version_date.tsv and changelogs after v23.8.11.28-lts [#61416](https://github.com/ClickHouse/ClickHouse/pull/61416) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.21.26-lts [#61418](https://github.com/ClickHouse/ClickHouse/pull/61418) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v24.1.7.18-stable [#61419](https://github.com/ClickHouse/ClickHouse/pull/61419) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v24.2.2.71-stable [#61420](https://github.com/ClickHouse/ClickHouse/pull/61420) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.12.5.81-stable [#61421](https://github.com/ClickHouse/ClickHouse/pull/61421) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Restore automerge for approved PRs [#61433](https://github.com/ClickHouse/ClickHouse/pull/61433) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Disable broken SonarCloud [#61434](https://github.com/ClickHouse/ClickHouse/pull/61434) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix `01599_multiline_input_and_singleline_comments` properly [#61440](https://github.com/ClickHouse/ClickHouse/pull/61440) ([Sergei Trifonov](https://github.com/serxa)). +* Convert test 02998_system_dns_cache_table to smoke and mirrors [#61443](https://github.com/ClickHouse/ClickHouse/pull/61443) ([vdimir](https://github.com/vdimir)). +* Check boundaries for some settings in parallel replicas [#61455](https://github.com/ClickHouse/ClickHouse/pull/61455) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use SHARD_LOAD_QUEUE_BACKLOG for dictionaries in tests [#61462](https://github.com/ClickHouse/ClickHouse/pull/61462) ([vdimir](https://github.com/vdimir)). +* Split `02125_lz4_compression_bug` [#61465](https://github.com/ClickHouse/ClickHouse/pull/61465) ([Antonio Andelic](https://github.com/antonio2368)). +* Correctly process last stacktrace in `postprocess-traces.pl` [#61470](https://github.com/ClickHouse/ClickHouse/pull/61470) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix test `test_polymorphic_parts` [#61477](https://github.com/ClickHouse/ClickHouse/pull/61477) ([Anton Popov](https://github.com/CurtizJ)). +* A definitive guide to CAST [#61491](https://github.com/ClickHouse/ClickHouse/pull/61491) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Minor rename in FileCache [#61494](https://github.com/ClickHouse/ClickHouse/pull/61494) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove useless code [#61498](https://github.com/ClickHouse/ClickHouse/pull/61498) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix fuzzers [#61499](https://github.com/ClickHouse/ClickHouse/pull/61499) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update jdbc.md [#61506](https://github.com/ClickHouse/ClickHouse/pull/61506) ([San](https://github.com/santrancisco)). +* Fix error in clickhouse-client [#61507](https://github.com/ClickHouse/ClickHouse/pull/61507) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix clang-tidy build [#61519](https://github.com/ClickHouse/ClickHouse/pull/61519) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix infinite loop in function `hop` [#61523](https://github.com/ClickHouse/ClickHouse/pull/61523) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve tests 00159_parallel_formatting_* to to avoid timeouts [#61532](https://github.com/ClickHouse/ClickHouse/pull/61532) ([Kruglov Pavel](https://github.com/Avogar)). +* Refactoring of reading from compact parts [#61535](https://github.com/ClickHouse/ClickHouse/pull/61535) ([Anton Popov](https://github.com/CurtizJ)). +* Don't run 01459_manual_write_to_replicas in debug build as it's too slow [#61538](https://github.com/ClickHouse/ClickHouse/pull/61538) ([Kruglov Pavel](https://github.com/Avogar)). +* CI: ARM integration test - skip hdfs, kerberos, kafka [#61542](https://github.com/ClickHouse/ClickHouse/pull/61542) ([Max K.](https://github.com/maxknv)). +* More logging for loading of tables [#61546](https://github.com/ClickHouse/ClickHouse/pull/61546) ([Sergei Trifonov](https://github.com/serxa)). +* Fixing 01584_distributed_buffer_cannot_find_column with analyzer. [#61550](https://github.com/ClickHouse/ClickHouse/pull/61550) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Wait for done mutation with more logs and asserts [#61554](https://github.com/ClickHouse/ClickHouse/pull/61554) ([alesapin](https://github.com/alesapin)). +* Fix read_rows count with external group by [#61555](https://github.com/ClickHouse/ClickHouse/pull/61555) ([Alexander Tokmakov](https://github.com/tavplubix)). +* queries-file should be used to specify file [#61557](https://github.com/ClickHouse/ClickHouse/pull/61557) ([danila-ermakov](https://github.com/danila-ermakov)). +* Fix `02481_async_insert_dedup_token` [#61568](https://github.com/ClickHouse/ClickHouse/pull/61568) ([Antonio Andelic](https://github.com/antonio2368)). +* Add a comment after [#61458](https://github.com/ClickHouse/ClickHouse/issues/61458) [#61580](https://github.com/ClickHouse/ClickHouse/pull/61580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix clickhouse-test client option and CLICKHOUSE_URL_PARAMS interference [#61596](https://github.com/ClickHouse/ClickHouse/pull/61596) ([vdimir](https://github.com/vdimir)). +* CI: remove compose files from integration test docker [#61597](https://github.com/ClickHouse/ClickHouse/pull/61597) ([Max K.](https://github.com/maxknv)). +* Fix 01244_optimize_distributed_group_by_sharding_key by ordering output [#61602](https://github.com/ClickHouse/ClickHouse/pull/61602) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove some tests from analyzer_tech_debt [#61603](https://github.com/ClickHouse/ClickHouse/pull/61603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Reduce header dependencies [#61604](https://github.com/ClickHouse/ClickHouse/pull/61604) ([Raúl Marín](https://github.com/Algunenano)). +* Remove some magic_enum from headers [#61606](https://github.com/ClickHouse/ClickHouse/pull/61606) ([Raúl Marín](https://github.com/Algunenano)). +* Fix configs for upgrade and bugfix [#61607](https://github.com/ClickHouse/ClickHouse/pull/61607) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add tests for multiple fuzzer issues [#61614](https://github.com/ClickHouse/ClickHouse/pull/61614) ([Raúl Marín](https://github.com/Algunenano)). +* Try to fix `02908_many_requests_to_system_replicas` again [#61616](https://github.com/ClickHouse/ClickHouse/pull/61616) ([Nikita Taranov](https://github.com/nickitat)). +* Verbose error message about analyzer_compatibility_join_using_top_level_identifier [#61631](https://github.com/ClickHouse/ClickHouse/pull/61631) ([vdimir](https://github.com/vdimir)). +* Fix 00223_shard_distributed_aggregation_memory_efficient with analyzer [#61649](https://github.com/ClickHouse/ClickHouse/pull/61649) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Better fuzzer logs [#61650](https://github.com/ClickHouse/ClickHouse/pull/61650) ([Raúl Marín](https://github.com/Algunenano)). +* Fix flaky `02122_parallel_formatting_Template` [#61651](https://github.com/ClickHouse/ClickHouse/pull/61651) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix Aggregator when data is empty [#61654](https://github.com/ClickHouse/ClickHouse/pull/61654) ([Antonio Andelic](https://github.com/antonio2368)). +* Restore poco SUN files [#61655](https://github.com/ClickHouse/ClickHouse/pull/61655) ([Andy Fiddaman](https://github.com/citrus-it)). +* Another fix for `SumIfToCountIfPass` [#61656](https://github.com/ClickHouse/ClickHouse/pull/61656) ([Antonio Andelic](https://github.com/antonio2368)). +* Keeper: fix data race during snapshot destructor call [#61657](https://github.com/ClickHouse/ClickHouse/pull/61657) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: integration tests: use runner as py module [#61658](https://github.com/ClickHouse/ClickHouse/pull/61658) ([Max K.](https://github.com/maxknv)). +* Fix logging of autoscaling lambda, add test for effective_capacity [#61662](https://github.com/ClickHouse/ClickHouse/pull/61662) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Small change in `DatabaseOnDisk::iterateMetadataFiles()` [#61664](https://github.com/ClickHouse/ClickHouse/pull/61664) ([Nikita Taranov](https://github.com/nickitat)). +* Build improvements by removing magic enum from header and apply some explicit template instantiation [#61665](https://github.com/ClickHouse/ClickHouse/pull/61665) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Update the dictionary for OSSFuzz [#61672](https://github.com/ClickHouse/ClickHouse/pull/61672) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Inhibit randomization in some tests and exclude some long tests from debug runs [#61676](https://github.com/ClickHouse/ClickHouse/pull/61676) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#61669](https://github.com/ClickHouse/ClickHouse/issues/61669) [#61678](https://github.com/ClickHouse/ClickHouse/pull/61678) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix use-of-uninitialized-value in HedgedConnections [#61679](https://github.com/ClickHouse/ClickHouse/pull/61679) ([Nikolay Degterinsky](https://github.com/evillique)). +* Remove clickhouse-diagnostics from the package [#61681](https://github.com/ClickHouse/ClickHouse/pull/61681) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix use-of-uninitialized-value in parseDateTimeBestEffort [#61694](https://github.com/ClickHouse/ClickHouse/pull/61694) ([Nikolay Degterinsky](https://github.com/evillique)). +* poco foundation: add illumos support [#61701](https://github.com/ClickHouse/ClickHouse/pull/61701) ([Andy Fiddaman](https://github.com/citrus-it)). +* contrib/c-ares: add illumos as a platform [#61702](https://github.com/ClickHouse/ClickHouse/pull/61702) ([Andy Fiddaman](https://github.com/citrus-it)). +* contrib/curl: Add illumos support [#61704](https://github.com/ClickHouse/ClickHouse/pull/61704) ([Andy Fiddaman](https://github.com/citrus-it)). +* Fuzzer: Try a different way to wait for the server [#61706](https://github.com/ClickHouse/ClickHouse/pull/61706) ([Raúl Marín](https://github.com/Algunenano)). +* Disable some tests for SMT [#61708](https://github.com/ClickHouse/ClickHouse/pull/61708) ([Raúl Marín](https://github.com/Algunenano)). +* Fix signal handler for sanitizer signals [#61709](https://github.com/ClickHouse/ClickHouse/pull/61709) ([Antonio Andelic](https://github.com/antonio2368)). +* Avoid `IsADirectoryError: Is a directory contrib/azure` [#61710](https://github.com/ClickHouse/ClickHouse/pull/61710) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Analyzer: fix group_by_use_nulls [#61717](https://github.com/ClickHouse/ClickHouse/pull/61717) ([Dmitry Novik](https://github.com/novikd)). +* Analyzer: Clear list of broken integration tests [#61718](https://github.com/ClickHouse/ClickHouse/pull/61718) ([Dmitry Novik](https://github.com/novikd)). +* CI: modify CI from PR body [#61725](https://github.com/ClickHouse/ClickHouse/pull/61725) ([Max K.](https://github.com/maxknv)). +* Add test for [#57820](https://github.com/ClickHouse/ClickHouse/issues/57820) [#61726](https://github.com/ClickHouse/ClickHouse/pull/61726) ([Dmitry Novik](https://github.com/novikd)). +* Revert "Revert "Un-flake test_undrop_query"" [#61727](https://github.com/ClickHouse/ClickHouse/pull/61727) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* FunctionsConversion: Start simplifying templates [#61733](https://github.com/ClickHouse/ClickHouse/pull/61733) ([Raúl Marín](https://github.com/Algunenano)). +* CI: modify it [#61735](https://github.com/ClickHouse/ClickHouse/pull/61735) ([Max K.](https://github.com/maxknv)). +* Fix segfault in SquashingTransform [#61736](https://github.com/ClickHouse/ClickHouse/pull/61736) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix DWARF format failing to skip DW_FORM_strx3 attributes [#61737](https://github.com/ClickHouse/ClickHouse/pull/61737) ([Michael Kolupaev](https://github.com/al13n321)). +* There is no such thing as broken tests [#61739](https://github.com/ClickHouse/ClickHouse/pull/61739) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Process removed files, decouple _check_mime [#61751](https://github.com/ClickHouse/ClickHouse/pull/61751) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Keeper fix: destroy `KeeperDispatcher` first [#61752](https://github.com/ClickHouse/ClickHouse/pull/61752) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky `03014_async_with_dedup_part_log_rmt` [#61757](https://github.com/ClickHouse/ClickHouse/pull/61757) ([Antonio Andelic](https://github.com/antonio2368)). +* FunctionsConversion: Remove another batch of bad templates [#61773](https://github.com/ClickHouse/ClickHouse/pull/61773) ([Raúl Marín](https://github.com/Algunenano)). +* Revert "Fix bug when reading system.parts using UUID (issue 61220)." [#61774](https://github.com/ClickHouse/ClickHouse/pull/61774) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* CI: disable grpc tests on ARM [#61778](https://github.com/ClickHouse/ClickHouse/pull/61778) ([Max K.](https://github.com/maxknv)). +* Fix more tests with virtual columns in StorageMerge. [#61787](https://github.com/ClickHouse/ClickHouse/pull/61787) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove already not flaky tests with analyzer. [#61788](https://github.com/ClickHouse/ClickHouse/pull/61788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Analyzer: Fix assert in JOIN with Distributed table [#61789](https://github.com/ClickHouse/ClickHouse/pull/61789) ([vdimir](https://github.com/vdimir)). +* A test can be slow in debug build [#61796](https://github.com/ClickHouse/ClickHouse/pull/61796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Updated clang-19 to master. [#61798](https://github.com/ClickHouse/ClickHouse/pull/61798) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test "00002_log_and_exception_messages_formatting" [#61821](https://github.com/ClickHouse/ClickHouse/pull/61821) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A test is too slow for debug [#61822](https://github.com/ClickHouse/ClickHouse/pull/61822) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove DataStreams [#61824](https://github.com/ClickHouse/ClickHouse/pull/61824) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better message for logging errors [#61827](https://github.com/ClickHouse/ClickHouse/pull/61827) ([Azat Khuzhin](https://github.com/azat)). +* Fix sanitizers suppressions [#61828](https://github.com/ClickHouse/ClickHouse/pull/61828) ([Azat Khuzhin](https://github.com/azat)). +* Remove unused code [#61830](https://github.com/ClickHouse/ClickHouse/pull/61830) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove DataStreams (2) [#61831](https://github.com/ClickHouse/ClickHouse/pull/61831) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update xxhash to v0.8.2 [#61838](https://github.com/ClickHouse/ClickHouse/pull/61838) ([Shubham Ranjan](https://github.com/shubhamranjan)). +* Fix: DISTINCT in subquery with analyzer [#61847](https://github.com/ClickHouse/ClickHouse/pull/61847) ([Igor Nikonov](https://github.com/devcrafter)). +* Analyzer: fix limit/offset on shards [#61849](https://github.com/ClickHouse/ClickHouse/pull/61849) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Remove PoolBase::AllocateNewBypassingPool [#61866](https://github.com/ClickHouse/ClickHouse/pull/61866) ([Azat Khuzhin](https://github.com/azat)). +* Try to fix 02901_parallel_replicas_rollup with analyzer. [#61875](https://github.com/ClickHouse/ClickHouse/pull/61875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add test for [#57808](https://github.com/ClickHouse/ClickHouse/issues/57808) [#61879](https://github.com/ClickHouse/ClickHouse/pull/61879) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* CI: merge queue support [#61881](https://github.com/ClickHouse/ClickHouse/pull/61881) ([Max K.](https://github.com/maxknv)). +* Update create.sql [#61885](https://github.com/ClickHouse/ClickHouse/pull/61885) ([Kseniia Sumarokova](https://github.com/kssenii)). +* no smaller unit in date_trunc [#61888](https://github.com/ClickHouse/ClickHouse/pull/61888) ([jsc0218](https://github.com/jsc0218)). +* Move KQL trash where it is supposed to be [#61903](https://github.com/ClickHouse/ClickHouse/pull/61903) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changelog for 24.3 [#61909](https://github.com/ClickHouse/ClickHouse/pull/61909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.3.22.3-lts [#61914](https://github.com/ClickHouse/ClickHouse/pull/61914) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.8.12.13-lts [#61915](https://github.com/ClickHouse/ClickHouse/pull/61915) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* No "please" [#61916](https://github.com/ClickHouse/ClickHouse/pull/61916) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.12.6.19-stable [#61917](https://github.com/ClickHouse/ClickHouse/pull/61917) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v24.1.8.22-stable [#61918](https://github.com/ClickHouse/ClickHouse/pull/61918) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix flaky test_broken_projestions/test.py::test_broken_ignored_replic… [#61932](https://github.com/ClickHouse/ClickHouse/pull/61932) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Check is Rust avaiable for build, if not, suggest a way to disable Rust support [#61938](https://github.com/ClickHouse/ClickHouse/pull/61938) ([Azat Khuzhin](https://github.com/azat)). +* CI: new ci menu in PR body [#61948](https://github.com/ClickHouse/ClickHouse/pull/61948) ([Max K.](https://github.com/maxknv)). +* Remove flaky test `01193_metadata_loading` [#61961](https://github.com/ClickHouse/ClickHouse/pull/61961) ([Nikita Taranov](https://github.com/nickitat)). + +#### Packaging Improvement + +* Adding the `--now` option to enable and start service automatically when installing the database server completely. [#60656](https://github.com/ClickHouse/ClickHouse/pull/60656) ([Chun-Sheng, Li](https://github.com/peter279k)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index e4482998fd6..ca1a23a99db 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v24.3.1.2672-lts 2024-03-27 v24.2.2.71-stable 2024-03-15 v24.2.1.2248-stable 2024-02-29 v24.1.8.22-stable 2024-03-26